diff --git a/.gitignore b/.gitignore index 798c94ffc..b5ad54644 100644 --- a/.gitignore +++ b/.gitignore @@ -11,7 +11,11 @@ .project .classpath .scala_dependencies +conf/*.conf +conf/*.properties +conf/*.sh lib_managed/ +logs/ src_managed/ target/ reports/ diff --git a/.travis.yml b/.travis.yml index 26976f393..e985c3150 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,8 +3,12 @@ dist: trusty language: scala env: - - MVN_FLAG=-Pspark-1.6 - - MVN_FLAG=-Pspark-2.0 + - MVN_FLAG='-Pspark-1.6 -DskipTests' + - MVN_FLAG='-Pspark-2.0 -DskipTests' + - MVN_FLAG='-Pspark-2.1 -DskipTests' + - MVN_FLAG='-Pspark-1.6 -DskipITs' + - MVN_FLAG='-Pspark-2.0 -DskipITs' + - MVN_FLAG='-Pspark-2.1 -DskipITs' jdk: - oraclejdk7 @@ -24,9 +28,10 @@ cache: before_install: - sudo apt-get -y install python3-pip python-dev + - sudo apt-get -y install libkrb5-dev - sudo apt-get -y remove python-setuptools - - pip install --user --upgrade pip setuptools - - pip3 install --user --upgrade pip setuptools + - pip install --user --upgrade pip "setuptools < 36" + - pip3 install --user --upgrade pip "setuptools < 36" - pip install --user codecov cloudpickle - pip3 install --user cloudpickle @@ -34,8 +39,8 @@ install: - mvn $MVN_FLAG install -Dskip -DskipTests -DskipITs -Dmaven.javadoc.skip=true -B -V before_script: - - pip install --user requests pytest flaky flake8 - - pip3 install --user requests pytest flaky + - pip install --user requests pytest flaky flake8 requests-kerberos + - pip3 install --user requests pytest flaky requests-kerberos script: - mvn $MVN_FLAG verify -e diff --git a/README.rst b/README.rst index b23136ad5..5e1fa13f7 100644 --- a/README.rst +++ b/README.rst @@ -1,6 +1,9 @@ Welcome to Livy =============== +.. image:: https://travis-ci.org/cloudera/livy.svg?branch=master + :target: https://travis-ci.org/cloudera/livy + Livy is an open source REST interface for interacting with `Apache Spark`_ from anywhere. It supports executing snippets of code or programs in a Spark context that runs locally or in `Apache Hadoop YARN`_. @@ -45,14 +48,21 @@ MacOS: Required python packages for building Livy: * cloudpickle * requests + * requests-kerberos * flake8 * flaky * pytest To run Livy, you will also need a Spark installation. You can get Spark releases at -https://spark.apache.org/downloads.html. Livy requires at least Spark 1.6 and currently -only supports Scala 2.10 builds of Spark. +https://spark.apache.org/downloads.html. + +Livy requires at least Spark 1.6 and supports both Scala 2.10 and 2.11 builds of Spark, Livy +will automatically pick repl dependencies through detecting the Scala version of Spark. + +Livy also supports Spark 2.0+ for both interactive and batch submission, you could seamlessly +switch to different versions of Spark through ``SPARK_HOME`` configuration, without needing to +rebuild Livy. Building Livy @@ -62,20 +72,15 @@ Livy is built using `Apache Maven`_. To check out and build Livy, run: .. code:: shell - git clone git@github.com:cloudera/livy.git + git clone https://github.com/cloudera/livy.git cd livy mvn package -By default Livy is built against the CDH 5.5 distribution of Spark (based off Spark 1.5.0). You can -build Livy against a different version of Spark by setting the ``spark.version`` property: - -.. code:: shell - - mvn -Dspark.version=1.6.1 package - -The version of Spark used when running Livy does not need to match the version used to build Livy. -The Livy package itself does not contain a Spark distribution, and will work with any supported -version of Spark. +By default Livy is built against Apache Spark 1.6.2, but the version of Spark used when running +Livy does not need to match the version used to build Livy. Livy internally uses reflection to +mitigate the gaps between different Spark versions, also Livy package itself does not +contain a Spark distribution, so it will work with any supported version of Spark (Spark 1.6+) +without needing to rebuild against specific version of Spark. .. _Apache Maven: http://maven.apache.org @@ -124,7 +129,6 @@ The configuration files used by Livy are: * ``log4j.properties``: configuration for Livy logging. Defines log levels and where log messages will be written to. The default configuration will print log messages to stderr. - Upgrade from Livy 0.1 ===================== @@ -154,9 +158,10 @@ A few things changed between since Livy 0.1 that require manual intervention whe Using the Programmatic API ========================== -Livy provides a programmatic Java API that allows applications to run code inside Spark without -having to maintain a local Spark context. To use the API, add the Cloudera repository to your -application's POM: +Livy provides a programmatic Java/Scala and Python API that allows applications to run code inside +Spark without having to maintain a local Spark context. Here shows how to use the Java API. + +Add the Cloudera repository to your application's POM: .. code:: xml @@ -322,7 +327,7 @@ Pi. This is from the `Spark Examples`_: .. code:: python data = { - 'code': textwrap.dedent("""\ + 'code': textwrap.dedent(""" val NUM_SAMPLES = 100000; val count = sc.parallelize(1 to NUM_SAMPLES).map { i => val x = Math.random(); @@ -336,6 +341,10 @@ Pi. This is from the `Spark Examples`_: r = requests.post(statements_url, data=json.dumps(data), headers=headers) pprint.pprint(r.json()) + statement_url = host + r.headers['location'] + r = requests.get(statement_url, headers=headers) + pprint.pprint(r.json()) + {u'id': 1, u'output': {u'data': {u'text/plain': u'Pi is roughly 3.14004\nNUM_SAMPLES: Int = 100000\ncount: Int = 78501'}, u'execution_count': 1, @@ -413,7 +422,7 @@ The Pi example from before then can be run as: .. code:: python data = { - 'code': textwrap.dedent("""\ + 'code': textwrap.dedent(""" n <- 100000 piFunc <- function(elem) { rands <- runif(n = 2, min = -1, max = 1) @@ -463,33 +472,72 @@ GET /sessions Returns all the active interactive sessions. +Request Parameters +^^^^^^^^^^^^^^^^^^ + ++------+-----------------------------------+------+ +| name | description | type | ++======+===================================+======+ +| from | The start index to fetch sessions | int | ++------+-----------------------------------+------+ +| size | Number of sessions to fetch | int | ++------+-----------------------------------+------+ + Response Body ^^^^^^^^^^^^^ -+----------+-----------------+------+ -| name | description | type | -+==========+=================+======+ -| sessions | `Session`_ list | list | -+----------+-----------------+------+ ++----------+-------------------------------------+------+ +| name | description | type | ++==========+=====================================+======+ +| from | The start index of fetched sessions | int | ++----------+-------------------------------------+------+ +| total | Number of sessions fetched | int | ++----------+-------------------------------------+------+ +| sessions | `Session`_ list | list | ++----------+-------------------------------------+------+ POST /sessions -------------- -Creates a new interative Scala, Python, or R shell in the cluster. +Creates a new interactive Scala, Python, or R shell in the cluster. Request Body ^^^^^^^^^^^^ -+----------------+------------------------------------------------+-----------------+ -| name | description | type | -+================+================================================+=================+ -| kind | The session kind (required) | `session kind`_ | -+----------------+------------------------------------------------+-----------------+ -| proxyUser | User to impersonate when starting the session | string | -+----------------+------------------------------------------------+-----------------+ -| conf | Spark configuration properties | Map of key=val | -+----------------+------------------------------------------------+-----------------+ ++--------------------------+------------------------------------------------+-----------------+ +| name | description | type | ++==========================+================================================+=================+ +| kind | The session kind (required) | `session kind`_ | ++--------------------------+------------------------------------------------+-----------------+ +| proxyUser | User to impersonate when starting the session | string | ++--------------------------+------------------------------------------------+-----------------+ +| jars | jars to be used in this session | List of string | ++--------------------------+------------------------------------------------+-----------------+ +| pyFiles | Python files to be used in this session | List of string | ++--------------------------+------------------------------------------------+-----------------+ +| files | files to be used in this session | List of string | ++--------------------------+------------------------------------------------+-----------------+ +| driverMemory | Amount of memory to use for the driver process | string | ++--------------------------+------------------------------------------------+-----------------+ +| driverCores | Number of cores to use for the driver process | int | ++--------------------------+------------------------------------------------+-----------------+ +| executorMemory | Amount of memory to use per executor process | string | ++--------------------------+------------------------------------------------+-----------------+ +| executorCores | Number of cores to use for each executor | int | ++--------------------------+------------------------------------------------+-----------------+ +| numExecutors | Number of executors to launch for this session | int | ++--------------------------+------------------------------------------------+-----------------+ +| archives | Archives to be used in this session | List of string | ++--------------------------+------------------------------------------------+-----------------+ +| queue | The name of the YARN queue to which submitted | string | ++--------------------------+------------------------------------------------+-----------------+ +| name | The name of this session | string | ++--------------------------+------------------------------------------------+-----------------+ +| conf | Spark configuration properties | Map of key=val | ++--------------------------+------------------------------------------------+-----------------+ +| heartbeatTimeoutInSecond | Timeout in second to which session be orphaned | int | ++--------------------------+------------------------------------------------+-----------------+ Response Body @@ -503,10 +551,27 @@ GET /sessions/{sessionId} Returns the session information. +Response Body +^^^^^^^^^^^^^ + +The `Session`_. + + +GET /sessions/{sessionId}/state +------------------------------- + +Returns the state of session + Response ^^^^^^^^ -The `Session`_. ++-------+-----------------------------------+--------+ +| name | description | type | ++=======+===================================+========+ +| id | Session id | int | ++-------+-----------------------------------+--------+ +| state | The current state of session | string | ++-------+-----------------------------------+--------+ DELETE /sessions/{sessionId} @@ -515,7 +580,7 @@ DELETE /sessions/{sessionId} Kills the `Session`_ job. -GET /sessions/{sessionId}/logs +GET /sessions/{sessionId}/log ------------------------------ Gets the log lines from this session. @@ -582,19 +647,60 @@ Response Body The `statement`_ object. +GET /sessions/{sessionId}/statements/{statementId} +-------------------------------------------------- + +Returns a specified statement in a session. + +Response Body +^^^^^^^^^^^^^ + +The `statement`_ object. + + +POST /sessions/{sessionId}/statements/{statementId}/cancel +---------------------------------------------------------- + +Cancel the specified statement in this session. + +Response Body +^^^^^^^^^^^^^ + ++------+----------------------------+--------+ +| name | description | type | ++======+============================+========+ +| msg | is always "cancelled" | string | ++------+----------------------------+--------+ + + GET /batches ------------- +------------- + +Returns all the active batch sessions. + +Request Parameters +^^^^^^^^^^^^^^^^^^ -Returns all the active batch jobs. ++------+-----------------------------------+------+ +| name | description | type | ++======+===================================+======+ +| from | The start index to fetch sessions | int | ++------+-----------------------------------+------+ +| size | Number of sessions to fetch | int | ++------+-----------------------------------+------+ Response Body ^^^^^^^^^^^^^ -+---------+---------------+------+ -| name | description | type | -+=========+===============+======+ -|sessions | `batch`_ list | list | -+---------+---------------+------+ ++----------+-------------------------------------+------+ +| name | description | type | ++==========+=====================================+======+ +| from | The start index of fetched sessions | int | ++----------+-------------------------------------+------+ +| total | Number of sessions fetched | int | ++----------+-------------------------------------+------+ +| sessions | `Batch`_ list | list | ++----------+-------------------------------------+------+ POST /batches @@ -603,20 +709,41 @@ POST /batches Request Body ^^^^^^^^^^^^ -+-------------+---------------------------------------------------+-----------------+ -| name | description | type | -+=============+===================================================+=================+ -| file | File containing the application to execute | path (required) | -+-------------+---------------------------------------------------+-----------------+ -| proxyUser | User to impersonate when running the job | string | -+-------------+---------------------------------------------------+-----------------+ -| className | Application Java/Spark main class | string | -+-------------+---------------------------------------------------+-----------------+ -| args | Command line arguments for the application | list of strings | -+-------------+---------------------------------------------------+-----------------+ -| conf | Spark configuration properties | Map of key=val | -+-------------+---------------------------------------------------+-----------------+ - ++----------------+---------------------------------------------------+-----------------+ +| name | description | type | ++================+===================================================+=================+ +| file | File containing the application to execute | path (required) | ++----------------+---------------------------------------------------+-----------------+ +| proxyUser | User to impersonate when running the job | string | ++----------------+---------------------------------------------------+-----------------+ +| className | Application Java/Spark main class | string | ++----------------+---------------------------------------------------+-----------------+ +| args | Command line arguments for the application | list of strings | ++----------------+---------------------------------------------------+-----------------+ +| jars | jars to be used in this session | List of string | ++----------------+---------------------------------------------------+-----------------+ +| pyFiles | Python files to be used in this session | List of string | ++----------------+---------------------------------------------------+-----------------+ +| files | files to be used in this session | List of string | ++----------------+---------------------------------------------------+-----------------+ +| driverMemory | Amount of memory to use for the driver process | string | ++----------------+---------------------------------------------------+-----------------+ +| driverCores | Number of cores to use for the driver process | int | ++----------------+---------------------------------------------------+-----------------+ +| executorMemory | Amount of memory to use per executor process | string | ++----------------+---------------------------------------------------+-----------------+ +| executorCores | Number of cores to use for each executor | int | ++----------------+---------------------------------------------------+-----------------+ +| numExecutors | Number of executors to launch for this session | int | ++----------------+---------------------------------------------------+-----------------+ +| archives | Archives to be used in this session | List of string | ++----------------+---------------------------------------------------+-----------------+ +| queue | The name of the YARN queue to which submitted | string | ++----------------+---------------------------------------------------+-----------------+ +| name | The name of this session | string | ++----------------+---------------------------------------------------+-----------------+ +| conf | Spark configuration properties | Map of key=val | ++----------------+---------------------------------------------------+-----------------+ Response Body ^^^^^^^^^^^^^ @@ -627,29 +754,29 @@ The created `Batch`_ object. GET /batches/{batchId} ---------------------- -Request Parameters -^^^^^^^^^^^^^^^^^^ - -+------+---------------------------------+------+ -| name | description | type | -+======+=================================+======+ -| from | Offset | int | -+------+---------------------------------+------+ -| size | Max number of batches to return | int | -+------+---------------------------------+------+ +Returns the batch session information. Response Body ^^^^^^^^^^^^^ -+-------+-----------------------------+-----------------+ -| name | description | type | -+=======+=============================+=================+ -| id | The batch id | int | -+-------+-----------------------------+-----------------+ -| state | The state of the batch | `batch`_ state | -+-------+-----------------------------+-----------------+ -| log | The output of the batch job | list of strings | -+-------+-----------------------------+-----------------+ +The `Batch`_. + + +GET /batches/{batchId}/state +---------------------------- + +Returns the state of batch session + +Response +^^^^^^^^ + ++-------+-----------------------------------+--------+ +| name | description | type | ++=======+===================================+========+ +| id | Batch session id | int | ++-------+-----------------------------------+--------+ +| state | The current state of batch session| string | ++-------+-----------------------------------+--------+ DELETE /batches/{batchId} @@ -659,7 +786,7 @@ Kills the `Batch`_ job. GET /batches/{batchId}/log ---------------------------- +-------------------------- Gets the log lines from this batch. @@ -703,45 +830,79 @@ A session represents an interactive shell. +================+==========================================+============================+ | id | The session id | int | +----------------+------------------------------------------+----------------------------+ -| kind | Session kind (spark, pyspark, or sparkr) | `session kind`_ (required) | +| appId | The application id of this session | String | ++----------------+------------------------------------------+----------------------------+ +| owner | Remote user who submitted this session | String | ++----------------+------------------------------------------+----------------------------+ +| proxyUser | User to impersonate when running | String | ++----------------+------------------------------------------+----------------------------+ +| kind | Session kind (spark, pyspark, or sparkr) | `session kind`_ | +----------------+------------------------------------------+----------------------------+ | log | The log lines | list of strings | +----------------+------------------------------------------+----------------------------+ | state | The session state | string | +----------------+------------------------------------------+----------------------------+ +| appInfo | The detailed application info | Map of key=val | ++----------------+------------------------------------------+----------------------------+ Session State ^^^^^^^^^^^^^ -+-------------+----------------------------------+ -| value | description | -+=============+==================================+ -| not_started | Session has not been started | -+-------------+----------------------------------+ -| starting | Session is starting | -+-------------+----------------------------------+ -| idle | Session is waiting for input | -+-------------+----------------------------------+ -| busy | Session is executing a statement | -+-------------+----------------------------------+ -| error | Session errored out | -+-------------+----------------------------------+ -| dead | Session has exited | -+-------------+----------------------------------+ ++---------------+----------------------------------+ +| value | description | ++===============+==================================+ +| not_started | Session has not been started | ++---------------+----------------------------------+ +| starting | Session is starting | ++---------------+----------------------------------+ +| idle | Session is waiting for input | ++---------------+----------------------------------+ +| busy | Session is executing a statement | ++---------------+----------------------------------+ +| shutting_down | Session is shutting down | ++---------------+----------------------------------+ +| error | Session errored out | ++---------------+----------------------------------+ +| dead | Session has exited | ++---------------+----------------------------------+ +| success | Session is successfully stopped | ++---------------+----------------------------------+ Session Kind ^^^^^^^^^^^^ -+---------+----------------------------------+ -| value | description | -+=========+==================================+ -| spark | Interactive Scala Spark session | -+---------+----------------------------------+ -| pyspark | Interactive Python Spark session | -+---------+----------------------------------+ -| sparkr | Interactive R Spark session | -+---------+----------------------------------+ ++-------------+------------------------------------+ +| value | description | ++=============+====================================+ +| spark | Interactive Scala Spark session | ++-------------+------------------------------------+ +| `pyspark`_ | Interactive Python 2 Spark session | ++-------------+------------------------------------+ +| `pyspark3`_ | Interactive Python 3 Spark session | ++-------------+------------------------------------+ +| sparkr | Interactive R Spark session | ++-------------+------------------------------------+ + +pyspark +^^^^^^^ +To change the Python executable the session uses, Livy reads the path from environment variable +``PYSPARK_PYTHON`` (Same as pyspark). + +Like pyspark, if Livy is running in ``local`` mode, just set the environment variable. +If the session is running in ``yarn-cluster`` mode, please set +``spark.yarn.appMasterEnv.PYSPARK_PYTHON`` in SparkConf so the environment variable is passed to +the driver. + +pyspark3 +^^^^^^^^ +To change the Python executable the session uses, Livy reads the path from environment variable +``PYSPARK3_PYTHON``. + +Like pyspark, if Livy is running in ``local`` mode, just set the environment variable. +If the session is running in ``yarn-cluster`` mode, please set +``spark.yarn.appMasterEnv.PYSPARK3_PYTHON`` in SparkConf so the environment variable is passed to +the driver. Statement --------- @@ -761,15 +922,21 @@ A statement represents the result of an execution statement. Statement State ^^^^^^^^^^^^^^^ -+-----------+----------------------------------+ -| value | description | -+===========+==================================+ -| running | Statement is currently running | -+-----------+----------------------------------+ -| available | Statement has a response ready | -+-----------+----------------------------------+ -| error | Statement failed | -+-----------+----------------------------------+ ++------------+----------------------------------------------------+ +| value | description | ++============+====================================================+ +| waiting | Statement is enqueued but execution hasn't started | ++------------+----------------------------------------------------+ +| running | Statement is currently running | ++------------+----------------------------------------------------+ +| available | Statement has a response ready | ++------------+----------------------------------------------------+ +| error | Statement failed | ++------------+----------------------------------------------------+ +| cancelling | Statement is being cancelling | ++------------+----------------------------------------------------+ +| cancelled | Statement is cancelled | ++------------+----------------------------------------------------+ Statement Output ^^^^^^^^^^^^^^^^ @@ -779,7 +946,7 @@ Statement Output +=================+===================+==================================+ | status | Execution status | string | +-----------------+-------------------+----------------------------------+ -| execution_count | A monotomically | integer | +| execution_count | A monotonically | integer | | | increasing number | | +-----------------+-------------------+----------------------------------+ | data | Statement output | An object mapping a mime type to | @@ -791,15 +958,19 @@ Statement Output Batch ----- -+----------------+------------------+----------------------------+ -| name | description | type | -+================+==================+============================+ -| id | The session id | int | -+----------------+------------------+----------------------------+ -| log | The log lines | list of strings | -+----------------+------------------+----------------------------+ -| state | The batch state | string | -+----------------+------------------+----------------------------+ ++----------------+-------------------------------------+-----------------+ +| name | description | type | ++================+=====================================+=================+ +| id | The session id | int | ++----------------+-------------------------------------+-----------------+ +| appId | The application id of this session | String | ++----------------+-------------------------------------+-----------------+ +| appInfo | The detailed application info | Map of key=val | ++----------------+-------------------------------------+-----------------+ +| log | The log lines | list of strings | ++----------------+-------------------------------------+-----------------+ +| state | The batch state | string | ++----------------+-------------------------------------+-----------------+ License diff --git a/api/pom.xml b/api/pom.xml index d196cc5b5..156d9872d 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -20,12 +20,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT com.cloudera.livy livy-api - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar diff --git a/api/src/main/java/com/cloudera/livy/JobContext.java b/api/src/main/java/com/cloudera/livy/JobContext.java index 21de2a3c2..d32adef15 100644 --- a/api/src/main/java/com/cloudera/livy/JobContext.java +++ b/api/src/main/java/com/cloudera/livy/JobContext.java @@ -38,10 +38,10 @@ public interface JobContext { /** The shared SparkContext instance. */ JavaSparkContext sc(); - /** The shared SQLContext inststance. */ + /** The shared SQLContext instance. */ SQLContext sqlctx(); - /** The shared HiveContext inststance. */ + /** The shared HiveContext instance. */ HiveContext hivectx(); /** Returns the JavaStreamingContext which has already been created. */ @@ -63,4 +63,8 @@ public interface JobContext { */ File getLocalTmpDir(); + /** + * Returns SparkSession if it existed, otherwise throws Exception. + */ + E sparkSession() throws Exception; } diff --git a/assembly/pom.xml b/assembly/pom.xml index e9df37727..87da4e5e1 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,12 +21,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml livy-assembly - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom diff --git a/bin/livy-server b/bin/livy-server index a2636ba4f..71b913440 100755 --- a/bin/livy-server +++ b/bin/livy-server @@ -108,7 +108,7 @@ start_livy_server() { if [ -f "$pid" ]; then TARGET_ID="$(cat "$pid")" if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then - echo "livy_server running as process $TARGET_ID. Stop it first." + echo "livy-server running as process $TARGET_ID. Stop it first." exit 1 fi fi @@ -125,6 +125,7 @@ start_livy_server() { tail -2 "$log" | sed 's/^/ /' echo "full log in $log" rm -rf "$pid" + exit 1 fi fi } @@ -146,13 +147,26 @@ case $option in if [ -f $pid ]; then TARGET_ID="$(cat "$pid")" if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then - echo "stopping livy_server" + echo "stopping livy-server" kill "$TARGET_ID" && rm -f "$pid" else - echo "no livy_server to stop" + echo "no livy-server to stop" fi else - echo "no livy_server to stop" + echo "no livy-server to stop" + fi + ;; + + (status) + if [ -f "$pid" ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo "livy-server is running (pid: $TARGET_ID)" + else + echo "livy-server is not running" + fi + else + echo "livy-server is not running" fi ;; diff --git a/client-common/pom.xml b/client-common/pom.xml index 810be59ad..e41aaae51 100644 --- a/client-common/pom.xml +++ b/client-common/pom.xml @@ -21,12 +21,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT com.cloudera.livy livy-client-common - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar @@ -45,5 +45,10 @@ jackson-databind test + + org.slf4j + slf4j-api + provided + diff --git a/client-common/src/main/java/com/cloudera/livy/client/common/ClientConf.java b/client-common/src/main/java/com/cloudera/livy/client/common/ClientConf.java index 59cce37ca..4eb79294d 100644 --- a/client-common/src/main/java/com/cloudera/livy/client/common/ClientConf.java +++ b/client-common/src/main/java/com/cloudera/livy/client/common/ClientConf.java @@ -28,6 +28,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.cloudera.livy.annotations.Private; /** @@ -37,6 +40,8 @@ public abstract class ClientConf implements Iterable> { + protected Logger LOG = LoggerFactory.getLogger(getClass()); + public static interface ConfEntry { /** The key in the configuration file. */ @@ -71,24 +76,37 @@ protected ClientConf(Properties config) { this.config = new ConcurrentHashMap<>(); if (config != null) { for (String key : config.stringPropertyNames()) { + logDeprecationWarning(key); this.config.put(key, config.getProperty(key)); } } } public String get(String key) { - return config.get(key); + String val = config.get(key); + if (val != null) { + return val; + } + DeprecatedConf depConf = getConfigsWithAlternatives().get(key); + if (depConf != null) { + return config.get(depConf.key()); + } else { + return val; + } } @SuppressWarnings("unchecked") public T set(String key, String value) { + logDeprecationWarning(key); config.put(key, value); return (T) this; } @SuppressWarnings("unchecked") public T setIfMissing(String key, String value) { - config.putIfAbsent(key, value); + if (config.putIfAbsent(key, value) == null) { + logDeprecationWarning(key); + } return (T) this; } @@ -163,6 +181,7 @@ public T set(ConfEntry e, Object value) { if (value == null) { config.remove(e.key()); } else { + logDeprecationWarning(e.key()); config.put(e.key(), value.toString()); } return (T) this; @@ -176,7 +195,7 @@ public Iterator> iterator() { private String get(ConfEntry e, Class requestedType) { check(getType(e.dflt()).equals(requestedType), "Invalid type conversion requested for %s.", e.key()); - return config.get(e.key()); + return this.get(e.key()); } private boolean typesMatch(Object test, Object expected) { @@ -193,4 +212,40 @@ private void check(boolean test, String message, Object... args) { } } + /** Logs a warning message if the given config key is deprecated. */ + private void logDeprecationWarning(String key) { + DeprecatedConf altConfs = getConfigsWithAlternatives().get(key); + if (altConfs != null) { + LOG.warn("The configuration key " + altConfs.key() + " has been deprecated as of Livy " + + altConfs.version() + " and may be removed in the future. Please use the new key " + + key + " instead."); + return; + } + + DeprecatedConf depConfs = getDeprecatedConfigs().get(key); + if (depConfs != null) { + LOG.warn("The configuration key " + depConfs.key() + " has been deprecated as of Livy " + + depConfs.version() + " and may be removed in the future. " + + depConfs.deprecationMessage()); + } + } + + /** Maps valid key to DeprecatedConf with the deprecated key. */ + protected abstract Map getConfigsWithAlternatives(); + + /** Maps deprecated key to DeprecatedConf with the same key. */ + protected abstract Map getDeprecatedConfigs(); + + public static interface DeprecatedConf { + + /** The key in the configuration file. */ + String key(); + + /** The Livy version in which the key was deprecated. */ + String version(); + + /** Message to include in the deprecation warning for configs without alternatives */ + String deprecationMessage(); + } + } diff --git a/client-common/src/test/java/com/cloudera/livy/client/common/TestClientConf.java b/client-common/src/test/java/com/cloudera/livy/client/common/TestClientConf.java index c8bd9c9aa..afb779827 100644 --- a/client-common/src/test/java/com/cloudera/livy/client/common/TestClientConf.java +++ b/client-common/src/test/java/com/cloudera/livy/client/common/TestClientConf.java @@ -18,6 +18,8 @@ package com.cloudera.livy.client.common; +import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -122,6 +124,31 @@ public void testTimeWithoutDefault() { conf.getTimeAsMs(TestConf.Entry.TIME_NO_DEFAULT); } + + @Test + public void testDeprecation() { + TestConf conf = new TestConf(null); + + assertNull(conf.get("depKey")); + assertNull(conf.get("dep_alt")); + assertNull(conf.get("new-key")); + assertEquals("value", conf.get(TestConf.Entry.NEW_CONF)); + + TestConf depProps = new TestConf(null); + depProps.set("depKey", "dep-val"); + depProps.set("dep_alt", "alt-val"); + conf.setAll(depProps); + assertEquals("dep-val", conf.get("depKey")); + assertEquals("alt-val", conf.get("dep_alt")); + assertEquals("alt-val", conf.get(TestConf.Entry.NEW_CONF)); + assertEquals("alt-val", conf.get("new-key")); + + conf.set("new-key", "new-val"); + assertEquals("new-val", conf.get(TestConf.Entry.NEW_CONF)); + assertEquals("alt-val", conf.get("dep_alt")); + assertEquals("new-val", conf.get("new-key")); + } + private static class TestConf extends ClientConf { static enum Entry implements ConfEntry { @@ -131,7 +158,8 @@ static enum Entry implements ConfEntry { INT("int", 42), LONG("long", 84L), TIME("time", "168ms"), - TIME_NO_DEFAULT("time2", null); + TIME_NO_DEFAULT("time2", null), + NEW_CONF("new-key", "value"); private final String key; private final Object dflt; @@ -153,6 +181,51 @@ private Entry(String key, Object dflt) { super(p); } + private static final Map configsWithAlternatives + = Collections.unmodifiableMap(new HashMap() {{ + put(TestConf.Entry.NEW_CONF.key, DepConf.DEP_WITH_ALT); + }}); + + private static final Map deprecatedConfigs + = Collections.unmodifiableMap(new HashMap() {{ + put(DepConf.DEP_NO_ALT.key, DepConf.DEP_NO_ALT); + }}); + + protected Map getConfigsWithAlternatives() { + return configsWithAlternatives; + } + + protected Map getDeprecatedConfigs() { + return deprecatedConfigs; + } + + static enum DepConf implements DeprecatedConf { + DEP_WITH_ALT("dep_alt", "0.4"), + DEP_NO_ALT("depKey", "1.0"); + + private final String key; + private final String version; + private final String deprecationMessage; + + private DepConf(String key, String version) { + this(key, version, ""); + } + + private DepConf(String key, String version, String deprecationMessage) { + this.key = key; + this.version = version; + this.deprecationMessage = deprecationMessage; + } + + @Override + public String key() { return key; } + + @Override + public String version() { return version; } + + @Override + public String deprecationMessage() { return deprecationMessage; } + } } } diff --git a/client-http/pom.xml b/client-http/pom.xml index 4c22ddb8d..478e2227a 100644 --- a/client-http/pom.xml +++ b/client-http/pom.xml @@ -21,12 +21,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT com.cloudera.livy livy-client-http - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar diff --git a/client-http/src/main/java/com/cloudera/livy/client/http/HttpConf.java b/client-http/src/main/java/com/cloudera/livy/client/http/HttpConf.java index d8f661778..2ae25e574 100644 --- a/client-http/src/main/java/com/cloudera/livy/client/http/HttpConf.java +++ b/client-http/src/main/java/com/cloudera/livy/client/http/HttpConf.java @@ -18,25 +18,38 @@ package com.cloudera.livy.client.http; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; import com.cloudera.livy.client.common.ClientConf; class HttpConf extends ClientConf { + private static final String HTTP_CONF_PREFIX = "livy.client.http."; + static enum Entry implements ConfEntry { - CONNETION_TIMEOUT("connection.timeout", "10s"), + CONNECTION_TIMEOUT("connection.timeout", "10s"), CONNECTION_IDLE_TIMEOUT("connection.idle.timeout", "10m"), SOCKET_TIMEOUT("connection.socket.timeout", "5m"), - JOB_INITIAL_POLL_INTERVAL("job.initial_poll_interval", "100ms"), - JOB_MAX_POLL_INTERVAL("job.max_poll_interval", "5s"); + JOB_INITIAL_POLL_INTERVAL("job.initial-poll-interval", "100ms"), + JOB_MAX_POLL_INTERVAL("job.max-poll-interval", "5s"), + + CONTENT_COMPRESS_ENABLE("content.compress.enable", true), + + // Kerberos related configuration + SPNEGO_ENABLED("spnego.enable", false), + AUTH_LOGIN_CONFIG("auth.login.config", null), + KRB5_DEBUG_ENABLED("krb5.debug", false), + KRB5_CONF("krb5.conf", null); private final String key; private final Object dflt; private Entry(String key, Object dflt) { - this.key = "livy.client.http." + key; + this.key = HTTP_CONF_PREFIX + key; this.dflt = dflt; } @@ -49,6 +62,75 @@ private Entry(String key, Object dflt) { HttpConf(Properties config) { super(config); + + if (getBoolean(Entry.SPNEGO_ENABLED)) { + if (get(Entry.AUTH_LOGIN_CONFIG ) == null) { + throw new IllegalArgumentException(Entry.AUTH_LOGIN_CONFIG.key + " should not be null"); + } + + if (get(Entry.KRB5_CONF) == null) { + throw new IllegalArgumentException(Entry.KRB5_CONF.key + " should not be null"); + } + + System.setProperty("java.security.auth.login.config", get(Entry.AUTH_LOGIN_CONFIG)); + System.setProperty("java.security.krb5.conf", get(Entry.KRB5_CONF)); + System.setProperty( + "sun.security.krb5.debug", String.valueOf(getBoolean(Entry.KRB5_DEBUG_ENABLED))); + // This is needed to get Kerberos credentials from the environment, instead of + // requiring the application to manually obtain the credentials. + System.setProperty("javax.security.auth.useSubjectCredsOnly", "false"); + } + } + + boolean isSpnegoEnabled() { + return getBoolean(Entry.SPNEGO_ENABLED); + } + + private static final Map configsWithAlternatives + = Collections.unmodifiableMap(new HashMap() {{ + put(HttpConf.Entry.JOB_INITIAL_POLL_INTERVAL.key, DepConf.JOB_INITIAL_POLL_INTERVAL); + put(HttpConf.Entry.JOB_MAX_POLL_INTERVAL.key, DepConf.JOB_MAX_POLL_INTERVAL); + }}); + + // Maps deprecated key to DeprecatedConf with the same key. + // There are no deprecated configs without alternatives currently. + private static final Map deprecatedConfigs + = Collections.unmodifiableMap(new HashMap()); + + protected Map getConfigsWithAlternatives() { + return configsWithAlternatives; + } + + protected Map getDeprecatedConfigs() { + return deprecatedConfigs; + } + + static enum DepConf implements DeprecatedConf { + JOB_INITIAL_POLL_INTERVAL("job.initial_poll_interval", "0.4"), + JOB_MAX_POLL_INTERVAL("job.max_poll_interval", "0.4"); + + private final String key; + private final String version; + private final String deprecationMessage; + + private DepConf(String key, String version) { + this(key, version, ""); + } + + private DepConf(String key, String version, String deprecationMessage) { + this.key = HTTP_CONF_PREFIX + key; + this.version = version; + this.deprecationMessage = deprecationMessage; + } + + @Override + public String key() { return key; } + + @Override + public String version() { return version; } + + @Override + public String deprecationMessage() { return deprecationMessage; } } } diff --git a/client-http/src/main/java/com/cloudera/livy/client/http/LivyConnection.java b/client-http/src/main/java/com/cloudera/livy/client/http/LivyConnection.java index 12decc029..851d8d6dc 100644 --- a/client-http/src/main/java/com/cloudera/livy/client/http/LivyConnection.java +++ b/client-http/src/main/java/com/cloudera/livy/client/http/LivyConnection.java @@ -21,24 +21,31 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; import java.security.Principal; import java.util.concurrent.TimeUnit; -import static java.nio.charset.StandardCharsets.UTF_8; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.http.HttpEntity; import org.apache.http.HttpHeaders; import org.apache.http.HttpStatus; +import org.apache.http.auth.AuthSchemeProvider; import org.apache.http.auth.AuthScope; import org.apache.http.auth.Credentials; +import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; +import org.apache.http.client.config.AuthSchemes; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.*; import org.apache.http.client.protocol.HttpClientContext; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.mime.MultipartEntityBuilder; import org.apache.http.entity.mime.content.FileBody; import org.apache.http.impl.NoConnectionReuseStrategy; +import org.apache.http.impl.auth.SPNegoSchemeFactory; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; @@ -71,7 +78,7 @@ class LivyConnection { RequestConfig reqConfig = new RequestConfig() { @Override public int getConnectTimeout() { - return (int) config.getTimeAsMs(CONNETION_TIMEOUT); + return (int) config.getTimeAsMs(CONNECTION_TIMEOUT); } @Override @@ -83,26 +90,46 @@ public int getSocketTimeout() { public boolean isAuthenticationEnabled() { return true; } - }; - Credentials dummyCredentials = new Credentials() { @Override - public String getPassword() { - return null; + public boolean isContentCompressionEnabled() { + return config.getBoolean(CONTENT_COMPRESS_ENABLE); } + }; - @Override - public Principal getUserPrincipal() { - return null; + Credentials credentials; + // If user info is specified in the url, pass them to the CredentialsProvider. + if (uri.getUserInfo() != null) { + String[] userInfo = uri.getUserInfo().split(":"); + if (userInfo.length < 1) { + throw new IllegalArgumentException("Malformed user info in the url."); } - }; + try { + String username = URLDecoder.decode(userInfo[0], StandardCharsets.UTF_8.name()); + String password = ""; + if (userInfo.length > 1) { + password = URLDecoder.decode(userInfo[1], StandardCharsets.UTF_8.name()); + } + credentials = new UsernamePasswordCredentials(username, password); + } catch (Exception e) { + throw new IllegalArgumentException("User info in the url contains bad characters.", e); + } + } else { + credentials = new Credentials() { + @Override + public String getPassword() { + return null; + } - // This is needed to get Kerberos credentials from the environment, instead of - // requiring the application to manually obtain the credentials. - System.setProperty("javax.security.auth.useSubjectCredsOnly", "false"); + @Override + public Principal getUserPrincipal() { + return null; + } + }; + } CredentialsProvider credsProvider = new BasicCredentialsProvider(); - credsProvider.setCredentials(AuthScope.ANY, dummyCredentials); + credsProvider.setCredentials(AuthScope.ANY, credentials); HttpClientBuilder builder = HttpClientBuilder.create() .disableAutomaticRetries() @@ -115,6 +142,14 @@ public Principal getUserPrincipal() { .setDefaultCredentialsProvider(credsProvider) .setUserAgent("livy-client-http"); + if (config.isSpnegoEnabled()) { + Registry authSchemeProviderRegistry = + RegistryBuilder.create() + .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory()) + .build(); + builder.setDefaultAuthSchemeRegistry(authSchemeProviderRegistry); + } + this.server = uri; this.client = builder.build(); this.mapper = new ObjectMapper(); diff --git a/client-http/src/test/scala/com/cloudera/livy/client/http/LivyConnectionSpec.scala b/client-http/src/test/scala/com/cloudera/livy/client/http/LivyConnectionSpec.scala new file mode 100644 index 000000000..4e3610692 --- /dev/null +++ b/client-http/src/test/scala/com/cloudera/livy/client/http/LivyConnectionSpec.scala @@ -0,0 +1,119 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.client.http + +import java.io.IOException +import java.net.URLEncoder +import java.nio.charset.StandardCharsets.UTF_8 + +import org.apache.http.client.utils.URIBuilder +import org.eclipse.jetty.security._ +import org.eclipse.jetty.security.authentication.BasicAuthenticator +import org.eclipse.jetty.util.security._ +import org.scalatest.{BeforeAndAfterAll, FunSpecLike} +import org.scalatest.Matchers._ +import org.scalatra.servlet.ScalatraListener + +import com.cloudera.livy.{LivyBaseUnitTestSuite, LivyConf} +import com.cloudera.livy.server.WebServer + +class LivyConnectionSpec extends FunSpecLike with BeforeAndAfterAll with LivyBaseUnitTestSuite { + describe("LivyConnection") { + def basicAuth(username: String, password: String, realm: String): SecurityHandler = { + val roles = Array("user") + + val l = new HashLoginService() + l.putUser(username, Credential.getCredential(password), roles) + l.setName(realm) + + val constraint = new Constraint() + constraint.setName(Constraint.__BASIC_AUTH) + constraint.setRoles(roles) + constraint.setAuthenticate(true) + + val cm = new ConstraintMapping() + cm.setConstraint(constraint) + cm.setPathSpec("/*") + + val csh = new ConstraintSecurityHandler() + csh.setAuthenticator(new BasicAuthenticator()) + csh.setRealmName(realm) + csh.addConstraintMapping(cm) + csh.setLoginService(l) + + csh + } + + def test(password: String, livyConf: LivyConf = new LivyConf()): Unit = { + val username = "user name" + + val server = new WebServer(livyConf, "0.0.0.0", 0) + server.context.setSecurityHandler(basicAuth(username, password, "realm")) + server.context.setResourceBase("src/main/com/cloudera/livy/server") + server.context.setInitParameter(ScalatraListener.LifeCycleKey, + classOf[HttpClientTestBootstrap].getCanonicalName) + server.context.addEventListener(new ScalatraListener) + server.start() + + val utf8Name = UTF_8.name() + val uri = new URIBuilder() + .setScheme(server.protocol) + .setHost(server.host) + .setPort(server.port) + .setUserInfo(URLEncoder.encode(username, utf8Name), URLEncoder.encode(password, utf8Name)) + .build() + info(uri.toString) + val conn = new LivyConnection(uri, new HttpConf(null)) + try { + conn.get(classOf[Object], "/") should not be (null) + + } finally { + conn.close() + } + + server.stop() + server.join() + } + + it("should support HTTP auth with password") { + test("pass:word") + } + + it("should support HTTP auth with empty password") { + test("") + } + + it("should be failed with large header size") { + val livyConf = new LivyConf() + .set(LivyConf.REQUEST_HEADER_SIZE, 1024) + .set(LivyConf.RESPONSE_HEADER_SIZE, 1024) + val pwd = "test-password" * 100 + val exception = intercept[IOException](test(pwd, livyConf)) + exception.getMessage.contains("Request Entity Too Large") should be(true) + } + + it("should be succeeded with configured header size") { + val livyConf = new LivyConf() + .set(LivyConf.REQUEST_HEADER_SIZE, 2048) + .set(LivyConf.RESPONSE_HEADER_SIZE, 2048) + val pwd = "test-password" * 100 + test(pwd, livyConf) + } + } +} diff --git a/conf/livy-client.conf.template b/conf/livy-client.conf.template new file mode 100644 index 000000000..06ad653c5 --- /dev/null +++ b/conf/livy-client.conf.template @@ -0,0 +1,87 @@ +# +# Configurations for a Livy Client, any configurations set here will override any +# livy or spark-default configurations. +# +# Before a Livy Client is able to load these configurations the folder containing +# this file must be added to the application classpath +# + +# +# Configurations for Livy HTTPClient +# + +# HTTP Request configurations +# How long before a request times out +# livy.client.http.connection.timeout = 10s +# How long between data packets before a request times out +# livy.client.http.connection.socket.timeout = 5m +# Whether content is compressed +# livy.client.http.content.compress.enable = true + +# How long before idle connections are closed +# livy.client.http.connection.idle.timeout = 10m + +# Initial interval before polling for Job results +# livy.client.http.job.initial-poll-interval = 100ms +# Maximum interval between successive polls +# livy.client.http.job.max-poll-interval = 5s + +# +# Configurations for Livy RSCClient +# + +# Configurations for registering a client with the rpc server +# Unique client id for connections to the rpc server +# livy.rsc.client.auth.id = +# Secret value for authenticating client connections with server +# livy.rsc.client.auth.secret = + +# Timeout when stopping a rsc client +# livy.rsc.client.shutdown-timeout = 10s + +# Class of the rsc driver to use +# livy.rsc.driver-class = +# The kind of rsc session. Examples: pyspark or sparkr +# livy.rsc.session.kind = + +# Comma-separated list of Livy RSC jars. By default Livy will upload jars from its installation +# directory every time a session is started. By caching these files in HDFS, for example, startup +# time of sessions on YARN can be reduced. +# livy.rsc.jars = +# Location of the SparkR package for running sparkr +# livy.rsc.sparkr.package = +# Location of the PySpark package for running pyspark +# livy.rsc.pyspark.archives = + +# Address for the RSC driver to connect back with it's connection info. +# livy.rsc.launcher.address = +# Port Range on which RPC will launch . Port range in inclusive of start and end port . +# livy.rsc.launcher.port.range = 10000~10110 + +# How long will the RSC wait for a connection for a Livy server before shutting itself down. +# livy.rsc.server.idle-timeout = 10m + +# The user that should be impersonated when requesting a Livy session +# livy.rsc.proxy-user = + +# Host or IP adress of the rpc server +# livy.rsc.rpc.server.address = +# How long the rsc client will wait when attempting to connect to the Livy server +# livy.rsc.server.connect.timeout = 90s + +# The logging level for the rpc channel. Possible values: TRACE, DEBUG, INFO, WARN, or ERROR +# livy.rsc.channel.log.level = + +# SASL configurations for authentication +# SASL mechanism used for authentication +# livy.rsc.rpc.sasl.mechanisms = DIGEST-MD5 +# SASL qop used for authentication +# livy.rsc.rpc.sasl.qop = + +# Time between status checks for cancelled a Job +# livy.rsc.job-cancel.trigger-interval = 100ms +# Time before a cancelled a Job is forced into a Cancelled state +# livy.rsc.job-cancel.timeout = 30s + +# Number of statements kept in driver's memory +# livy.rsc.retained-statements = 100 diff --git a/conf/livy-env.sh b/conf/livy-env.sh.template similarity index 100% rename from conf/livy-env.sh rename to conf/livy-env.sh.template diff --git a/conf/livy.conf b/conf/livy.conf.template similarity index 70% rename from conf/livy.conf rename to conf/livy.conf.template index c91b3927b..d57717a13 100644 --- a/conf/livy.conf +++ b/conf/livy.conf.template @@ -3,6 +3,9 @@ # Specify the keystore password. # livy.keystore.password = +# +# Specify the key password. +# livy.key-password = # What host address to start the server on. By default, Livy will bind to all network interfaces. # livy.server.host = 0.0.0.0 @@ -14,10 +17,20 @@ # livy.spark.master = local # What spark deploy mode Livy sessions should use. -# livy.spark.deployMode = +# livy.spark.deploy-mode = + +# Configure Livy server http request and response header size. +# livy.server.request-header.size = 131072 +# livy.server.response-header.size = 131072 + +# Enabled to check whether timeout Livy sessions should be stopped. +# livy.server.session.timeout-check = true # Time in milliseconds on how long Livy will wait before timing out an idle session. # livy.server.session.timeout = 1h +# +# How long a finished session state should be kept in LivyServer for query. +# livy.server.session.state-retain.sec = 600s # If livy should impersonate the requesting users when creating a new session. # livy.impersonation.enabled = true @@ -25,11 +38,13 @@ # Comma-separated list of Livy RSC jars. By default Livy will upload jars from its installation # directory every time a session is started. By caching these files in HDFS, for example, startup # time of sessions on YARN can be reduced. -# livy.jars = +# livy.rsc.jars = # Comma-separated list of Livy REPL jars. By default Livy will upload jars from its installation # directory every time a session is started. By caching these files in HDFS, for example, startup -# time of sessions on YARN can be reduced. +# time of sessions on YARN can be reduced. Please list all the repl dependencies including +# livy-repl_2.10 and livy-repl_2.11 jars, Livy will automatically pick the right dependencies in +# session creation. # livy.repl.jars = # Location of PySpark archives. By default Livy will upload the file from SPARK_HOME, but @@ -47,11 +62,11 @@ # Whether to enable csrf protection, by default it is false. If it is enabled, client should add # http-header "X-Requested-By" in request if the http method is POST/DELETE/PUT/PATCH. -# livy.server.csrf_protection.enabled = +# livy.server.csrf-protection.enabled = # Whether to enable HiveContext in livy interpreter, if it is true hive-site.xml will be detected # on user request and then livy server classpath automatically. -# livy.repl.enableHiveContext = +# livy.repl.enable-hive-context = # Recovery mode of Livy. Possible values: # off: Default. Turn off recovery. Every time Livy shuts down, it stops and forgets all sessions. @@ -73,7 +88,19 @@ # livy.server.recovery.state-store.url = # If Livy can't find the yarn app within this time, consider it lost. -# livy.server.yarn.app-lookup-timeout = 30s +# livy.server.yarn.app-lookup-timeout = 60s +# When the cluster is busy, we may fail to launch yarn app in app-lookup-timeout, then it would +# cause session leakage, so we need to check session leakage. +# How long to check livy session leakage +# livy.server.yarn.app-leakage.check-timeout = 600s +# how often to check livy session leakage +# livy.server.yarn.app-leakage.check-interval = 60s # How often Livy polls YARN to refresh YARN app state. # livy.server.yarn.poll-interval = 1s +# +# Days to keep Livy server request logs. +# livy.server.request-log-retain.days = 5 + +# If the Livy Web UI should be included in the Livy Server. Enabled by default. +# livy.ui.enabled = true diff --git a/conf/log4j.properties b/conf/log4j.properties.template similarity index 100% rename from conf/log4j.properties rename to conf/log4j.properties.template diff --git a/conf/spark-blacklist.conf b/conf/spark-blacklist.conf.template similarity index 94% rename from conf/spark-blacklist.conf rename to conf/spark-blacklist.conf.template index f0919b0b2..b9f0ec26d 100644 --- a/conf/spark-blacklist.conf +++ b/conf/spark-blacklist.conf.template @@ -16,4 +16,4 @@ spark.yarn.jars spark.yarn.archive # Don't allow users to override the RSC timeout. -livy.rsc.server.idle_timeout +livy.rsc.server.idle-timeout diff --git a/core/pom.xml b/core/pom.xml index bfac5fd75..437f00ff5 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -23,18 +23,14 @@ com.cloudera.livy multi-scala-project-root - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../scala/pom.xml livy-core-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom - - true - - com.cloudera.livy @@ -68,20 +64,4 @@ - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - diff --git a/core/scala-2.10/pom.xml b/core/scala-2.10/pom.xml index d757f55fb..6ac60d661 100644 --- a/core/scala-2.10/pom.xml +++ b/core/scala-2.10/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy livy-core_2.10 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-core-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml @@ -34,4 +34,21 @@ ${scala-2.10.version} 2.10 + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + diff --git a/core/scala-2.11/pom.xml b/core/scala-2.11/pom.xml index 36c3ae220..e9e57df19 100644 --- a/core/scala-2.11/pom.xml +++ b/core/scala-2.11/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy livy-core_2.11 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-core-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml @@ -34,4 +34,21 @@ ${scala-2.11.version} 2.11 + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + diff --git a/core/src/main/resources/build.marker b/core/src/main/resources/build.marker new file mode 100644 index 000000000..e69de29bb diff --git a/core/src/main/scala/com/cloudera/livy/sessions/SessionState.scala b/core/src/main/scala/com/cloudera/livy/sessions/SessionState.scala index 975d5927d..ac4bf9b4a 100644 --- a/core/src/main/scala/com/cloudera/livy/sessions/SessionState.scala +++ b/core/src/main/scala/com/cloudera/livy/sessions/SessionState.scala @@ -23,6 +23,11 @@ sealed trait SessionState { def isActive: Boolean } +sealed trait FinishedSessionState extends SessionState { + /** When session is finished. */ + def time: Long +} + object SessionState { def apply(s: String): SessionState = { @@ -83,19 +88,19 @@ object SessionState { override def toString: String = "shutting_down" } - case class Error(time: Long = System.nanoTime()) extends SessionState { + case class Error(time: Long = System.nanoTime()) extends FinishedSessionState { override def isActive: Boolean = true override def toString: String = "error" } - case class Dead(time: Long = System.nanoTime()) extends SessionState { + case class Dead(time: Long = System.nanoTime()) extends FinishedSessionState { override def isActive: Boolean = false override def toString: String = "dead" } - case class Success(time: Long = System.nanoTime()) extends SessionState { + case class Success(time: Long = System.nanoTime()) extends FinishedSessionState { override def isActive: Boolean = false override def toString: String = "success" diff --git a/coverage/pom.xml b/coverage/pom.xml index 96b5c944f..d83428828 100644 --- a/coverage/pom.xml +++ b/coverage/pom.xml @@ -24,11 +24,11 @@ com.cloudera.livy livy-main ../pom.xml - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT livy-coverage-report - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom diff --git a/dev/livy-build-info.sh b/dev/livy-build-info.sh new file mode 100755 index 000000000..04666350a --- /dev/null +++ b/dev/livy-build-info.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This script generates the build info for Livy and places it into the livy-version-info.properties file. +# Arguments: +# build_tgt_directory - The target directory where properties file would be created. [./server/target/extra-resources] +# livy_version - The current version of livy + +RESOURCE_DIR="$1" +mkdir -p "$RESOURCE_DIR" +LIVY_BUILD_INFO="${RESOURCE_DIR}"/livy-version-info.properties + +echo_build_properties() { + echo version=$1 + echo user=$USER + echo revision=$(git rev-parse HEAD) + echo branch=$(git rev-parse --abbrev-ref HEAD) + echo date=$(date -u +%Y-%m-%dT%H:%M:%SZ) + echo url=$(git config --get remote.origin.url) +} + +echo_build_properties $2 > "$LIVY_BUILD_INFO" diff --git a/examples/pom.xml b/examples/pom.xml index 14fbf1323..66a1c61fa 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -19,58 +19,58 @@ - 4.0.0 - - - com.cloudera.livy - livy-main - 0.3.0-SNAPSHOT - ../pom.xml - + 4.0.0 + com.cloudera.livy - livy-examples - 0.3.0-SNAPSHOT - jar + livy-main + 0.4.0-SNAPSHOT + ../pom.xml + + + com.cloudera.livy + livy-examples + 0.4.0-SNAPSHOT + jar - - true - true - true - + + true + true + true + - - - com.cloudera.livy - livy-api - ${project.version} - - - com.cloudera.livy - livy-scala-api_${scala.binary.version} - ${project.version} - - - com.cloudera.livy - livy-client-http - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - - - org.apache.spark - spark-hive_${scala.binary.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - - + + + com.cloudera.livy + livy-api + ${project.version} + + + com.cloudera.livy + livy-scala-api_${scala.binary.version} + ${project.version} + + + com.cloudera.livy + livy-client-http + ${project.version} + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.spark + spark-hive_${scala.binary.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + + diff --git a/examples/src/main/java/com/cloudera/livy/examples/PiApp.java b/examples/src/main/java/com/cloudera/livy/examples/PiApp.java new file mode 100644 index 000000000..f40a911aa --- /dev/null +++ b/examples/src/main/java/com/cloudera/livy/examples/PiApp.java @@ -0,0 +1,102 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.examples; + +import java.io.File; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; + +import com.cloudera.livy.*; + +class PiJob implements + Job, + Function, + Function2 { + + private final int slices; + private final int samples; + + public PiJob(int slices) { + this.slices = slices; + this.samples = (int) Math.min(100000L * slices, Integer.MAX_VALUE); + } + + @Override + public Double call(JobContext ctx) throws Exception { + List sampleList = new ArrayList<>(); + for (int i = 0; i < samples; i++) { + sampleList.add(i); + } + + return 4.0d * ctx.sc().parallelize(sampleList, slices).map(this).reduce(this) / samples; + } + + @Override + public Integer call(Integer v1) { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + return (x * x + y * y < 1) ? 1 : 0; + } + + @Override + public Integer call(Integer v1, Integer v2) { + return v1 + v2; + } +} + +/** + * Example execution: + * java -cp /pathTo/spark-core_2.10-*version*.jar:/pathTo/livy-api-*version*.jar: + * /pathTo/livy-client-http-*version*.jar:/pathTo/livy-examples-*version*.jar + * com.cloudera.livy.examples.PiApp http://livy-host:8998 2 + */ +public class PiApp { + public static void main(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: PiJob "); + System.exit(-1); + } + + LivyClient client = new LivyClientBuilder() + .setURI(new URI(args[0])) + .build(); + + try { + System.out.println("Uploading livy-example jar to the SparkContext..."); + for (String s : System.getProperty("java.class.path").split(File.pathSeparator)) { + if (new File(s).getName().startsWith("livy-examples")) { + client.uploadJar(new File(s)).get(); + break; + } + } + + final int slices = Integer.parseInt(args[1]); + double pi = client.submit(new PiJob(slices)).get(); + + System.out.println("Pi is roughly " + pi); + } finally { + client.stop(true); + } + } +} + diff --git a/examples/src/main/python/pi_app.py b/examples/src/main/python/pi_app.py new file mode 100644 index 000000000..7586906af --- /dev/null +++ b/examples/src/main/python/pi_app.py @@ -0,0 +1,59 @@ +# +# Licensed to Cloudera, Inc. under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Cloudera, Inc. licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +import sys +from random import random +from operator import add + +from livy.client import HttpClient + +if __name__ == "__main__": + """ + Usage: pi_app [livy url] [slices] + + To run this Python script you need to install livy-python-api-*version*.tar.gz with + easy_install first. + + python /pathTo/pi_app.py http://:8998 2 + """ + + if len(sys.argv) != 3: + print("Usage: pi_app ", file=sys.stderr) + exit(-1) + + slices = int(sys.argv[2]) + samples = 100000 * slices + + client = HttpClient(sys.argv[1]) + + def f(_): + x = random() * 2 - 1 + y = random() * 2 - 1 + return 1 if x ** 2 + y ** 2 <= 1 else 0 + + def pi_job(context): + count = context.sc.parallelize(range(1, samples + 1), slices).map(f).reduce(add) + return 4.0 * count / samples + + pi = client.submit(pi_job).result() + + print("Pi is roughly %f" % pi) + client.stop(True) + diff --git a/integration-test/minicluster-dependencies/pom.xml b/integration-test/minicluster-dependencies/pom.xml index 8fbf4b384..c19ff6cd8 100644 --- a/integration-test/minicluster-dependencies/pom.xml +++ b/integration-test/minicluster-dependencies/pom.xml @@ -27,10 +27,10 @@ com.cloudera.livy multi-scala-project-root ../../scala/pom.xml - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT minicluster-dependencies-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom true diff --git a/integration-test/minicluster-dependencies/scala-2.10/pom.xml b/integration-test/minicluster-dependencies/scala-2.10/pom.xml index 3bb2259d0..611830138 100644 --- a/integration-test/minicluster-dependencies/scala-2.10/pom.xml +++ b/integration-test/minicluster-dependencies/scala-2.10/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy minicluster-dependencies_2.10 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy minicluster-dependencies-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/integration-test/minicluster-dependencies/scala-2.11/pom.xml b/integration-test/minicluster-dependencies/scala-2.11/pom.xml index a3a4feffa..8bdf7f4c8 100644 --- a/integration-test/minicluster-dependencies/scala-2.11/pom.xml +++ b/integration-test/minicluster-dependencies/scala-2.11/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy minicluster-dependencies_2.11 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy minicluster-dependencies-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/integration-test/minicluster-dependencies/src/main/resources/build.marker b/integration-test/minicluster-dependencies/src/main/resources/build.marker new file mode 100644 index 000000000..e69de29bb diff --git a/integration-test/pom.xml b/integration-test/pom.xml index f9f6b33d4..54df07f1f 100644 --- a/integration-test/pom.xml +++ b/integration-test/pom.xml @@ -24,11 +24,11 @@ com.cloudera.livy livy-main ../pom.xml - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT livy-integration-test - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar @@ -209,6 +209,35 @@ + + org.codehaus.mojo + build-helper-maven-plugin + + + parse-spark-version + process-test-sources + + parse-version + + + spark + ${spark.version} + + + + add-spark-version-specific-test + process-test-sources + + add-test-source + + + + ${project.basedir}/src/test/spark${spark.majorVersion}/scala + + + + + org.apache.maven.plugins maven-surefire-plugin diff --git a/integration-test/src/main/scala/com/cloudera/livy/test/framework/BaseIntegrationTestSuite.scala b/integration-test/src/main/scala/com/cloudera/livy/test/framework/BaseIntegrationTestSuite.scala index bca80ad53..4df224ba2 100644 --- a/integration-test/src/main/scala/com/cloudera/livy/test/framework/BaseIntegrationTestSuite.scala +++ b/integration-test/src/main/scala/com/cloudera/livy/test/framework/BaseIntegrationTestSuite.scala @@ -21,6 +21,8 @@ package com.cloudera.livy.test.framework import java.io.File import java.util.UUID +import scala.concurrent._ +import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.control.NonFatal @@ -30,6 +32,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.scalatest._ abstract class BaseIntegrationTestSuite extends FunSuite with Matchers with BeforeAndAfterAll { + import scala.concurrent.ExecutionContext.Implicits.global + var cluster: Cluster = _ var httpClient: AsyncHttpClient = _ var livyClient: LivyRestClient = _ @@ -50,6 +54,14 @@ abstract class BaseIntegrationTestSuite extends FunSuite with Matchers with Befo appReport.getDiagnostics() } + protected def restartLivy(): Unit = { + val f = future { + cluster.stopLivy() + cluster.runLivy() + } + Await.result(f, 3 minutes) + } + /** Uploads a file to HDFS and returns just its path. */ protected def uploadToHdfs(file: File): String = { val hdfsPath = new Path(cluster.hdfsScratchDir(), diff --git a/integration-test/src/main/scala/com/cloudera/livy/test/framework/LivyRestClient.scala b/integration-test/src/main/scala/com/cloudera/livy/test/framework/LivyRestClient.scala index 8a1e4e66d..68dc65659 100644 --- a/integration-test/src/main/scala/com/cloudera/livy/test/framework/LivyRestClient.scala +++ b/integration-test/src/main/scala/com/cloudera/livy/test/framework/LivyRestClient.scala @@ -143,7 +143,12 @@ class LivyRestClient(val httpClient: AsyncHttpClient, val livyEndpoint: String) output.get("status") match { case Some("ok") => val data = output("data").asInstanceOf[Map[String, Any]] - Left(data("text/plain").asInstanceOf[String]) + var rst = data.getOrElse("text/plain", "") + val magicRst = data.getOrElse("application/vnd.livy.table.v1+json", null) + if (magicRst != null) { + rst = mapper.writeValueAsString(magicRst) + } + Left(rst.asInstanceOf[String]) case Some("error") => Right(mapper.convertValue(output, classOf[StatementError])) case Some(status) => throw new IllegalStateException(s"Unknown statement $stmtId status: $status") @@ -215,10 +220,14 @@ class LivyRestClient(val httpClient: AsyncHttpClient, val livyEndpoint: String) new BatchSession(id) } - def startSession(kind: Kind, sparkConf: Map[String, String]): InteractiveSession = { + def startSession( + kind: Kind, + sparkConf: Map[String, String], + heartbeatTimeoutInSecond: Int): InteractiveSession = { val r = new CreateInteractiveRequest() r.kind = kind r.conf = sparkConf + r.heartbeatTimeoutInSecond = heartbeatTimeoutInSecond val id = start(INTERACTIVE_TYPE, mapper.writeValueAsString(r)) new InteractiveSession(id) diff --git a/integration-test/src/main/scala/com/cloudera/livy/test/framework/MiniCluster.scala b/integration-test/src/main/scala/com/cloudera/livy/test/framework/MiniCluster.scala index 3e743ca6d..973d7d757 100644 --- a/integration-test/src/main/scala/com/cloudera/livy/test/framework/MiniCluster.scala +++ b/integration-test/src/main/scala/com/cloudera/livy/test/framework/MiniCluster.scala @@ -157,6 +157,7 @@ object MiniLivyMain extends MiniClusterBase { LivyConf.LIVY_SPARK_MASTER.key -> "yarn", LivyConf.LIVY_SPARK_DEPLOY_MODE.key -> "cluster", LivyConf.LIVY_SPARK_SCALA_VERSION.key -> getSparkScalaVersion(), + LivyConf.HEARTBEAT_WATCHDOG_INTERVAL.key -> "1s", LivyConf.YARN_POLL_INTERVAL.key -> "500ms", LivyConf.RECOVERY_MODE.key -> "recovery", LivyConf.RECOVERY_STATE_STORE.key -> "filesystem", @@ -175,7 +176,7 @@ object MiniLivyMain extends MiniClusterBase { // server. Do it atomically since it's used by MiniCluster to detect when the Livy server // is up and ready. eventually(timeout(30 seconds), interval(1 second)) { - val serverUrlConf = Map("livy.server.serverUrl" -> server.serverUrl()) + val serverUrlConf = Map("livy.server.server-url" -> server.serverUrl()) saveProperties(serverUrlConf, new File(configPath + "/serverUrl.conf")) } } @@ -237,16 +238,16 @@ class MiniCluster(config: Map[String, String]) extends Cluster with MiniClusterU assert(tempDir.mkdir(), "Cannot create temp test dir.") sparkConfDir = mkdir("spark-conf") - val sparkScalaVersion = getSparkScalaVersion() - val classPathFile = - new File(s"minicluster-dependencies/scala-$sparkScalaVersion/target/classpath") - assert(classPathFile.isFile, - s"Cannot read MiniCluster classpath file: ${classPathFile.getCanonicalPath}") - val sparkClassPath = - FileUtils.readFileToString(classPathFile, Charset.defaultCharset()) - // When running a real Spark cluster, don't set the classpath. val extraCp = if (!isRealSpark()) { + val sparkScalaVersion = getSparkScalaVersion() + val classPathFile = + new File(s"minicluster-dependencies/scala-$sparkScalaVersion/target/classpath") + assert(classPathFile.isFile, + s"Cannot read MiniCluster classpath file: ${classPathFile.getCanonicalPath}") + val sparkClassPath = + FileUtils.readFileToString(classPathFile, Charset.defaultCharset()) + val dummyJar = Files.createTempFile(Paths.get(tempDir.toURI), "dummy", "jar").toFile Map( SparkLauncher.DRIVER_EXTRA_CLASSPATH -> sparkClassPath, @@ -296,12 +297,12 @@ class MiniCluster(config: Map[String, String]) extends Cluster with MiniClusterU val localLivy = start(MiniLivyMain.getClass, confFile, extraJavaArgs = jacocoArgs) val props = loadProperties(confFile) - livyUrl = props("livy.server.serverUrl") + livyUrl = props("livy.server.server-url") // Wait until Livy server responds. val httpClient = new AsyncHttpClient() eventually(timeout(30 seconds), interval(1 second)) { - val res = httpClient.prepareGet(livyUrl).execute().get() + val res = httpClient.prepareGet(livyUrl + "/metrics").execute().get() assert(res.getStatusCode() == HttpServletResponse.SC_OK) } diff --git a/integration-test/src/test/scala/com/cloudera/livy/test/BatchIT.scala b/integration-test/src/test/scala/com/cloudera/livy/test/BatchIT.scala index 7c0bbbebc..3669ccfb1 100644 --- a/integration-test/src/test/scala/com/cloudera/livy/test/BatchIT.scala +++ b/integration-test/src/test/scala/com/cloudera/livy/test/BatchIT.scala @@ -125,9 +125,7 @@ class BatchIT extends BaseIntegrationTestSuite with BeforeAndAfterAll { withTestLib(classOf[SimpleSparkApp], List(output2, "false")) { s2 => s2.verifySessionRunning() - // Restart Livy. - cluster.stopLivy() - cluster.runLivy() + restartLivy() // Verify previous active session still appears after restart. s2.verifySessionRunning() diff --git a/integration-test/src/test/scala/com/cloudera/livy/test/InteractiveIT.scala b/integration-test/src/test/scala/com/cloudera/livy/test/InteractiveIT.scala index c5a056917..50405f33c 100644 --- a/integration-test/src/test/scala/com/cloudera/livy/test/InteractiveIT.scala +++ b/integration-test/src/test/scala/com/cloudera/livy/test/InteractiveIT.scala @@ -21,10 +21,11 @@ package com.cloudera.livy.test import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern -import scala.language.postfixOps - import org.apache.hadoop.yarn.api.records.YarnApplicationState +import org.scalatest.concurrent.Eventually._ import org.scalatest.OptionValues._ +import scala.concurrent.duration._ +import scala.language.postfixOps import com.cloudera.livy.rsc.RSCConf import com.cloudera.livy.sessions._ @@ -69,7 +70,10 @@ class InteractiveIT extends BaseIntegrationTestSuite { s.run("sqlContext").verifyResult(startsWith(" "true"), false) { s => + eventually(timeout(2 minutes), interval(5 seconds)) { + val appId = s.appId() + appId should not be null + val appReport = cluster.yarnClient.getApplicationReport(appId) + appReport.getYarnApplicationState() shouldBe YarnApplicationState.KILLED + } + } + } + test("user jars are properly imported in Scala interactive sessions") { // Include a popular Java library to test importing user jars. val sparkConf = Map("spark.jars.packages" -> "org.codehaus.plexus:plexus-utils:3.0.24") @@ -131,14 +148,23 @@ class InteractiveIT extends BaseIntegrationTestSuite { } } + test("heartbeat should kill expired session") { + // Set it to 2s because verifySessionIdle() is calling GET every second. + val heartbeatTimeout = Duration.create("2s") + withNewSession(Spark(), Map.empty, true, heartbeatTimeout.toSeconds.toInt) { s => + // If the test reaches here, that means verifySessionIdle() is successfully keeping the + // session alive. Now verify heartbeat is killing expired session. + Thread.sleep(heartbeatTimeout.toMillis * 2) + s.verifySessionDoesNotExist() + } + } + test("recover interactive session") { withNewSession(Spark()) { s => val stmt1 = s.run("1") stmt1.verifyResult("res0: Int = 1") - // Restart Livy. - cluster.stopLivy() - cluster.runLivy() + restartLivy() // Verify session still exists. s.verifySessionIdle() @@ -148,9 +174,7 @@ class InteractiveIT extends BaseIntegrationTestSuite { s.stop() - // Restart Livy. - cluster.stopLivy() - cluster.runLivy() + restartLivy() // Verify deleted session doesn't show up after recovery. s.verifySessionDoesNotExist() @@ -162,10 +186,13 @@ class InteractiveIT extends BaseIntegrationTestSuite { } } - private def withNewSession[R] - (kind: Kind, sparkConf: Map[String, String] = Map.empty, waitForIdle: Boolean = true) + private def withNewSession[R] ( + kind: Kind, + sparkConf: Map[String, String] = Map.empty, + waitForIdle: Boolean = true, + heartbeatTimeoutInSecond: Int = 0) (f: (LivyRestClient#InteractiveSession) => R): R = { - withSession(livyClient.startSession(kind, sparkConf)) { s => + withSession(livyClient.startSession(kind, sparkConf, heartbeatTimeoutInSecond)) { s => if (waitForIdle) { s.verifySessionIdle() } diff --git a/integration-test/src/test/scala/com/cloudera/livy/test/JobApiIT.scala b/integration-test/src/test/scala/com/cloudera/livy/test/JobApiIT.scala index 1b83a819a..9a3f57a0f 100644 --- a/integration-test/src/test/scala/com/cloudera/livy/test/JobApiIT.scala +++ b/integration-test/src/test/scala/com/cloudera/livy/test/JobApiIT.scala @@ -41,7 +41,7 @@ import com.cloudera.livy.utils.LivySparkUtils // Proper type representing the return value of "GET /sessions". At some point we should make // SessionServlet use something like this. -private class SessionList { +class SessionList { val from: Int = -1 val total: Int = -1 val sessions: List[SessionInfo] = Nil @@ -130,13 +130,13 @@ class JobApiIT extends BaseIntegrationTestSuite with BeforeAndAfterAll with Logg test("run spark job") { assume(client != null, "Client not active.") - val result = waitFor(client.submit(new SmallCount(100))); + val result = waitFor(client.submit(new SmallCount(100))) assert(result === 100) } test("run spark sql job") { assume(client != null, "Client not active.") - val result = waitFor(client.submit(new SQLGetTweets(false))); + val result = waitFor(client.submit(new SQLGetTweets(false))) assert(result.size() > 0) } diff --git a/integration-test/src/test/spark2/scala/Spark2JobApiIT.scala b/integration-test/src/test/spark2/scala/Spark2JobApiIT.scala new file mode 100644 index 000000000..0403000d5 --- /dev/null +++ b/integration-test/src/test/spark2/scala/Spark2JobApiIT.scala @@ -0,0 +1,107 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.test + +import java.io.File +import java.net.URI +import java.util.concurrent.{TimeUnit, Future => JFuture} +import javax.servlet.http.HttpServletResponse + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.scalatest.BeforeAndAfterAll + +import com.cloudera.livy._ +import com.cloudera.livy.client.common.HttpMessages._ +import com.cloudera.livy.sessions.SessionKindModule +import com.cloudera.livy.test.framework.BaseIntegrationTestSuite +import com.cloudera.livy.test.jobs.spark2._ + +class Spark2JobApiIT extends BaseIntegrationTestSuite with BeforeAndAfterAll with Logging { + + private var client: LivyClient = _ + private var sessionId: Int = _ + private val mapper = new ObjectMapper() + .registerModule(DefaultScalaModule) + .registerModule(new SessionKindModule()) + + override def afterAll(): Unit = { + super.afterAll() + + if (client != null) { + client.stop(true) + } + + livyClient.connectSession(sessionId).stop() + } + + test("create a new session and upload test jar") { + val tempClient = createClient(livyEndpoint) + + try { + // Figure out the session ID by poking at the REST endpoint. We should probably expose this + // in the Java API. + val list = sessionList() + assert(list.total === 1) + val tempSessionId = list.sessions(0).id + + livyClient.connectSession(tempSessionId).verifySessionIdle() + waitFor(tempClient.uploadJar(new File(testLib))) + + client = tempClient + sessionId = tempSessionId + } finally { + if (client == null) { + try { + if (tempClient != null) { + tempClient.stop(true) + } + } catch { + case e: Exception => warn("Error stopping client.", e) + } + } + } + } + + test("run spark2 job") { + assume(client != null, "Client not active.") + val result = waitFor(client.submit(new SparkSessionTest())) + assert(result === 3) + } + + test("run spark2 dataset job") { + assume(client != null, "Client not active.") + val result = waitFor(client.submit(new DatasetTest())) + assert(result === 2) + } + + private def waitFor[T](future: JFuture[T]): T = { + future.get(30, TimeUnit.SECONDS) + } + + private def sessionList(): SessionList = { + val response = httpClient.prepareGet(s"$livyEndpoint/sessions/").execute().get() + assert(response.getStatusCode === HttpServletResponse.SC_OK) + mapper.readValue(response.getResponseBodyAsStream, classOf[SessionList]) + } + + private def createClient(uri: String): LivyClient = { + new LivyClientBuilder().setURI(new URI(uri)).build() + } +} diff --git a/pom.xml b/pom.xml index b1a5c0e7d..cb6d80fef 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom livy-main @@ -42,7 +42,7 @@ - 2.6.0-cdh5.5.0 + 2.7.3 compile 1.6.2 1.9 @@ -71,6 +71,8 @@ true ${user.dir} ${execution.root}/dev/spark + + ${basedir}/target false @@ -178,18 +180,23 @@ assembly client-common client-http + core core/scala-2.10 core/scala-2.11 coverage examples python-api + repl repl/scala-2.10 repl/scala-2.11 rsc + scala + scala-api scala-api/scala-2.10 scala-api/scala-2.11 server test-lib + integration-test/minicluster-dependencies integration-test/minicluster-dependencies/scala-2.10 integration-test/minicluster-dependencies/scala-2.11 integration-test @@ -620,6 +627,13 @@ testCompile + + scala-compile-first + process-resources + + compile + + ${scala.version} @@ -693,6 +707,7 @@ true ${spark.home} + ${livy.log.dir} true @@ -785,11 +800,6 @@ - - net.alchim31.maven - scala-maven-plugin - - org.apache.maven.plugins maven-antrun-plugin @@ -929,11 +939,6 @@ maven-surefire-plugin - - org.scalatest - scalatest-maven-plugin - - org.jacoco jacoco-maven-plugin @@ -1002,6 +1007,40 @@ 2.0.1 + + + spark-2.1 + + + spark-2.1 + + + + 2.1.0 + + + + + skip-parent-modules + + + src/main/resources/build.marker + + + + + + org.scalatest + scalatest-maven-plugin + + + net.alchim31.maven + scala-maven-plugin + + + + + diff --git a/python-api/pom.xml b/python-api/pom.xml index b40bb07c0..c28142adc 100644 --- a/python-api/pom.xml +++ b/python-api/pom.xml @@ -7,13 +7,13 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml com.cloudera.livy livy-python-api - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom diff --git a/python-api/setup.py b/python-api/setup.py index 3144495d2..69ffbc0a0 100644 --- a/python-api/setup.py +++ b/python-api/setup.py @@ -35,11 +35,12 @@ 'futures>=3.0.5', 'requests>=2.10.0', 'responses>=0.5.1', + 'requests-kerberos>=0.11.0', ] setup( name='livy-python-api', - version="0.3.0-SNAPSHOT", + version="0.4.0-SNAPSHOT", packages=["livy", "livy-tests"], package_dir={ "": "src/main/python", diff --git a/python-api/src/main/python/livy/client.py b/python-api/src/main/python/livy/client.py index fd2a4b102..7154abad9 100644 --- a/python-api/src/main/python/livy/client.py +++ b/python-api/src/main/python/livy/client.py @@ -28,6 +28,7 @@ from concurrent.futures import ThreadPoolExecutor from future.moves.urllib.parse import ParseResult, urlparse from io import open, StringIO +from requests_kerberos import HTTPKerberosAuth, REQUIRED from livy.job_handle import JobHandle @@ -76,13 +77,13 @@ def __init__(self, url, load_defaults=True, conf_dict=None): path=match.group(1), params=uri.params, query=uri.query, fragment=uri.fragment) self._set_uri(base) - self._conn = _LivyConnection(base) + self._conn = _LivyConnection(base, self._config) self._session_id = int(match.group(2)) self._reconnect_to_existing_session() else: self._set_uri(uri) session_conf_dict = dict(self._config.items(self._CONFIG_SECTION)) - self._conn = _LivyConnection(uri) + self._conn = _LivyConnection(uri, self._config) self._session_id = self._create_new_session( session_conf_dict).json()['id'] self._executor = ThreadPoolExecutor(max_workers=1) @@ -434,10 +435,22 @@ class _LivyConnection(object): 'Accept': 'application/json', } - def __init__(self, uri): + _SPNEGO_ENABLED_CONF = 'livy.client.http.spnego.enable' + + def __init__(self, uri, config): self._server_url_prefix = uri.geturl() + self._SESSIONS_URI self._requests = requests self.lock = threading.Lock() + self._spnego_enabled = \ + config.getboolean('env', self._SPNEGO_ENABLED_CONF) \ + if config.has_option('env', self._SPNEGO_ENABLED_CONF) else False + + def _spnego_auth(self): + if self._spnego_enabled: + return HTTPKerberosAuth(mutual_authentication=REQUIRED, + sanitize_mutual_error_response=False) + else: + return None def send_request( self, @@ -483,7 +496,7 @@ def send_request( request_url = self._server_url_prefix + suffix_url return self._requests.request(method, request_url, timeout=self._TIMEOUT, headers=local_headers, files=files, - json=data) + json=data, auth=self._spnego_auth()) finally: if files is not None: files.clear() diff --git a/python-api/src/main/python/livy/job_context.py b/python-api/src/main/python/livy/job_context.py index 169f5ea3d..db0791a23 100644 --- a/python-api/src/main/python/livy/job_context.py +++ b/python-api/src/main/python/livy/job_context.py @@ -152,3 +152,23 @@ def local_tmp_dir_path(self): Returns a local tmp dir path specific to the context """ pass + + @abstractproperty + def spark_session(self): + """ + The shared SparkSession instance. + + Returns + ------- + sc : pyspark.sql.SparkSession + A SparkSession instance + + Examples + ------- + + >>> def simple_spark_job(context): + >>> session = context.spark_session + >>> df1 = session.read.json('/sample.json') + >>> return df1.dTypes() + """ + pass diff --git a/python-api/src/test/python/livy-tests/client_test.py b/python-api/src/test/python/livy-tests/client_test.py index bb1dbc5fa..bab958e52 100644 --- a/python-api/src/test/python/livy-tests/client_test.py +++ b/python-api/src/test/python/livy-tests/client_test.py @@ -25,7 +25,7 @@ session_id = 0 job_id = 1 -base_uri = 'http://{}:{}'.format(socket.gethostname(), 8998) +base_uri = 'http://{0}:{1}'.format(socket.gethostname(), 8998) client_test = None invoked_queued_callback = False invoked_running_callback = False diff --git a/repl/pom.xml b/repl/pom.xml index c3f4c3866..6d5f73c9f 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -19,231 +19,227 @@ - 4.0.0 - - com.cloudera.livy - multi-scala-project-root - ../scala/pom.xml - 0.3.0-SNAPSHOT - - - livy-repl-parent - 0.3.0-SNAPSHOT - pom - - - true - - - - - - com.cloudera.livy - livy-rsc - ${project.version} - provided - - - - com.cloudera.livy - livy-core_${scala.binary.version} - ${project.version} - - - - com.cloudera.livy - livy-client-common - - - - - - ${project.groupId} - livy-core_${scala.binary.version} - ${project.version} - test-jar - test - - - - com.fasterxml.jackson.core - jackson-core - provided - - - - com.fasterxml.jackson.module - jackson-module-scala_${scala.binary.version} - provided - - - - com.fasterxml.jackson.core - jackson-databind - provided - - - - commons-codec - commons-codec - - - - log4j - log4j - provided - - - - org.apache.spark - spark-repl_${scala.binary.version} - provided - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - provided - - - - org.apache.spark - spark-hive_${scala.binary.version} - provided - - - - org.json4s - json4s-ast_${scala.binary.version} - - - - org.json4s - json4s-core_${scala.binary.version} - - - - org.json4s - json4s-jackson_${scala.binary.version} - - - - org.scala-lang - scala-compiler - provided - - - - org.scala-lang - scala-library - provided - - - - org.scala-lang - scala-reflect - provided - - - - org.slf4j - slf4j-api - provided - - - - net.sf.py4j - py4j - provided - - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade - package - - shade - - - false - - - org.json4s:json4s-ast_${scala.binary.version} - org.json4s:json4s-core_${scala.binary.version} - org.json4s:json4s-jackson_${scala.binary.version} - - - - - *:* - - *.jar - META-INF/maven/** - - - - - - org.json4s - com.cloudera.livy.shaded.json4s - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - package - - copy-dependencies - - - - json4s-ast_${scala.binary.version}, - json4s-core_${scala.binary.version}, - json4s-jackson_${scala.binary.version}, - paranamer, - scalap - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - ${project.build.directory}/jars - - - - - org.scalatest - scalatest-maven-plugin - - - Livy - local - - - - - - + 4.0.0 + + com.cloudera.livy + multi-scala-project-root + ../scala/pom.xml + 0.4.0-SNAPSHOT + + + livy-repl-parent + 0.4.0-SNAPSHOT + pom + + + + + com.cloudera.livy + livy-rsc + ${project.version} + provided + + + + com.cloudera.livy + livy-core_${scala.binary.version} + ${project.version} + + + + com.cloudera.livy + livy-client-common + + + + + + ${project.groupId} + livy-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + com.fasterxml.jackson.core + jackson-core + provided + + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + provided + + + + com.fasterxml.jackson.core + jackson-databind + provided + + + + commons-codec + commons-codec + + + + log4j + log4j + provided + + + + org.apache.spark + spark-repl_${scala.binary.version} + provided + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + + org.apache.spark + spark-hive_${scala.binary.version} + provided + + + + org.json4s + json4s-ast_${scala.binary.version} + + + + org.json4s + json4s-core_${scala.binary.version} + + + + org.json4s + json4s-jackson_${scala.binary.version} + + + + org.scala-lang + scala-compiler + provided + + + + org.scala-lang + scala-library + provided + + + + org.scala-lang + scala-reflect + provided + + + + org.slf4j + slf4j-api + provided + + + + net.sf.py4j + py4j + provided + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade + package + + shade + + + false + + + org.json4s:json4s-ast_${scala.binary.version} + org.json4s:json4s-core_${scala.binary.version} + org.json4s:json4s-jackson_${scala.binary.version} + + + + + *:* + + *.jar + META-INF/maven/** + + + + + + org.json4s + com.cloudera.livy.shaded.json4s + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy-dependencies + + + + json4s-ast_${scala.binary.version}, + json4s-core_${scala.binary.version}, + json4s-jackson_${scala.binary.version}, + paranamer, + scalap + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + ${project.build.directory}/jars + + + + + org.scalatest + scalatest-maven-plugin + + + Livy + local + + + + + + diff --git a/repl/scala-2.10/pom.xml b/repl/scala-2.10/pom.xml index 887742f90..3661e2b44 100644 --- a/repl/scala-2.10/pom.xml +++ b/repl/scala-2.10/pom.xml @@ -22,13 +22,13 @@ 4.0.0 com.cloudera.livy livy-repl_2.10 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-repl-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/repl/scala-2.10/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala b/repl/scala-2.10/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala index 312a63bba..d736125a5 100644 --- a/repl/scala-2.10/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala +++ b/repl/scala-2.10/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala @@ -91,6 +91,10 @@ class SparkInterpreter(conf: SparkConf) .filter { u => u.getProtocol == "file" && new File(u.getPath).isFile } // Livy rsc and repl are also in the extra jars list. Filter them out. .filterNot { u => Paths.get(u.toURI).getFileName.toString.startsWith("livy-") } + // Some bad spark packages depend on the wrong version of scala-reflect. Blacklist it. + .filterNot { u => + Paths.get(u.toURI).getFileName.toString.contains("org.scala-lang_scala-reflect") + } extraJarPath.foreach { p => debug(s"Adding $p to Scala interpreter's class path...") } sparkIMain.addUrlsToClassPath(extraJarPath: _*) @@ -132,6 +136,43 @@ class SparkInterpreter(conf: SparkConf) sparkIMain.interpret(code) } + override protected[repl] def parseError(stdout: String): (String, Seq[String]) = { + // An example of Scala 2.10 runtime exception error message: + // java.lang.Exception: message + // at $iwC$$iwC$$iwC$$iwC$$iwC.error(:25) + // at $iwC$$iwC$$iwC.error2(:27) + // at $iwC$$iwC.(:41) + // at $iwC.(:43) + // at (:45) + // at .(:49) + // at .() + // at .(:7) + // at .() + // at $print() + // at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + // at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) + // ... + + val (ename, traceback) = super.parseError(stdout) + + // Remove internal frames. + val startOfInternalFrames = traceback.indexWhere(_.contains("$iwC$$iwC.")) + var endOfInternalFrames = traceback.indexWhere(!_.trim.startsWith("at"), startOfInternalFrames) + if (endOfInternalFrames == -1) { + endOfInternalFrames = traceback.length + } + + val cleanedTraceback = if (startOfInternalFrames == -1) { + traceback + } else { + traceback.view.zipWithIndex + .filterNot { z => z._2 >= startOfInternalFrames && z._2 < endOfInternalFrames } + .map { _._1.replaceAll("(\\$iwC\\$)*\\$iwC", "") } + } + + (ename, cleanedTraceback) + } + override protected def valueOfTerm(name: String): Option[Any] = { sparkIMain.valueOfTerm(name) } diff --git a/repl/scala-2.10/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala b/repl/scala-2.10/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala new file mode 100644 index 000000000..e2b783a17 --- /dev/null +++ b/repl/scala-2.10/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala @@ -0,0 +1,87 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.repl + +import org.scalatest._ + +import com.cloudera.livy.LivyBaseUnitTestSuite + +class SparkInterpreterSpec extends FunSpec with Matchers with LivyBaseUnitTestSuite { + describe("SparkInterpreter") { + val interpreter = new SparkInterpreter(null) + + it("should parse Scala compile error.") { + // Regression test for LIVY-260. + val error = + """:27: error: type mismatch; + | found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |:27: error: type mismatch; + | found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |""".stripMargin + + val expectedTraceback = AbstractSparkInterpreter.KEEP_NEWLINE_REGEX.split( + """ found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |:27: error: type mismatch; + | found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |""".stripMargin) + + val (ename, traceback) = interpreter.parseError(error) + ename shouldBe ":27: error: type mismatch;" + traceback shouldBe expectedTraceback + } + + it("should parse Scala runtime error and remove internal frames.") { + val error = + """java.lang.RuntimeException: message + | at $iwC$$iwC$$iwC$$iwC$$iwC.error(:25) + | at $iwC$$iwC$$iwC.error2(:27) + | at $iwC$$iwC.(:41) + | at $iwC.(:43) + | at (:45) + | at .(:49) + | at .() + | at .(:7) + | at .() + | at $print() + | at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + |""".stripMargin + + val expectedTraceback = AbstractSparkInterpreter.KEEP_NEWLINE_REGEX.split( + """ at .error(:25) + | at .error2(:27) + |""".stripMargin) + + val (ename, traceback) = interpreter.parseError(error) + ename shouldBe "java.lang.RuntimeException: message" + traceback shouldBe expectedTraceback + } + } +} diff --git a/repl/scala-2.11/pom.xml b/repl/scala-2.11/pom.xml index 5a2093fbf..d81bde204 100644 --- a/repl/scala-2.11/pom.xml +++ b/repl/scala-2.11/pom.xml @@ -22,13 +22,13 @@ 4.0.0 com.cloudera.livy livy-repl_2.11 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-repl-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/repl/scala-2.11/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala b/repl/scala-2.11/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala index 2bf6347ea..f08a46e1b 100644 --- a/repl/scala-2.11/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala +++ b/repl/scala-2.11/src/main/scala/com/cloudera/livy/repl/SparkInterpreter.scala @@ -77,6 +77,10 @@ class SparkInterpreter(conf: SparkConf) .filter { u => u.getProtocol == "file" && new File(u.getPath).isFile } // Livy rsc and repl are also in the extra jars list. Filter them out. .filterNot { u => Paths.get(u.toURI).getFileName.toString.startsWith("livy-") } + // Some bad spark packages depend on the wrong version of scala-reflect. Blacklist it. + .filterNot { u => + Paths.get(u.toURI).getFileName.toString.contains("org.scala-lang_scala-reflect") + } extraJarPath.foreach { p => debug(s"Adding $p to Scala interpreter's class path...") } sparkILoop.addUrlsToClassPath(extraJarPath: _*) diff --git a/repl/scala-2.11/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala b/repl/scala-2.11/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala new file mode 100644 index 000000000..5cb88e3b4 --- /dev/null +++ b/repl/scala-2.11/src/test/scala/com/cloudera/livy/repl/SparkInterpreterSpec.scala @@ -0,0 +1,69 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.repl + +import org.scalatest._ + +import com.cloudera.livy.LivyBaseUnitTestSuite + +class SparkInterpreterSpec extends FunSpec with Matchers with LivyBaseUnitTestSuite { + describe("SparkInterpreter") { + val interpreter = new SparkInterpreter(null) + + it("should parse Scala compile error.") { + // Regression test for LIVY-. + val error = + """:27: error: type mismatch; + | found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |:27: error: type mismatch; + | found : Int + | required: String + | sc.setJobGroup(groupName, groupName, true) + | ^ + |""".stripMargin + + val parsedError = AbstractSparkInterpreter.KEEP_NEWLINE_REGEX.split(error) + + val expectedTraceback = parsedError.tail + + val (ename, traceback) = interpreter.parseError(error) + ename shouldBe ":27: error: type mismatch;" + traceback shouldBe expectedTraceback + } + + it("should parse Scala runtime error.") { + val error = + """java.lang.RuntimeException: message + | ... 48 elided + | + |Tailing message""".stripMargin + + val parsedError = AbstractSparkInterpreter.KEEP_NEWLINE_REGEX.split(error) + + val expectedTraceback = parsedError.tail + + val (ename, traceback) = interpreter.parseError(error) + ename shouldBe "java.lang.RuntimeException: message" + traceback shouldBe expectedTraceback + } + } +} diff --git a/repl/src/main/resources/build.marker b/repl/src/main/resources/build.marker new file mode 100644 index 000000000..e69de29bb diff --git a/repl/src/main/resources/fake_shell.py b/repl/src/main/resources/fake_shell.py index 898fe2bbb..00a495633 100644 --- a/repl/src/main/resources/fake_shell.py +++ b/repl/src/main/resources/fake_shell.py @@ -30,6 +30,7 @@ import tempfile import shutil import pickle +import textwrap if sys.version >= '3': unicode = str @@ -97,6 +98,7 @@ def __init__(self): self.hive_ctx = None self.streaming_ctx = None self.local_tmp_dir_path = local_tmp_dir_path + self.spark_session = global_dict.get('spark') def sc(self): return self.sc @@ -143,6 +145,9 @@ def stop(self): if self.sc is not None: self.sc.stop() + def spark_session(self): + return self.spark_session + class PySparkJobProcessorImpl(object): def processBypassJob(self, serialized_job): @@ -402,6 +407,9 @@ def magic_table(name): for row in value: cols = [] data.append(cols) + + if 'Row' == row.__class__.__name__: + row = row.asDict() if not isinstance(row, (list, tuple, dict)): row = [row] @@ -423,11 +431,14 @@ def magic_table(name): } headers[name] = header else: - # Reject columns that have a different type. - if header['type'] != col_type: - exc_type = Exception - exc_value = 'table rows have different types' - return execute_reply_error(exc_type, exc_value, None) + # Reject columns that have a different type. (allow none value) + if col_type != "NULL_TYPE" and header['type'] != col_type: + if header['type'] == "NULL_TYPE": + header['type'] = col_type + else: + exc_type = Exception + exc_value = 'table rows have different types' + return execute_reply_error(exc_type, exc_value, None) cols.append(col) @@ -524,8 +535,21 @@ def main(): exec('from pyspark.sql import HiveContext', global_dict) exec('from pyspark.streaming import StreamingContext', global_dict) exec('import pyspark.cloudpickle as cloudpickle', global_dict) + if spark_major_version >= "2": exec('from pyspark.shell import spark', global_dict) + else: + # LIVY-294, need to check whether HiveContext can work properly, + # fallback to SQLContext if HiveContext can not be initialized successfully. + # Only for spark-1. + code = textwrap.dedent(""" + import py4j + from pyspark.sql import SQLContext + try: + sqlContext.tables() + except py4j.protocol.Py4JError: + sqlContext = SQLContext(sc)""") + exec(code, global_dict) #Start py4j callback server from py4j.protocol import ENTRY_POINT_OBJECT_ID diff --git a/repl/src/main/scala/com/cloudera/livy/repl/AbstractSparkInterpreter.scala b/repl/src/main/scala/com/cloudera/livy/repl/AbstractSparkInterpreter.scala index 283490dd1..d117da7f8 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/AbstractSparkInterpreter.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/AbstractSparkInterpreter.scala @@ -31,10 +31,8 @@ import org.json4s.JsonDSL._ import com.cloudera.livy.Logging object AbstractSparkInterpreter { - private val EXCEPTION_STACK_TRACE_REGEX = """(.+?)\n((?:[ |\t].+?\n?)*)""".r - private val KEEP_NEWLINE_REGEX = """(?=\n)""".r + private[repl] val KEEP_NEWLINE_REGEX = """(?<=\n)""".r private val MAGIC_REGEX = "^%(\\w+)\\W*(.*)".r - val USER_CODE_FRAME_NAME = "" } abstract class AbstractSparkInterpreter extends Interpreter with Logging { @@ -52,12 +50,13 @@ abstract class AbstractSparkInterpreter extends Interpreter with Logging { protected def valueOfTerm(name: String): Option[Any] - override def execute(code: String): Interpreter.ExecuteResponse = restoreContextClassLoader { - require(isStarted()) + override protected[repl] def execute(code: String): Interpreter.ExecuteResponse = + restoreContextClassLoader { + require(isStarted()) - executeLines(code.trim.split("\n").toList, Interpreter.ExecuteSuccess(JObject( - (TEXT_PLAIN, JString("")) - ))) + executeLines(code.trim.split("\n").toList, Interpreter.ExecuteSuccess(JObject( + (TEXT_PLAIN, JString("")) + ))) } private def executeMagic(magic: String, rest: String): Interpreter.ExecuteResponse = { @@ -225,30 +224,33 @@ abstract class AbstractSparkInterpreter extends Interpreter with Logging { ) case Results.Incomplete => Interpreter.ExecuteIncomplete() case Results.Error => - def parseStdout(stdout: String): (String, Seq[String]) = { - stdout match { - case EXCEPTION_STACK_TRACE_REGEX(ename, tracebackLines) => - var traceback = KEEP_NEWLINE_REGEX.pattern.split(tracebackLines) - val interpreterFrameIdx = traceback.indexWhere(_.contains("$iwC$$iwC.")) - if (interpreterFrameIdx >= 0) { - traceback = traceback - // Remove Interpreter frames - .take(interpreterFrameIdx) - // Replace weird internal class name - .map(_.replaceAll("(\\$iwC\\$)*\\$iwC", "")) - // TODO Proper translate line number in stack trace for $iwC$$iwC. - } - (ename.trim, traceback) - case _ => (stdout, Seq.empty) - } - } - val (ename, traceback) = parseStdout(readStdout()) + val (ename, traceback) = parseError(readStdout()) Interpreter.ExecuteError("Error", ename, traceback) } } } } + protected[repl] def parseError(stdout: String): (String, Seq[String]) = { + // An example of Scala compile error message: + // :27: error: type mismatch; + // found : Int + // required: Boolean + + // An example of Scala runtime exception error message: + // java.lang.RuntimeException: message + // at .error(:11) + // ... 32 elided + + // Return the first line as ename. Lines following as traceback. + + val lines = KEEP_NEWLINE_REGEX.split(stdout) + val ename = lines.headOption.map(_.trim).getOrElse("unknown error") + val traceback = lines.tail + + (ename, traceback) + } + protected def restoreContextClassLoader[T](fn: => T): T = { val currentClassLoader = Thread.currentThread().getContextClassLoader() try { diff --git a/repl/src/main/scala/com/cloudera/livy/repl/BypassPySparkJob.scala b/repl/src/main/scala/com/cloudera/livy/repl/BypassPySparkJob.scala index 69f617149..850ab4fb4 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/BypassPySparkJob.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/BypassPySparkJob.scala @@ -23,10 +23,14 @@ import com.cloudera.livy.{Job, JobContext} class BypassPySparkJob( serializedJob: Array[Byte], - pysparkJobProcessor: PySparkJobProcessor) extends Job[Array[Byte]] { + replDriver: ReplDriver) extends Job[Array[Byte]] { override def call(jc: JobContext): Array[Byte] = { - val resultByteArray = pysparkJobProcessor.processBypassJob(serializedJob) + val interpreter = replDriver.interpreter + require(interpreter != null && interpreter.isInstanceOf[PythonInterpreter]) + val pi = interpreter.asInstanceOf[PythonInterpreter] + + val resultByteArray = pi.pysparkJobProcessor.processBypassJob(serializedJob) val resultString = new String(resultByteArray, StandardCharsets.UTF_8) if (resultString.startsWith("Client job error:")) { throw new PythonJobException(resultString) diff --git a/repl/src/main/scala/com/cloudera/livy/repl/Interpreter.scala b/repl/src/main/scala/com/cloudera/livy/repl/Interpreter.scala index 069953ee0..fa3b6405f 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/Interpreter.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/Interpreter.scala @@ -40,15 +40,15 @@ trait Interpreter { /** * Start the Interpreter. * - * @return A SparkContext, which may be null. + * @return A SparkContext */ def start(): SparkContext /** - * Execute the code and return the result as a Future as it may + * Execute the code and return the result, it may * take some time to execute. */ - def execute(code: String): ExecuteResponse + protected[repl] def execute(code: String): ExecuteResponse /** Shut down the interpreter. */ def close(): Unit diff --git a/repl/src/main/scala/com/cloudera/livy/repl/ProcessInterpreter.scala b/repl/src/main/scala/com/cloudera/livy/repl/ProcessInterpreter.scala index f173f8b45..fe10697ef 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/ProcessInterpreter.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/ProcessInterpreter.scala @@ -28,6 +28,7 @@ import org.apache.spark.SparkContext import org.json4s.JValue import com.cloudera.livy.{Logging, Utils} +import com.cloudera.livy.client.common.ClientConf private sealed trait Request private case class ExecuteRequest(code: String, promise: Promise[JValue]) extends Request @@ -41,21 +42,21 @@ private case class ShutdownRequest(promise: Promise[Unit]) extends Request * @param process */ abstract class ProcessInterpreter(process: Process) - extends Interpreter - with Logging -{ + extends Interpreter with Logging { protected[this] val stdin = new PrintWriter(process.getOutputStream) protected[this] val stdout = new BufferedReader(new InputStreamReader(process.getInputStream), 1) override def start(): SparkContext = { waitUntilReady() - // At this point there should be an already active SparkContext that can be retrieved - // using SparkContext.getOrCreate. But we don't really support running "pre-compiled" - // jobs against pyspark or sparkr, so just return null here. - null + if (ClientConf.TEST_MODE) { + null.asInstanceOf[SparkContext] + } else { + SparkContext.getOrCreate() + } } - override def execute(code: String): Interpreter.ExecuteResponse = { + + override protected[repl] def execute(code: String): Interpreter.ExecuteResponse = { try { sendExecuteRequest(code) } catch { @@ -127,6 +128,7 @@ abstract class ProcessInterpreter(process: Process) val exitCode = process.waitFor() if (exitCode != 0) { error(f"Process has died with $exitCode") + error(stderrLines.mkString("\n")) } } } diff --git a/repl/src/main/scala/com/cloudera/livy/repl/PythonInterpreter.scala b/repl/src/main/scala/com/cloudera/livy/repl/PythonInterpreter.scala index 6b3521c6a..a04dfef5e 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/PythonInterpreter.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/PythonInterpreter.scala @@ -47,8 +47,9 @@ object PythonInterpreter extends Logging { def apply(conf: SparkConf, kind: Kind): Interpreter = { val pythonExec = kind match { - case PySpark3() => sys.env.getOrElse("PYSPARK3_DRIVER_PYTHON", "python3") - case PySpark() => sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", "python") + case PySpark() => sys.env.getOrElse("PYSPARK_PYTHON", "python") + case PySpark3() => sys.env.getOrElse("PYSPARK3_PYTHON", "python3") + case _ => throw new IllegalArgumentException(s"Unknown kind: $kind") } val gatewayServer = new GatewayServer(null, 0) @@ -186,7 +187,10 @@ object PythonInterpreter extends Logging { } } -private class PythonInterpreter(process: Process, gatewayServer: GatewayServer, pyKind: String) +private class PythonInterpreter( + process: Process, + gatewayServer: GatewayServer, + pyKind: String) extends ProcessInterpreter(process) with Logging { @@ -194,7 +198,8 @@ private class PythonInterpreter(process: Process, gatewayServer: GatewayServer, override def kind: String = pyKind - private val pysparkJobProcessor = PythonInterpreter.initiatePy4jCallbackGateway(gatewayServer) + private[repl] val pysparkJobProcessor = + PythonInterpreter.initiatePy4jCallbackGateway(gatewayServer) override def close(): Unit = { try { @@ -256,11 +261,6 @@ private class PythonInterpreter(process: Process, gatewayServer: GatewayServer, } } - def createWrapper(driver: ReplDriver, msg: BaseProtocol.BypassJobRequest): BypassJobWrapper = { - new BypassJobWrapper(driver, msg.id, - new BypassPySparkJob(msg.serializedJob, pysparkJobProcessor)) - } - def addFile(path: String): Unit = { pysparkJobProcessor.addFile(path) } diff --git a/repl/src/main/scala/com/cloudera/livy/repl/ReplDriver.scala b/repl/src/main/scala/com/cloudera/livy/repl/ReplDriver.scala index 99e9f71e0..695a9d075 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/ReplDriver.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/ReplDriver.scala @@ -26,8 +26,8 @@ import org.apache.spark.SparkConf import org.apache.spark.api.java.JavaSparkContext import com.cloudera.livy.Logging +import com.cloudera.livy.rsc.{BaseProtocol, ReplJobResults, RSCConf} import com.cloudera.livy.rsc.BaseProtocol.ReplState -import com.cloudera.livy.rsc.{BaseProtocol, RSCConf, ReplJobResults} import com.cloudera.livy.rsc.driver._ import com.cloudera.livy.rsc.rpc.Rpc import com.cloudera.livy.sessions._ @@ -45,11 +45,12 @@ class ReplDriver(conf: SparkConf, livyConf: RSCConf) override protected def initializeContext(): JavaSparkContext = { interpreter = kind match { case PySpark() => PythonInterpreter(conf, PySpark()) - case PySpark3() => PythonInterpreter(conf, PySpark3()) + case PySpark3() => + PythonInterpreter(conf, PySpark3()) case Spark() => new SparkInterpreter(conf) case SparkR() => SparkRInterpreter(conf) } - session = new Session(interpreter, { s => broadcast(new ReplState(s.toString)) }) + session = new Session(livyConf, interpreter, { s => broadcast(new ReplState(s.toString)) }) Option(Await.result(session.start(), Duration.Inf)) .map(new JavaSparkContext(_)) @@ -70,6 +71,10 @@ class ReplDriver(conf: SparkConf, livyConf: RSCConf) session.execute(msg.code) } + def handle(ctx: ChannelHandlerContext, msg: BaseProtocol.CancelReplJobRequest): Unit = { + session.cancel(msg.id) + } + /** * Return statement results. Results are sorted by statement id. */ @@ -86,17 +91,25 @@ class ReplDriver(conf: SparkConf, livyConf: RSCConf) session.statements.filterKeys(id => id >= msg.from && id < until).values.toArray } } + + // Update progress of statements when queried + statements.foreach { s => + s.updateProgress(session.progressOfStatement(s.id)) + } + new ReplJobResults(statements.sortBy(_.id)) } override protected def createWrapper(msg: BaseProtocol.BypassJobRequest): BypassJobWrapper = { - interpreter match { - case pi: PythonInterpreter => pi.createWrapper(this, msg) + kind match { + case PySpark() | PySpark3() => new BypassJobWrapper(this, msg.id, + new BypassPySparkJob(msg.serializedJob, this)) case _ => super.createWrapper(msg) } } override protected def addFile(path: String): Unit = { + require(interpreter != null) interpreter match { case pi: PythonInterpreter => pi.addFile(path) case _ => super.addFile(path) @@ -104,6 +117,7 @@ class ReplDriver(conf: SparkConf, livyConf: RSCConf) } override protected def addJarOrPyFile(path: String): Unit = { + require(interpreter != null) interpreter match { case pi: PythonInterpreter => pi.addPyFile(this, conf, path) case _ => super.addJarOrPyFile(path) diff --git a/repl/src/main/scala/com/cloudera/livy/repl/Session.scala b/repl/src/main/scala/com/cloudera/livy/repl/Session.scala index c713f6110..31e520c7d 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/Session.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/Session.scala @@ -18,11 +18,14 @@ package com.cloudera.livy.repl +import java.util.{LinkedHashMap => JLinkedHashMap} +import java.util.Map.Entry import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger -import scala.collection.concurrent.TrieMap +import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ import org.apache.spark.SparkContext import org.json4s.jackson.JsonMethods.{compact, render} @@ -30,6 +33,7 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import com.cloudera.livy.Logging +import com.cloudera.livy.rsc.RSCConf import com.cloudera.livy.rsc.driver.{Statement, StatementState} import com.cloudera.livy.sessions._ @@ -44,17 +48,33 @@ object Session { val TRACEBACK = "traceback" } -class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Unit = { _ => } ) - extends Logging -{ +class Session( + livyConf: RSCConf, + interpreter: Interpreter, + stateChangedCallback: SessionState => Unit = { _ => }) + extends Logging { import Session._ - private implicit val executor = ExecutionContext.fromExecutorService( + private val interpreterExecutor = ExecutionContext.fromExecutorService( Executors.newSingleThreadExecutor()) + + private val cancelExecutor = ExecutionContext.fromExecutorService( + Executors.newSingleThreadExecutor()) + private implicit val formats = DefaultFormats + @volatile private[repl] var _sc: Option[SparkContext] = None + private var _state: SessionState = SessionState.NotStarted() - private val _statements = TrieMap[Int, Statement]() + + // Number of statements kept in driver's memory + private val numRetainedStatements = livyConf.getInt(RSCConf.Entry.RETAINED_STATEMENT_NUMBER) + + private val _statements = new JLinkedHashMap[Int, Statement] { + protected override def removeEldestEntry(eldest: Entry[Int, Statement]): Boolean = { + size() > numRetainedStatements + } + }.asScala private val newStatementId = new AtomicInteger(0) @@ -64,12 +84,12 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un val future = Future { changeState(SessionState.Starting()) val sc = interpreter.start() + _sc = Option(sc) changeState(SessionState.Idle()) sc - } - future.onFailure { case _ => - changeState(SessionState.Error()) - } + }(interpreterExecutor) + + future.onFailure { case _ => changeState(SessionState.Error()) }(interpreterExecutor) future } @@ -77,27 +97,100 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un def state: SessionState = _state - def statements: collection.Map[Int, Statement] = _statements.readOnlySnapshot() + def statements: collection.Map[Int, Statement] = _statements.synchronized { + _statements.toMap + } def execute(code: String): Int = { val statementId = newStatementId.getAndIncrement() - _statements(statementId) = new Statement(statementId, StatementState.Waiting, null) + val statement = new Statement(statementId, StatementState.Waiting, null) + _statements.synchronized { _statements(statementId) = statement } + Future { - _statements(statementId) = new Statement(statementId, StatementState.Running, null) + setJobGroup(statementId) + statement.compareAndTransit(StatementState.Waiting, StatementState.Running) + + if (statement.state.get() == StatementState.Running) { + statement.output = executeCode(statementId, code) + } + + statement.compareAndTransit(StatementState.Running, StatementState.Available) + statement.compareAndTransit(StatementState.Cancelling, StatementState.Cancelled) + statement.updateProgress(1.0) + }(interpreterExecutor) - _statements(statementId) = - new Statement(statementId, StatementState.Available, executeCode(statementId, code)) - } statementId } + def cancel(statementId: Int): Unit = { + val statementOpt = _statements.synchronized { _statements.get(statementId) } + if (statementOpt.isEmpty) { + return + } + + val statement = statementOpt.get + if (statement.state.get().isOneOf( + StatementState.Available, StatementState.Cancelled, StatementState.Cancelling)) { + return + } else { + // statement 1 is running and statement 2 is waiting. User cancels + // statement 2 then cancels statement 1. The 2nd cancel call will loop and block the 1st + // cancel call since cancelExecutor is single threaded. To avoid this, set the statement + // state to cancelled when cancelling a waiting statement. + statement.compareAndTransit(StatementState.Waiting, StatementState.Cancelled) + statement.compareAndTransit(StatementState.Running, StatementState.Cancelling) + } + + info(s"Cancelling statement $statementId...") + + Future { + val deadline = livyConf.getTimeAsMs(RSCConf.Entry.JOB_CANCEL_TIMEOUT).millis.fromNow + + while (statement.state.get() == StatementState.Cancelling) { + if (deadline.isOverdue()) { + info(s"Failed to cancel statement $statementId.") + statement.compareAndTransit(StatementState.Cancelling, StatementState.Cancelled) + } else { + _sc.foreach(_.cancelJobGroup(statementId.toString)) + if (statement.state.get() == StatementState.Cancelling) { + Thread.sleep(livyConf.getTimeAsMs(RSCConf.Entry.JOB_CANCEL_TRIGGER_INTERVAL)) + } + } + } + + if (statement.state.get() == StatementState.Cancelled) { + info(s"Statement $statementId cancelled.") + } + }(cancelExecutor) + } + def close(): Unit = { - executor.shutdown() + interpreterExecutor.shutdown() + cancelExecutor.shutdown() interpreter.close() } - def clearStatements(): Unit = synchronized { - _statements.clear() + /** + * Get the current progress of given statement id. + */ + def progressOfStatement(stmtId: Int): Double = { + val jobGroup = statementIdToJobGroup(stmtId) + + _sc.map { sc => + val jobIds = sc.statusTracker.getJobIdsForGroup(jobGroup) + val jobs = jobIds.flatMap { id => sc.statusTracker.getJobInfo(id) } + val stages = jobs.flatMap { job => + job.stageIds().flatMap(sc.statusTracker.getStageInfo) + } + + val taskCount = stages.map(_.numTasks).sum + val completedTaskCount = stages.map(_.numCompletedTasks).sum + if (taskCount == 0) { + 0.0 + } else { + completedTaskCount.toDouble / taskCount + } + }.getOrElse(0.0) } private def changeState(newState: SessionState): Unit = { @@ -107,7 +200,7 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un stateChangedCallback(newState) } - private def executeCode(executionCount: Int, code: String): String = synchronized { + private def executeCode(executionCount: Int, code: String): String = { changeState(SessionState.Busy()) def transitToIdle() = { @@ -133,7 +226,7 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un (EXECUTION_COUNT -> executionCount) ~ (ENAME -> "Error") ~ (EVALUE -> "incomplete statement") ~ - (TRACEBACK -> List()) + (TRACEBACK -> Seq.empty[String]) case Interpreter.ExecuteError(ename, evalue, traceback) => transitToIdle() @@ -151,7 +244,7 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un (EXECUTION_COUNT -> executionCount) ~ (ENAME -> "Error") ~ (EVALUE -> f"Interpreter died:\n$message") ~ - (TRACEBACK -> List()) + (TRACEBACK -> Seq.empty[String]) } } catch { case e: Throwable => @@ -163,9 +256,35 @@ class Session(interpreter: Interpreter, stateChangedCallback: SessionState => Un (EXECUTION_COUNT -> executionCount) ~ (ENAME -> f"Internal Error: ${e.getClass.getName}") ~ (EVALUE -> e.getMessage) ~ - (TRACEBACK -> List()) + (TRACEBACK -> Seq.empty[String]) } compact(render(resultInJson)) } + + private def setJobGroup(statementId: Int): String = { + val jobGroup = statementIdToJobGroup(statementId) + val cmd = Kind(interpreter.kind) match { + case Spark() => + // A dummy value to avoid automatic value binding in scala REPL. + s"""val _livyJobGroup$jobGroup = sc.setJobGroup("$jobGroup",""" + + s""""Job group for statement $jobGroup")""" + case PySpark() | PySpark3() => + s"""sc.setJobGroup("$jobGroup", "Job group for statement $jobGroup")""" + case SparkR() => + interpreter.asInstanceOf[SparkRInterpreter].sparkMajorVersion match { + case "1" => + s"""setJobGroup(sc, "$jobGroup", "Job group for statement $jobGroup", """ + + "FALSE)" + case "2" => + s"""setJobGroup("$jobGroup", "Job group for statement $jobGroup", FALSE)""" + } + } + // Set the job group + executeCode(statementId, cmd) + } + + private def statementIdToJobGroup(statementId: Int): String = { + statementId.toString + } } diff --git a/repl/src/main/scala/com/cloudera/livy/repl/SparkContextInitializer.scala b/repl/src/main/scala/com/cloudera/livy/repl/SparkContextInitializer.scala index 33c7f37a3..533741da5 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/SparkContextInitializer.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/SparkContextInitializer.scala @@ -46,7 +46,7 @@ trait SparkContextInitializer extends Logging { val loader = Option(Thread.currentThread().getContextClassLoader) .getOrElse(getClass.getClassLoader) if (loader.getResource("hive-site.xml") == null) { - warn("livy.repl.enableHiveContext is true but no hive-site.xml found on classpath.") + warn("livy.repl.enable-hive-context is true but no hive-site.xml found on classpath.") } sqlContext = Class.forName("org.apache.spark.sql.hive.HiveContext") @@ -82,12 +82,12 @@ trait SparkContextInitializer extends Logging { builder.getClass.getMethod("config", classOf[SparkConf]).invoke(builder, conf) var spark: Object = null - if (conf.get("spark.sql.catalogImplementation", "hive").toLowerCase == "hive") { + if (conf.get("spark.sql.catalogImplementation", "in-memory").toLowerCase == "hive") { if (sparkClz.getMethod("hiveClassesArePresent").invoke(sparkObj).asInstanceOf[Boolean]) { val loader = Option(Thread.currentThread().getContextClassLoader) .getOrElse(getClass.getClassLoader) if (loader.getResource("hive-site.xml") == null) { - warn("livy.repl.enableHiveContext is true but no hive-site.xml found on classpath.") + warn("livy.repl.enable-hive-context is true but no hive-site.xml found on classpath.") } builder.getClass.getMethod("enableHiveSupport").invoke(builder) diff --git a/repl/src/main/scala/com/cloudera/livy/repl/SparkRInterpreter.scala b/repl/src/main/scala/com/cloudera/livy/repl/SparkRInterpreter.scala index 0e5eb271a..469d0a581 100644 --- a/repl/src/main/scala/com/cloudera/livy/repl/SparkRInterpreter.scala +++ b/repl/src/main/scala/com/cloudera/livy/repl/SparkRInterpreter.scala @@ -28,16 +28,21 @@ import scala.collection.JavaConverters._ import scala.reflect.runtime.universe import org.apache.commons.codec.binary.Base64 +import org.apache.commons.lang.StringEscapeUtils import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} import org.json4s._ import org.json4s.JsonDSL._ import com.cloudera.livy.client.common.ClientConf +import com.cloudera.livy.rsc.RSCConf + +private case class RequestResponse(content: String, error: Boolean) // scalastyle:off println object SparkRInterpreter { private val LIVY_END_MARKER = "----LIVY_END_OF_COMMAND----" + private val LIVY_ERROR_MARKER = "----LIVY_END_OF_ERROR----" private val PRINT_MARKER = f"""print("$LIVY_END_MARKER")""" private val EXPECTED_OUTPUT = f"""[1] "$LIVY_END_MARKER"""" @@ -112,10 +117,11 @@ object SparkRInterpreter { env.put("R_PROFILE_USER", Seq(packageDir, "SparkR", "profile", "general.R").mkString(File.separator)) - builder.redirectError(Redirect.PIPE) + builder.redirectErrorStream(true) val process = builder.start() new SparkRInterpreter(process, backendInstance, backendThread, - conf.get("spark.livy.spark_major_version", "1")) + conf.get("spark.livy.spark_major_version", "1"), + conf.getBoolean("spark.repl.enableHiveContext", false)) } catch { case e: Exception => if (backendThread != null) { @@ -126,30 +132,41 @@ object SparkRInterpreter { } } -class SparkRInterpreter(process: Process, backendInstance: Any, backendThread: Thread, - sparkMajorVersion: String) extends ProcessInterpreter(process) { +class SparkRInterpreter(process: Process, + backendInstance: Any, + backendThread: Thread, + val sparkMajorVersion: String, + hiveEnabled: Boolean) + extends ProcessInterpreter(process) { import SparkRInterpreter._ implicit val formats = DefaultFormats private[this] var executionCount = 0 - override def kind: String = "sparkR" - private[this] val isStarted = new CountDownLatch(1); + override def kind: String = "sparkr" + private[this] val isStarted = new CountDownLatch(1) final override protected def waitUntilReady(): Unit = { // Set the option to catch and ignore errors instead of halting. sendRequest("options(error = dump.frames)") if (!ClientConf.TEST_MODE) { sendRequest("library(SparkR)") - if (sparkMajorVersion >= "2") { - sendRequest("spark <- SparkR::sparkR.session()") + if (hiveEnabled) { + sendRequest("spark <- SparkR::sparkR.session()") + } else { + sendRequest("spark <- SparkR::sparkR.session(enableHiveSupport=FALSE)") + } sendRequest( """sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark)""") } else { sendRequest("sc <- sparkR.init()") - sendRequest("sqlContext <- sparkRSQL.init(sc)") + if (hiveEnabled) { + sendRequest("sqlContext <- sparkRHive.init(sc)") + } else { + sendRequest("sqlContext <- sparkRSQL.init(sc)") + } } } @@ -172,38 +189,46 @@ class SparkRInterpreter(process: Process, backendInstance: Any, backendThread: T } try { - var content: JObject = TEXT_PLAIN -> (sendRequest(code) + takeErrorLines()) - - // If we rendered anything, pass along the last image. - tempFile.foreach { case file => - val bytes = Files.readAllBytes(file) - if (bytes.nonEmpty) { - val image = Base64.encodeBase64String(bytes) - content = content ~ (IMAGE_PNG -> image) + val response = sendRequest(code) + + if (response.error) { + Interpreter.ExecuteError("Error", response.content) + } else { + var content: JObject = TEXT_PLAIN -> response.content + + // If we rendered anything, pass along the last image. + tempFile.foreach { case file => + val bytes = Files.readAllBytes(file) + if (bytes.nonEmpty) { + val image = Base64.encodeBase64String(bytes) + content = content ~ (IMAGE_PNG -> image) + } } + + Interpreter.ExecuteSuccess(content) } - Interpreter.ExecuteSuccess(content) } catch { case e: Error => - val message = Seq(e.output, takeErrorLines()).mkString("\n") - Interpreter.ExecuteError("Error", message) + Interpreter.ExecuteError("Error", e.output) case e: Exited => - Interpreter.ExecuteAborted(takeErrorLines()) + Interpreter.ExecuteAborted(e.getMessage) } finally { tempFile.foreach(Files.delete) } } - private def sendRequest(code: String): String = { - stdin.println(code) + private def sendRequest(code: String): RequestResponse = { + stdin.println(s"""tryCatch(eval(parse(text="${StringEscapeUtils.escapeJava(code)}")) + |,error = function(e) sprintf("%s%s", e, "${LIVY_ERROR_MARKER}")) + """.stripMargin) stdin.flush() stdin.println(PRINT_MARKER) stdin.flush() - readTo(EXPECTED_OUTPUT) + readTo(EXPECTED_OUTPUT, LIVY_ERROR_MARKER) } override protected def sendShutdownRequest() = { @@ -227,7 +252,10 @@ class SparkRInterpreter(process: Process, backendInstance: Any, backendThread: T } @tailrec - private def readTo(marker: String, output: StringBuilder = StringBuilder.newBuilder): String = { + private def readTo( + marker: String, + errorMarker: String, + output: StringBuilder = StringBuilder.newBuilder): RequestResponse = { var char = readChar(output) // Remove any ANSI color codes which match the pattern "\u001b\\[[0-9;]*[mG]". @@ -244,15 +272,25 @@ class SparkRInterpreter(process: Process, backendInstance: Any, backendThread: T } if (output.endsWith(marker)) { - val result = output.toString() - result.substring(0, result.length - marker.length) - .stripPrefix("\n") - .stripSuffix("\n") + var result = stripMarker(output.toString(), marker) + + if (result.endsWith(errorMarker + "\"")) { + result = stripMarker(result, "\\n" + errorMarker) + RequestResponse(result, error = true) + } else { + RequestResponse(result, error = false) + } } else { - readTo(marker, output) + readTo(marker, errorMarker, output) } } + private def stripMarker(result: String, marker: String): String = { + result.replace(marker, "") + .stripPrefix("\n") + .stripSuffix("\n") + } + private def readChar(output: StringBuilder): Char = { val byte = stdout.read() if (byte == -1) { diff --git a/repl/src/test/scala/com/cloudera/livy/repl/BaseSessionSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/BaseSessionSpec.scala index 73192ff1f..00733e464 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/BaseSessionSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/BaseSessionSpec.scala @@ -18,6 +18,7 @@ package com.cloudera.livy.repl +import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.Await @@ -29,6 +30,7 @@ import org.scalatest.{FlatSpec, Matchers} import org.scalatest.concurrent.Eventually._ import com.cloudera.livy.LivyBaseUnitTestSuite +import com.cloudera.livy.rsc.RSCConf import com.cloudera.livy.rsc.driver.{Statement, StatementState} import com.cloudera.livy.sessions.SessionState @@ -36,18 +38,21 @@ abstract class BaseSessionSpec extends FlatSpec with Matchers with LivyBaseUnitT implicit val formats = DefaultFormats + private val rscConf = new RSCConf(new Properties()) + protected def execute(session: Session)(code: String): Statement = { val id = session.execute(code) eventually(timeout(30 seconds), interval(100 millis)) { val s = session.statements(id) - s.state shouldBe StatementState.Available + s.state.get() shouldBe StatementState.Available s } } protected def withSession(testCode: Session => Any): Unit = { val stateChangedCalled = new AtomicInteger() - val session = new Session(createInterpreter(), { _ => stateChangedCalled.incrementAndGet() }) + val session = + new Session(rscConf, createInterpreter(), { _ => stateChangedCalled.incrementAndGet() }) try { // Session's constructor should fire an initial state change event. stateChangedCalled.intValue() shouldBe 1 @@ -64,7 +69,7 @@ abstract class BaseSessionSpec extends FlatSpec with Matchers with LivyBaseUnitT protected def createInterpreter(): Interpreter it should "start in the starting or idle state" in { - val session = new Session(createInterpreter()) + val session = new Session(rscConf, createInterpreter()) val future = session.start() try { eventually(timeout(30 seconds), interval(100 millis)) { diff --git a/repl/src/test/scala/com/cloudera/livy/repl/PythonInterpreterSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/PythonInterpreterSpec.scala index 754c2f270..a4a40af96 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/PythonInterpreterSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/PythonInterpreterSpec.scala @@ -19,10 +19,11 @@ package com.cloudera.livy.repl import org.apache.spark.SparkConf -import org.json4s.{DefaultFormats, JValue} +import org.json4s.{DefaultFormats, JNull, JValue} import org.json4s.JsonDSL._ import org.scalatest._ +import com.cloudera.livy.rsc.RSCConf import com.cloudera.livy.sessions._ abstract class PythonBaseInterpreterSpec extends BaseInterpreterSpec { @@ -122,6 +123,46 @@ abstract class PythonBaseInterpreterSpec extends BaseInterpreterSpec { )) } + it should "do table magic with None type value" in withInterpreter { interpreter => + val response = interpreter.execute( + """x = [{"a":"1", "b":None}, {"a":"2", "b":2}] + |%table x + """.stripMargin) + + response should equal(Interpreter.ExecuteSuccess( + APPLICATION_LIVY_TABLE_JSON -> ( + ("headers" -> List( + ("type" -> "STRING_TYPE") ~ ("name" -> "a"), + ("type" -> "INT_TYPE") ~ ("name" -> "b") + )) ~ + ("data" -> List( + List[JValue]("1", JNull), + List[JValue]("2", 2) + )) + ) + )) + } + + it should "do table magic with None type Row" in withInterpreter { interpreter => + val response = interpreter.execute( + """x = [{"a":None, "b":None}, {"a":"2", "b":2}] + |%table x + """.stripMargin) + + response should equal(Interpreter.ExecuteSuccess( + APPLICATION_LIVY_TABLE_JSON -> ( + ("headers" -> List( + ("type" -> "STRING_TYPE") ~ ("name" -> "a"), + ("type" -> "INT_TYPE") ~ ("name" -> "b") + )) ~ + ("data" -> List( + List[JValue](JNull, JNull), + List[JValue]("2", 2) + )) + ) + )) + } + it should "allow magic inside statements" in withInterpreter { interpreter => val response = interpreter.execute( """x = [[1, 'a'], [3, 'b']] diff --git a/repl/src/test/scala/com/cloudera/livy/repl/PythonSessionSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/PythonSessionSpec.scala index 4582acdf1..1e5958dea 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/PythonSessionSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/PythonSessionSpec.scala @@ -23,6 +23,7 @@ import org.json4s.Extraction import org.json4s.jackson.JsonMethods.parse import org.scalatest._ +import com.cloudera.livy.rsc.RSCConf import com.cloudera.livy.sessions._ abstract class PythonSessionSpec extends BaseSessionSpec { diff --git a/repl/src/test/scala/com/cloudera/livy/repl/ScalaInterpreterSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/ScalaInterpreterSpec.scala index 63076e70a..012679621 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/ScalaInterpreterSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/ScalaInterpreterSpec.scala @@ -22,11 +22,14 @@ import org.apache.spark.SparkConf import org.json4s.{DefaultFormats, JValue} import org.json4s.JsonDSL._ +import com.cloudera.livy.rsc.RSCConf + class ScalaInterpreterSpec extends BaseInterpreterSpec { implicit val formats = DefaultFormats - override def createInterpreter(): Interpreter = new SparkInterpreter(new SparkConf()) + override def createInterpreter(): Interpreter = + new SparkInterpreter(new SparkConf()) it should "execute `1 + 2` == 3" in withInterpreter { interpreter => val response = interpreter.execute("1 + 2") diff --git a/repl/src/test/scala/com/cloudera/livy/repl/SessionSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/SessionSpec.scala index 12dcaa802..203d15ef1 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/SessionSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/SessionSpec.scala @@ -18,9 +18,9 @@ package com.cloudera.livy.repl +import java.util.Properties import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit} - import org.mockito.Mockito.when import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -32,18 +32,27 @@ import org.scalatest.time._ import com.cloudera.livy.LivyBaseUnitTestSuite import com.cloudera.livy.repl.Interpreter.ExecuteResponse +import com.cloudera.livy.rsc.RSCConf class SessionSpec extends FunSpec with Eventually with LivyBaseUnitTestSuite { override implicit val patienceConfig = PatienceConfig(timeout = scaled(Span(10, Seconds)), interval = scaled(Span(100, Millis))) + private val rscConf = new RSCConf(new Properties()) + describe("Session") { it("should call state changed callbacks in happy path") { - val expectedStateTransitions = Array("not_started", "starting", "idle", "busy", "idle") + val expectedStateTransitions = + Array("not_started", "starting", "idle", "busy", "idle", "busy", "idle") val actualStateTransitions = new ConcurrentLinkedQueue[String]() val interpreter = mock[Interpreter] - val session = new Session(interpreter, { s => actualStateTransitions.add(s.toString) }) + when(interpreter.kind).thenAnswer(new Answer[String] { + override def answer(invocationOnMock: InvocationOnMock): String = "spark" + }) + + val session = + new Session(rscConf, interpreter, { s => actualStateTransitions.add(s.toString) }) session.start() @@ -55,10 +64,15 @@ class SessionSpec extends FunSpec with Eventually with LivyBaseUnitTestSuite { } it("should not transit to idle if there're any pending statements.") { - val expectedStateTransitions = Array("not_started", "busy", "busy", "idle") + val expectedStateTransitions = + Array("not_started", "busy", "busy", "busy", "idle", "busy", "idle") val actualStateTransitions = new ConcurrentLinkedQueue[String]() val interpreter = mock[Interpreter] + when(interpreter.kind).thenAnswer(new Answer[String] { + override def answer(invocationOnMock: InvocationOnMock): String = "spark" + }) + val blockFirstExecuteCall = new CountDownLatch(1) when(interpreter.execute("")).thenAnswer(new Answer[Interpreter.ExecuteResponse] { override def answer(invocation: InvocationOnMock): ExecuteResponse = { @@ -66,7 +80,8 @@ class SessionSpec extends FunSpec with Eventually with LivyBaseUnitTestSuite { null } }) - val session = new Session(interpreter, { s => actualStateTransitions.add(s.toString) }) + val session = + new Session(rscConf, interpreter, { s => actualStateTransitions.add(s.toString) }) for (_ <- 1 to 2) { session.execute("") @@ -77,5 +92,36 @@ class SessionSpec extends FunSpec with Eventually with LivyBaseUnitTestSuite { actualStateTransitions.toArray shouldBe expectedStateTransitions } } + + it("should remove old statements when reaching threshold") { + val interpreter = mock[Interpreter] + when(interpreter.kind).thenAnswer(new Answer[String] { + override def answer(invocationOnMock: InvocationOnMock): String = "spark" + }) + + rscConf.set(RSCConf.Entry.RETAINED_STATEMENT_NUMBER, 2) + val session = new Session(rscConf, interpreter) + session.start() + + session.statements.size should be (0) + session.execute("") + session.statements.size should be (1) + session.statements.map(_._1).toSet should be (Set(0)) + session.execute("") + session.statements.size should be (2) + session.statements.map(_._1).toSet should be (Set(0, 1)) + session.execute("") + eventually { + session.statements.size should be (2) + session.statements.map(_._1).toSet should be (Set(1, 2)) + } + + // Continue submitting statements, total statements in memory should be 2. + session.execute("") + eventually { + session.statements.size should be (2) + session.statements.map(_._1).toSet should be (Set(2, 3)) + } + } } } diff --git a/repl/src/test/scala/com/cloudera/livy/repl/SparkRInterpreterSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/SparkRInterpreterSpec.scala index 989c4fee5..61f1a3615 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/SparkRInterpreterSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/SparkRInterpreterSpec.scala @@ -23,6 +23,8 @@ import org.json4s.{DefaultFormats, JValue} import org.json4s.JsonDSL._ import org.scalatest._ +import com.cloudera.livy.rsc.RSCConf + class SparkRInterpreterSpec extends BaseInterpreterSpec { implicit val formats = DefaultFormats @@ -81,8 +83,25 @@ class SparkRInterpreterSpec extends BaseInterpreterSpec { it should "report an error if accessing an unknown variable" in withInterpreter { interpreter => val response = interpreter.execute("x") + assert(response.isInstanceOf[Interpreter.ExecuteError]) + val errorResponse = response.asInstanceOf[Interpreter.ExecuteError] + errorResponse.ename should be ("Error") + assert(errorResponse.evalue.contains("object 'x' not found")) + } + + + it should "not hang when executing incomplete statements" in withInterpreter { interpreter => + val response = interpreter.execute("x[") + response should equal(Interpreter.ExecuteError( + "Error", + """[1] "Error in parse(text = \"x[\"): :2:0: unexpected end of input\n1: x[\n ^"""" + )) + } + + it should "escape the statement" in withInterpreter { interpreter => + val response = interpreter.execute("print(\"a\")") response should equal(Interpreter.ExecuteSuccess( - TEXT_PLAIN -> "Error: object 'x' not found" + TEXT_PLAIN -> "[1] \"a\"" )) } diff --git a/repl/src/test/scala/com/cloudera/livy/repl/SparkRSessionSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/SparkRSessionSpec.scala index b42b4f3ff..c60420533 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/SparkRSessionSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/SparkRSessionSpec.scala @@ -22,6 +22,8 @@ import org.apache.spark.SparkConf import org.json4s.Extraction import org.json4s.jackson.JsonMethods.parse +import com.cloudera.livy.rsc.RSCConf + class SparkRSessionSpec extends BaseSessionSpec { override protected def withFixture(test: NoArgTest) = { @@ -129,15 +131,11 @@ class SparkRSessionSpec extends BaseSessionSpec { statement.id should equal (0) val result = parse(statement.output) - val expectedResult = Extraction.decompose(Map( - "status" -> "ok", - "execution_count" -> 0, - "data" -> Map( - "text/plain" -> "Error: object 'x' not found" - ) - )) - - result should equal (expectedResult) + (result \ "status").extract[String] should be ("error") + (result \ "execution_count").extract[Int] should be (0) + (result \ "ename").extract[String] should be ("Error") + assert((result \ "evalue").extract[String].contains("object 'x' not found")) + (result \ "traceback").extract[List[String]] should be (List()) } } diff --git a/repl/src/test/scala/com/cloudera/livy/repl/SparkSessionSpec.scala b/repl/src/test/scala/com/cloudera/livy/repl/SparkSessionSpec.scala index b36bb1a59..52b6b4260 100644 --- a/repl/src/test/scala/com/cloudera/livy/repl/SparkSessionSpec.scala +++ b/repl/src/test/scala/com/cloudera/livy/repl/SparkSessionSpec.scala @@ -18,10 +18,17 @@ package com.cloudera.livy.repl +import scala.concurrent.duration._ +import scala.language.postfixOps + import org.apache.spark.SparkConf import org.json4s.Extraction -import org.json4s.jackson.JsonMethods.parse import org.json4s.JsonAST.JValue +import org.json4s.jackson.JsonMethods.parse +import org.scalatest.concurrent.Eventually._ + +import com.cloudera.livy.rsc.RSCConf +import com.cloudera.livy.rsc.driver.StatementState class SparkSessionSpec extends BaseSessionSpec { @@ -194,4 +201,74 @@ class SparkSessionSpec extends BaseSessionSpec { result should equal (expectedResult) } + + it should "cancel spark jobs" in withSession { session => + val stmtId = session.execute( + """sc.parallelize(0 to 10).map { i => Thread.sleep(10000); i + 1 }.collect""".stripMargin) + eventually(timeout(30 seconds), interval(100 millis)) { + assert(session.statements(stmtId).state.get() == StatementState.Running) + } + session.cancel(stmtId) + + eventually(timeout(30 seconds), interval(100 millis)) { + assert(session.statements(stmtId).state.get() == StatementState.Cancelled) + session.statements(stmtId).output should include ( + "Job 0 cancelled part of cancelled job group 0") + } + } + + it should "cancel waiting statement" in withSession { session => + val stmtId1 = session.execute( + """sc.parallelize(0 to 10).map { i => Thread.sleep(10000); i + 1 }.collect""".stripMargin) + val stmtId2 = session.execute( + """sc.parallelize(0 to 10).map { i => Thread.sleep(10000); i + 1 }.collect""".stripMargin) + eventually(timeout(30 seconds), interval(100 millis)) { + assert(session.statements(stmtId1).state.get() == StatementState.Running) + } + + assert(session.statements(stmtId2).state.get() == StatementState.Waiting) + + session.cancel(stmtId2) + assert(session.statements(stmtId2).state.get() == StatementState.Cancelled) + + session.cancel(stmtId1) + assert(session.statements(stmtId1).state.get() == StatementState.Cancelling) + eventually(timeout(30 seconds), interval(100 millis)) { + assert(session.statements(stmtId1).state.get() == StatementState.Cancelled) + session.statements(stmtId1).output should include ( + "Job 0 cancelled part of cancelled job group 0") + } + } + + it should "correctly calculate progress" in withSession { session => + val executeCode = + """ + |sc.parallelize(1 to 2, 2).map(i => (i, 1)).collect() + """.stripMargin + + val stmtId = session.execute(executeCode) + eventually(timeout(30 seconds), interval(100 millis)) { + session.progressOfStatement(stmtId) should be(1.0) + } + } + + it should "not generate Spark jobs for plain Scala code" in withSession { session => + val executeCode = """1 + 1""" + + val stmtId = session.execute(executeCode) + session.progressOfStatement(stmtId) should be (0.0) + } + + it should "handle multiple jobs in one statement" in withSession { session => + val executeCode = + """ + |sc.parallelize(1 to 2, 2).map(i => (i, 1)).collect() + |sc.parallelize(1 to 2, 2).map(i => (i, 1)).collect() + """.stripMargin + + val stmtId = session.execute(executeCode) + eventually(timeout(30 seconds), interval(100 millis)) { + session.progressOfStatement(stmtId) should be(1.0) + } + } } diff --git a/rsc/pom.xml b/rsc/pom.xml index 3c6153ac7..7ff8c2a91 100644 --- a/rsc/pom.xml +++ b/rsc/pom.xml @@ -21,12 +21,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT com.cloudera.livy livy-rsc - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/BaseProtocol.java b/rsc/src/main/java/com/cloudera/livy/rsc/BaseProtocol.java index 0e234f3be..419c1f723 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/BaseProtocol.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/BaseProtocol.java @@ -210,7 +210,18 @@ public ReplState(String state) { public ReplState() { this(null); } + } + + public static class CancelReplJobRequest { + public final int id; + + public CancelReplJobRequest(int id) { + this.id = id; + } + public CancelReplJobRequest() { + this(-1); + } } public static class InitializationError { diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/ContextLauncher.java b/rsc/src/main/java/com/cloudera/livy/rsc/ContextLauncher.java index d1692f184..9a5e447ea 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/ContextLauncher.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/ContextLauncher.java @@ -69,10 +69,10 @@ class ContextLauncher { private static final String SPARK_ARCHIVES_KEY = "spark.yarn.dist.archives"; private static final String SPARK_HOME_ENV = "SPARK_HOME"; - static Promise create(RSCClientFactory factory, RSCConf conf) + static DriverProcessInfo create(RSCClientFactory factory, RSCConf conf) throws IOException { ContextLauncher launcher = new ContextLauncher(factory, conf); - return launcher.promise; + return new DriverProcessInfo(launcher.promise, launcher.child.child); } private final Promise promise; @@ -105,7 +105,9 @@ private ContextLauncher(RSCClientFactory factory, RSCConf conf) throws IOExcepti @Override public void onFailure(Throwable error) throws Exception { // If promise is cancelled or failed, make sure spark-submit is not leaked. - child.kill(); + if (child != null) { + child.kill(); + } } }); @@ -137,10 +139,12 @@ private void connectTimeout(RegistrationHandler handler) { private void dispose(boolean forceKill) { factory.getServer().unregisterClient(clientId); try { - if (forceKill) { - child.kill(); - } else { - child.detach(); + if (child != null) { + if (forceKill) { + child.kill(); + } else { + child.detach(); + } } } finally { factory.unref(); @@ -300,33 +304,6 @@ private static File writeConfToFile(RSCConf conf) throws IOException { return file; } - private static class Redirector implements Runnable { - - private final BufferedReader in; - - Redirector(InputStream in) { - this.in = new BufferedReader(new InputStreamReader(in)); - } - - @Override - public void run() { - try { - String line = null; - while ((line = in.readLine()) != null) { - LOG.info(line); - } - } catch (Exception e) { - LOG.warn("Error in redirector thread.", e); - } - - try { - in.close(); - } catch (IOException ioe) { - LOG.warn("Error closing child stream.", ioe); - } - } - - } private class RegistrationHandler extends BaseProtocol implements RpcServer.ClientCallback { @@ -379,8 +356,6 @@ private static class ChildProcess { private final Promise promise; private final Process child; private final Thread monitor; - private final Thread stdout; - private final Thread stderr; private final File confFile; public ChildProcess(RSCConf conf, Promise promise, Runnable child, File confFile) { @@ -388,8 +363,6 @@ public ChildProcess(RSCConf conf, Promise promise, Runnable child, File confF this.promise = promise; this.monitor = monitor(child, CHILD_IDS.incrementAndGet()); this.child = null; - this.stdout = null; - this.stderr = null; this.confFile = confFile; } @@ -398,8 +371,6 @@ public ChildProcess(RSCConf conf, Promise promise, final Process childProc, F this.conf = conf; this.promise = promise; this.child = childProc; - this.stdout = redirect("stdout-redir-" + childId, child.getInputStream()); - this.stderr = redirect("stderr-redir-" + childId, child.getErrorStream()); this.confFile = confFile; Runnable monitorTask = new Runnable() { @@ -446,23 +417,6 @@ public void kill() { } public void detach() { - if (stdout != null) { - stdout.interrupt(); - try { - stdout.join(conf.getTimeAsMs(CLIENT_SHUTDOWN_TIMEOUT)); - } catch (InterruptedException ie) { - LOG.info("Interrupted while waiting for child stdout to finish."); - } - } - if (stderr != null) { - stderr.interrupt(); - try { - stderr.join(conf.getTimeAsMs(CLIENT_SHUTDOWN_TIMEOUT)); - } catch (InterruptedException ie) { - LOG.info("Interrupted while waiting for child stderr to finish."); - } - } - try { monitor.join(conf.getTimeAsMs(CLIENT_SHUTDOWN_TIMEOUT)); } catch (InterruptedException ie) { @@ -470,14 +424,6 @@ public void detach() { } } - private Thread redirect(String name, InputStream in) { - Thread thread = new Thread(new Redirector(in)); - thread.setName(name); - thread.setDaemon(true); - thread.start(); - return thread; - } - private Thread monitor(final Runnable task, int childId) { Runnable wrappedTask = new Runnable() { @Override diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/DriverProcessInfo.java b/rsc/src/main/java/com/cloudera/livy/rsc/DriverProcessInfo.java new file mode 100644 index 000000000..a224fd628 --- /dev/null +++ b/rsc/src/main/java/com/cloudera/livy/rsc/DriverProcessInfo.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.rsc; + +import io.netty.util.concurrent.Promise; + +/** + * Information about driver process and @{@link ContextInfo} + */ +public class DriverProcessInfo { + + private Promise contextInfo; + private transient Process driverProcess; + + public DriverProcessInfo(Promise contextInfo, Process driverProcess) { + this.contextInfo = contextInfo; + this.driverProcess = driverProcess; + } + + public Promise getContextInfo() { + return contextInfo; + } + + public Process getDriverProcess() { + return driverProcess; + } +} diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/RSCClient.java b/rsc/src/main/java/com/cloudera/livy/rsc/RSCClient.java index aca9cc468..11cb0f6c0 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/RSCClient.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/RSCClient.java @@ -60,12 +60,14 @@ public class RSCClient implements LivyClient { private final Promise serverUriPromise; private ContextInfo contextInfo; + private Process driverProcess; private volatile boolean isAlive; private volatile String replState; - RSCClient(RSCConf conf, Promise ctx) throws IOException { + RSCClient(RSCConf conf, Promise ctx, Process driverProcess) throws IOException { this.conf = conf; this.contextInfoPromise = ctx; + this.driverProcess = driverProcess; this.jobs = new ConcurrentHashMap<>(); this.protocol = new ClientProtocol(); this.driverRpc = ImmediateEventExecutor.INSTANCE.newPromise(); @@ -94,6 +96,14 @@ public void onFailure(Throwable error) { isAlive = true; } + public boolean isAlive() { + return isAlive; + } + + public Process getDriverProcess() { + return driverProcess; + } + private synchronized void connectToContext(final ContextInfo info) throws Exception { this.contextInfo = info; @@ -279,6 +289,10 @@ public Future submitReplCode(String code) throws Exception { return deferredCall(new BaseProtocol.ReplJobRequest(code), Integer.class); } + public void cancelReplCode(int statementId) throws Exception { + deferredCall(new BaseProtocol.CancelReplJobRequest(statementId), Void.class); + } + public Future getReplJobResults(Integer from, Integer size) throws Exception { return deferredCall(new BaseProtocol.GetReplJobResults(from, size), ReplJobResults.class); } diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/RSCClientFactory.java b/rsc/src/main/java/com/cloudera/livy/rsc/RSCClientFactory.java index d50c62ef8..27540bb95 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/RSCClientFactory.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/RSCClientFactory.java @@ -57,14 +57,17 @@ public LivyClient createClient(URI uri, Properties config) { boolean needsServer = false; try { Promise info; + Process driverProcess = null; if (uri.getUserInfo() != null && uri.getHost() != null && uri.getPort() > 0) { info = createContextInfo(uri); } else { needsServer = true; ref(lconf); - info = ContextLauncher.create(this, lconf); + DriverProcessInfo processInfo = ContextLauncher.create(this, lconf); + info = processInfo.getContextInfo(); + driverProcess = processInfo.getDriverProcess(); } - return new RSCClient(lconf, info); + return new RSCClient(lconf, info, driverProcess); } catch (Exception e) { if (needsServer) { unref(); diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/RSCConf.java b/rsc/src/main/java/com/cloudera/livy/rsc/RSCConf.java index 269a1b271..afd935d57 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/RSCConf.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/RSCConf.java @@ -21,15 +21,13 @@ import java.net.Inet4Address; import java.net.InetAddress; import java.net.NetworkInterface; +import java.util.Collections; import java.util.Enumeration; import java.util.HashMap; import java.util.Map; import java.util.Properties; import javax.security.sasl.Sasl; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.cloudera.livy.client.common.ClientConf; public class RSCConf extends ClientConf { @@ -37,14 +35,14 @@ public class RSCConf extends ClientConf { public static final String SPARK_CONF_PREFIX = "spark."; public static final String LIVY_SPARK_PREFIX = SPARK_CONF_PREFIX + "__livy__."; - private static final Logger LOG = LoggerFactory.getLogger(RSCConf.class); + private static final String RSC_CONF_PREFIX = "livy.rsc."; public static enum Entry implements ConfEntry { CLIENT_ID("client.auth.id", null), CLIENT_SECRET("client.auth.secret", null), - CLIENT_IN_PROCESS("client.do_not_use.run_driver_in_process", false), - CLIENT_SHUTDOWN_TIMEOUT("client.shutdown_timeout", "10s"), - DRIVER_CLASS("driver_class", null), + CLIENT_IN_PROCESS("client.do-not-use.run-driver-in-process", false), + CLIENT_SHUTDOWN_TIMEOUT("client.shutdown-timeout", "10s"), + DRIVER_CLASS("driver-class", null), SESSION_KIND("session.kind", null), LIVY_JARS("jars", null), @@ -53,12 +51,14 @@ public static enum Entry implements ConfEntry { // Address for the RSC driver to connect back with it's connection info. LAUNCHER_ADDRESS("launcher.address", null), + LAUNCHER_PORT_RANGE("launcher.port.range", "10000~10010"), + // Setting up of this propety by user has no benefit. It is currently being used + // to pass port information from ContextLauncher to RSCDriver LAUNCHER_PORT("launcher.port", -1), - // How long will the RSC wait for a connection for a Livy server before shutting itself down. - SERVER_IDLE_TIMEOUT("server.idle_timeout", "10m"), + SERVER_IDLE_TIMEOUT("server.idle-timeout", "10m"), - PROXY_USER("proxy_user", null), + PROXY_USER("proxy-user", null), RPC_SERVER_ADDRESS("rpc.server.address", null), RPC_CLIENT_HANDSHAKE_TIMEOUT("server.connect.timeout", "90s"), @@ -71,13 +71,19 @@ public static enum Entry implements ConfEntry { SASL_MECHANISMS("rpc.sasl.mechanisms", "DIGEST-MD5"), SASL_QOP("rpc.sasl.qop", null), - TEST_STUCK_END_SESSION("test.do_not_use.stuck_end_session", false); + TEST_STUCK_END_SESSION("test.do-not-use.stuck-end-session", false), + TEST_STUCK_START_DRIVER("test.do-not-use.stuck-start-driver", false), + + JOB_CANCEL_TRIGGER_INTERVAL("job-cancel.trigger-interval", "100ms"), + JOB_CANCEL_TIMEOUT("job-cancel.timeout", "30s"), + + RETAINED_STATEMENT_NUMBER("retained-statements", 100); private final String key; private final Object dflt; private Entry(String key, Object dflt) { - this.key = "livy.rsc." + key; + this.key = RSC_CONF_PREFIX + key; this.dflt = dflt; } @@ -88,6 +94,10 @@ private Entry(String key, Object dflt) { public Object dflt() { return dflt; } } + public RSCConf() { + this(new Properties()); + } + public RSCConf(Properties config) { super(config); } @@ -136,4 +146,67 @@ public String findLocalAddress() throws IOException { return address.getCanonicalHostName(); } + private static final Map configsWithAlternatives + = Collections.unmodifiableMap(new HashMap() {{ + put(RSCConf.Entry.CLIENT_IN_PROCESS.key, DepConf.CLIENT_IN_PROCESS); + put(RSCConf.Entry.CLIENT_SHUTDOWN_TIMEOUT.key, DepConf.CLIENT_SHUTDOWN_TIMEOUT); + put(RSCConf.Entry.DRIVER_CLASS.key, DepConf.DRIVER_CLASS); + put(RSCConf.Entry.SERVER_IDLE_TIMEOUT.key, DepConf.SERVER_IDLE_TIMEOUT); + put(RSCConf.Entry.PROXY_USER.key, DepConf.PROXY_USER); + put(RSCConf.Entry.TEST_STUCK_END_SESSION.key, DepConf.TEST_STUCK_END_SESSION); + put(RSCConf.Entry.TEST_STUCK_START_DRIVER.key, DepConf.TEST_STUCK_START_DRIVER); + put(RSCConf.Entry.JOB_CANCEL_TRIGGER_INTERVAL.key, DepConf.JOB_CANCEL_TRIGGER_INTERVAL); + put(RSCConf.Entry.JOB_CANCEL_TIMEOUT.key, DepConf.JOB_CANCEL_TIMEOUT); + put(RSCConf.Entry.RETAINED_STATEMENT_NUMBER.key, DepConf.RETAINED_STATEMENT_NUMBER); + }}); + + // Maps deprecated key to DeprecatedConf with the same key. + // There are no deprecated configs without alternatives currently. + private static final Map deprecatedConfigs + = Collections.unmodifiableMap(new HashMap()); + + protected Map getConfigsWithAlternatives() { + return configsWithAlternatives; + } + + protected Map getDeprecatedConfigs() { + return deprecatedConfigs; + } + + static enum DepConf implements DeprecatedConf { + CLIENT_IN_PROCESS("client.do_not_use.run_driver_in_process", "0.4"), + CLIENT_SHUTDOWN_TIMEOUT("client.shutdown_timeout", "0.4"), + DRIVER_CLASS("driver_class", "0.4"), + SERVER_IDLE_TIMEOUT("server.idle_timeout", "0.4"), + PROXY_USER("proxy_user", "0.4"), + TEST_STUCK_END_SESSION("test.do_not_use.stuck_end_session", "0.4"), + TEST_STUCK_START_DRIVER("test.do_not_use.stuck_start_driver", "0.4"), + JOB_CANCEL_TRIGGER_INTERVAL("job_cancel.trigger_interval", "0.4"), + JOB_CANCEL_TIMEOUT("job_cancel.timeout", "0.4"), + RETAINED_STATEMENT_NUMBER("retained_statements", "0.4"); + + private final String key; + private final String version; + private final String deprecationMessage; + + private DepConf(String key, String version) { + this(key, version, ""); + } + + private DepConf(String key, String version, String deprecationMessage) { + this.key = RSC_CONF_PREFIX + key; + this.version = version; + this.deprecationMessage = deprecationMessage; + } + + @Override + public String key() { return key; } + + @Override + public String version() { return version; } + + @Override + public String deprecationMessage() { return deprecationMessage; } + } + } diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/Utils.java b/rsc/src/main/java/com/cloudera/livy/rsc/Utils.java index aa4efae69..993ea1e2f 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/Utils.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/Utils.java @@ -80,7 +80,7 @@ public Thread newThread(Runnable r) { public static String join(Iterable strs, String sep) { StringBuilder sb = new StringBuilder(); for (String s : strs) { - if (s != null) { + if (s != null && !s.isEmpty()) { sb.append(s).append(sep); } } diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/driver/JobContextImpl.java b/rsc/src/main/java/com/cloudera/livy/rsc/driver/JobContextImpl.java index d86939fa9..de2f6a4bb 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/driver/JobContextImpl.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/driver/JobContextImpl.java @@ -18,7 +18,9 @@ package com.cloudera.livy.rsc.driver; import java.io.File; +import java.lang.reflect.Method; +import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaFutureAction; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -41,6 +43,7 @@ class JobContextImpl implements JobContext { private volatile HiveContext hivectx; private volatile JavaStreamingContext streamingctx; private final RSCDriver driver; + private volatile Object sparksession; public JobContextImpl(JavaSparkContext sc, File localTmpDir, RSCDriver driver) { this.sc = sc; @@ -53,6 +56,30 @@ public JavaSparkContext sc() { return sc; } + @Override + public Object sparkSession() throws Exception { + if (sparksession == null) { + synchronized (this) { + if (sparksession == null) { + try { + Class clz = Class.forName("org.apache.spark.sql.SparkSession$"); + Object spark = clz.getField("MODULE$").get(null); + Method m = clz.getMethod("builder"); + Object builder = m.invoke(spark); + builder.getClass().getMethod("sparkContext", SparkContext.class) + .invoke(builder, sc.sc()); + sparksession = builder.getClass().getMethod("getOrCreate").invoke(builder); + } catch (Exception e) { + LOG.warn("SparkSession is not supported", e); + throw e; + } + } + } + } + + return sparksession; + } + @Override public SQLContext sqlctx() { if (sqlctx == null) { diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/driver/RSCDriver.java b/rsc/src/main/java/com/cloudera/livy/rsc/driver/RSCDriver.java index bd55df978..ce567b419 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/driver/RSCDriver.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/driver/RSCDriver.java @@ -163,6 +163,19 @@ private void initializeServer() throws Exception { // on the cluster, it would be tricky to solve that problem in a generic way. livyConf.set(RPC_SERVER_ADDRESS, null); + if (livyConf.getBoolean(TEST_STUCK_START_DRIVER)) { + // Test flag is turned on so we will just infinite loop here. It should cause + // timeout and we should still see yarn application being cleaned up. + LOG.info("Infinite looping as test flag TEST_STUCK_START_SESSION is turned on."); + while(true) { + try { + TimeUnit.MINUTES.sleep(10); + } catch (InterruptedException e) { + LOG.warn("Interrupted during test sleep.", e); + } + } + } + // Bring up the RpcServer an register the secret provided by the Livy server as a client. LOG.info("Starting RPC server..."); this.server = new RpcServer(livyConf); diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/driver/Statement.java b/rsc/src/main/java/com/cloudera/livy/rsc/driver/Statement.java index 6f5235e9e..c1717a97b 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/driver/Statement.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/driver/Statement.java @@ -17,21 +17,41 @@ package com.cloudera.livy.rsc.driver; +import java.util.concurrent.atomic.AtomicReference; + import com.fasterxml.jackson.annotation.JsonRawValue; public class Statement { public final Integer id; - public final StatementState state; + public final AtomicReference state; @JsonRawValue - public final String output; + public volatile String output; + public double progress; public Statement(Integer id, StatementState state, String output) { this.id = id; - this.state = state; + this.state = new AtomicReference<>(state); this.output = output; + this.progress = 0.0; } public Statement() { this(null, null, null); } + + public boolean compareAndTransit(final StatementState from, final StatementState to) { + if (state.compareAndSet(from, to)) { + StatementState.validate(from, to); + return true; + } + return false; + } + + public void updateProgress(double p) { + if (this.state.get().isOneOf(StatementState.Cancelled, StatementState.Available)) { + this.progress = 1.0; + } else { + this.progress = p; + } + } } diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/driver/StatementState.java b/rsc/src/main/java/com/cloudera/livy/rsc/driver/StatementState.java index 590a55362..61a86d0a9 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/driver/StatementState.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/driver/StatementState.java @@ -17,12 +17,20 @@ package com.cloudera.livy.rsc.driver; +import java.util.*; + import com.fasterxml.jackson.annotation.JsonValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public enum StatementState { Waiting("waiting"), Running("running"), - Available("available"); + Available("available"), + Cancelling("cancelling"), + Cancelled("cancelled"); + + private static final Logger LOG = LoggerFactory.getLogger(StatementState.class); private final String state; @@ -35,4 +43,44 @@ public enum StatementState { public String toString() { return state; } + + public boolean isOneOf(StatementState... states) { + for (StatementState s : states) { + if (s == this) { + return true; + } + } + return false; + } + + private static final Map> PREDECESSORS; + + static void put(StatementState key, + Map> map, + StatementState... values) { + map.put(key, Collections.unmodifiableList(Arrays.asList(values))); + } + + static { + final Map> predecessors = + new EnumMap<>(StatementState.class); + put(Waiting, predecessors); + put(Running, predecessors, Waiting); + put(Available, predecessors, Running); + put(Cancelling, predecessors, Running); + put(Cancelled, predecessors, Waiting, Cancelling); + + PREDECESSORS = Collections.unmodifiableMap(predecessors); + } + + static boolean isValid(StatementState from, StatementState to) { + return PREDECESSORS.get(to).contains(from); + } + + static void validate(StatementState from, StatementState to) { + LOG.debug("{} -> {}", from, to); + if (!isValid(from, to)) { + throw new IllegalStateException("Illegal Transition: " + from + " -> " + to); + } + } } diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/rpc/Rpc.java b/rsc/src/main/java/com/cloudera/livy/rsc/rpc/Rpc.java index 454bb27c1..f095effa2 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/rpc/Rpc.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/rpc/Rpc.java @@ -216,7 +216,6 @@ static Rpc createEmbedded(RpcDispatcher dispatcher) { private final AtomicLong rpcId; private final Channel channel; private final EventExecutorGroup egroup; - private final Object channelLock; private volatile RpcDispatcher dispatcher; private Rpc(RSCConf config, Channel channel, EventExecutorGroup egroup) { @@ -224,7 +223,6 @@ private Rpc(RSCConf config, Channel channel, EventExecutorGroup egroup) { Utils.checkArgument(egroup != null); this.config = config; this.channel = channel; - this.channelLock = new Object(); this.dispatcher = null; this.egroup = egroup; this.rpcClosed = new AtomicBoolean(); @@ -256,13 +254,13 @@ public Future call(Object msg) { * @param retType Type of expected reply. * @return A future used to monitor the operation. */ - public Future call(Object msg, Class retType) { + public Future call(final Object msg, Class retType) { Utils.checkArgument(msg != null); Utils.checkState(channel.isOpen(), "RPC channel is closed."); try { final long id = rpcId.getAndIncrement(); final Promise promise = egroup.next().newPromise(); - ChannelFutureListener listener = new ChannelFutureListener() { + final ChannelFutureListener listener = new ChannelFutureListener() { @Override public void operationComplete(ChannelFuture cf) { if (!cf.isSuccess() && !promise.isDone()) { @@ -275,10 +273,14 @@ public void operationComplete(ChannelFuture cf) { }; dispatcher.registerRpc(id, promise, msg.getClass().getName()); - synchronized (channelLock) { - channel.write(new MessageHeader(id, Rpc.MessageType.CALL)).addListener(listener); - channel.writeAndFlush(msg).addListener(listener); - } + channel.eventLoop().submit(new Runnable() { + @Override + public void run() { + channel.write(new MessageHeader(id, Rpc.MessageType.CALL)).addListener(listener); + channel.writeAndFlush(msg).addListener(listener); + } + }); + return promise; } catch (Exception e) { throw Utils.propagate(e); diff --git a/rsc/src/main/java/com/cloudera/livy/rsc/rpc/RpcServer.java b/rsc/src/main/java/com/cloudera/livy/rsc/rpc/RpcServer.java index 1d3e6c501..44db976e5 100644 --- a/rsc/src/main/java/com/cloudera/livy/rsc/rpc/RpcServer.java +++ b/rsc/src/main/java/com/cloudera/livy/rsc/rpc/RpcServer.java @@ -19,7 +19,10 @@ import java.io.Closeable; import java.io.IOException; +import java.net.BindException; import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.SocketException; import java.security.SecureRandom; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -61,18 +64,78 @@ public class RpcServer implements Closeable { private static final SecureRandom RND = new SecureRandom(); private final String address; - private final Channel channel; + private Channel channel; private final EventLoopGroup group; private final int port; private final ConcurrentMap pendingClients; private final RSCConf config; - + private final String portRange; + private static enum PortRangeSchema{START_PORT, END_PORT, MAX}; + private final String PORT_DELIMITER = "~"; + /** + * Creating RPC Server + * @param lconf + * @throws IOException + * @throws InterruptedException + */ public RpcServer(RSCConf lconf) throws IOException, InterruptedException { this.config = lconf; + this.portRange = config.get(LAUNCHER_PORT_RANGE); this.group = new NioEventLoopGroup( - this.config.getInt(RPC_MAX_THREADS), - Utils.newDaemonThreadFactory("RPC-Handler-%d")); - this.channel = new ServerBootstrap() + this.config.getInt(RPC_MAX_THREADS), + Utils.newDaemonThreadFactory("RPC-Handler-%d")); + int [] portData = getPortNumberAndRange(); + int startingPortNumber = portData[PortRangeSchema.START_PORT.ordinal()]; + int endPort = portData[PortRangeSchema.END_PORT.ordinal()]; + boolean isContected = false; + for(int tries = startingPortNumber ; tries<=endPort ; tries++){ + try { + this.channel = getChannel(tries); + isContected = true; + break; + } catch(SocketException e){ + LOG.debug("RPC not able to connect port " + tries + " " + e.getMessage()); + } + } + if(!isContected) { + throw new IOException("Unable to connect to provided ports " + this.portRange); + } + this.port = ((InetSocketAddress) channel.localAddress()).getPort(); + this.pendingClients = new ConcurrentHashMap<>(); + LOG.info("Connected to the port " + this.port); + String address = config.get(RPC_SERVER_ADDRESS); + if (address == null) { + address = config.findLocalAddress(); + } + this.address = address; + } + + /** + * Get Port Numbers + */ + private int[] getPortNumberAndRange() throws ArrayIndexOutOfBoundsException, + NumberFormatException { + String[] split = this.portRange.split(PORT_DELIMITER); + int [] portRange = new int [PortRangeSchema.MAX.ordinal()]; + try { + portRange[PortRangeSchema.START_PORT.ordinal()] = + Integer.parseInt(split[PortRangeSchema.START_PORT.ordinal()]); + portRange[PortRangeSchema.END_PORT.ordinal()] = + Integer.parseInt(split[PortRangeSchema.END_PORT.ordinal()]); + } catch(ArrayIndexOutOfBoundsException e) { + LOG.error("Port Range format is not correct " + this.portRange); + throw e; + } catch(NumberFormatException e) { + LOG.error("Port are not in numeric format " + this.portRange); + throw e; + } + return portRange; + } + /** + * @throws InterruptedException + **/ + private Channel getChannel(int portNumber) throws BindException, InterruptedException { + Channel channel = new ServerBootstrap() .group(group) .channel(NioServerSocketChannel.class) .childHandler(new ChannelInitializer() { @@ -97,19 +160,11 @@ public void run() { .option(ChannelOption.SO_BACKLOG, 1) .option(ChannelOption.SO_REUSEADDR, true) .childOption(ChannelOption.SO_KEEPALIVE, true) - .bind(0) + .bind(portNumber) .sync() .channel(); - this.port = ((InetSocketAddress) channel.localAddress()).getPort(); - this.pendingClients = new ConcurrentHashMap<>(); - - String address = config.get(RPC_SERVER_ADDRESS); - if (address == null) { - address = config.findLocalAddress(); - } - this.address = address; + return channel; } - /** * Tells the RPC server to expect connections from clients. * @@ -310,3 +365,4 @@ private ClientInfo(String id, String secret, ClientCallback callback) { } } + diff --git a/rsc/src/test/java/com/cloudera/livy/rsc/rpc/TestRpc.java b/rsc/src/test/java/com/cloudera/livy/rsc/rpc/TestRpc.java index f0ef575ce..48abe9436 100644 --- a/rsc/src/test/java/com/cloudera/livy/rsc/rpc/TestRpc.java +++ b/rsc/src/test/java/com/cloudera/livy/rsc/rpc/TestRpc.java @@ -18,6 +18,8 @@ package com.cloudera.livy.rsc.rpc; import java.io.Closeable; +import java.io.IOException; +import java.net.SocketException; import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.CountDownLatch; @@ -186,10 +188,38 @@ public void testEncryption() throws Exception { assertEquals(outbound.message, reply.message); } + @Test + public void testPortRange() throws Exception { + String portRange = "a~b"; + emptyConfig.set(LAUNCHER_PORT_RANGE, portRange); + try { + autoClose(new RpcServer(emptyConfig)); + } catch (Exception ee) { + assertTrue(ee instanceof NumberFormatException); + } + portRange = "11000"; + emptyConfig.set(LAUNCHER_PORT_RANGE, portRange); + try { + autoClose(new RpcServer(emptyConfig)); + } catch (Exception ee) { + assertTrue(ee instanceof ArrayIndexOutOfBoundsException); + } + portRange = "11000~11110"; + emptyConfig.set(LAUNCHER_PORT_RANGE, portRange); + String [] portRangeData = portRange.split("~"); + int startPort = Integer.parseInt(portRangeData[0]); + int endPort = Integer.parseInt(portRangeData[1]); + RpcServer server = autoClose(new RpcServer(emptyConfig)); + assertTrue(startPort <= server.getPort() && server.getPort() <= endPort); + } + private void transfer(Rpc serverRpc, Rpc clientRpc) { EmbeddedChannel client = (EmbeddedChannel) clientRpc.getChannel(); EmbeddedChannel server = (EmbeddedChannel) serverRpc.getChannel(); + server.runPendingTasks(); + client.runPendingTasks(); + int count = 0; while (!client.outboundMessages().isEmpty()) { server.writeInbound(client.readOutbound()); diff --git a/scala-api/pom.xml b/scala-api/pom.xml index b27b62d2e..91b2e9c7b 100644 --- a/scala-api/pom.xml +++ b/scala-api/pom.xml @@ -19,65 +19,65 @@ - 4.0.0 + 4.0.0 - - com.cloudera.livy - multi-scala-project-root - 0.3.0-SNAPSHOT - ../scala/pom.xml - + + com.cloudera.livy + multi-scala-project-root + 0.4.0-SNAPSHOT + ../scala/pom.xml + - livy-scala-api-parent - 0.3.0-SNAPSHOT - pom + livy-scala-api-parent + 0.4.0-SNAPSHOT + pom - - - com.cloudera.livy - livy-api - ${project.version} - provided - - - com.cloudera.livy - livy-rsc - ${project.version} - test - - - com.cloudera.livy - livy-core_${scala.binary.version} - ${project.version} - test - - - com.cloudera.livy - livy-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-core_${scala.binary.version} - provided - - - org.apache.spark - spark-sql_${scala.binary.version} - provided - - - org.apache.spark - spark-hive_${scala.binary.version} - provided - - - org.apache.spark - spark-streaming_${scala.binary.version} - provided - - + + + com.cloudera.livy + livy-api + ${project.version} + provided + + + com.cloudera.livy + livy-rsc + ${project.version} + test + + + com.cloudera.livy + livy-core_${scala.binary.version} + ${project.version} + test + + + com.cloudera.livy + livy-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-core_${scala.binary.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + provided + + diff --git a/scala-api/scala-2.10/pom.xml b/scala-api/scala-2.10/pom.xml index b7b91ebc7..5401405ba 100644 --- a/scala-api/scala-2.10/pom.xml +++ b/scala-api/scala-2.10/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy livy-scala-api_2.10 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-scala-api-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/scala-api/scala-2.11/pom.xml b/scala-api/scala-2.11/pom.xml index 9bedbc98d..a79c97291 100644 --- a/scala-api/scala-2.11/pom.xml +++ b/scala-api/scala-2.11/pom.xml @@ -20,13 +20,13 @@ 4.0.0 com.cloudera.livy livy-scala-api_2.11 - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar com.cloudera.livy livy-scala-api-parent - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml diff --git a/scala-api/src/main/resources/build.marker b/scala-api/src/main/resources/build.marker new file mode 100644 index 000000000..e69de29bb diff --git a/scala-api/src/main/scala/com/cloudera/livy/scalaapi/ScalaJobContext.scala b/scala-api/src/main/scala/com/cloudera/livy/scalaapi/ScalaJobContext.scala index 3a59e7123..7dd372ac8 100644 --- a/scala-api/src/main/scala/com/cloudera/livy/scalaapi/ScalaJobContext.scala +++ b/scala-api/src/main/scala/com/cloudera/livy/scalaapi/ScalaJobContext.scala @@ -46,6 +46,8 @@ class ScalaJobContext private[livy] (context: JobContext) { /** Returns the StreamingContext which has already been created. */ def streamingctx: StreamingContext = context.streamingctx().ssc + def sparkSession[E]: E = context.sparkSession() + /** * Creates the SparkStreaming context. * diff --git a/scala/pom.xml b/scala/pom.xml index ca71dcdfa..2d25d9dac 100644 --- a/scala/pom.xml +++ b/scala/pom.xml @@ -22,13 +22,13 @@ 4.0.0 com.cloudera.livy multi-scala-project-root - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT pom com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT ../pom.xml @@ -100,19 +100,6 @@ - - net.alchim31.maven - scala-maven-plugin - - - scala-compile-first - process-resources - - compile - - - - diff --git a/server/pom.xml b/server/pom.xml index b2fc858fc..3bca3fdf0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -19,237 +19,267 @@ - 4.0.0 - - com.cloudera.livy - livy-main - ../pom.xml - 0.3.0-SNAPSHOT - - - livy-server - 0.3.0-SNAPSHOT - jar - - - true - - - - - - ${project.groupId} - livy-core_${scala.binary.version} - ${project.version} - - - - ${project.groupId} - livy-core_${scala.binary.version} - ${project.version} - test-jar - test - - - - ${project.groupId} - livy-rsc - ${project.version} - - - - com.cloudera.livy - livy-test-lib - ${project.version} - test - - - - com.fasterxml.jackson.core - jackson-core - - - - com.fasterxml.jackson.core - jackson-databind - - - - com.fasterxml.jackson.module - jackson-module-scala_${scala.binary.version} - - - - io.dropwizard.metrics - metrics-core - - - - io.dropwizard.metrics - metrics-healthchecks - - - - javax.servlet - javax.servlet-api - - - - org.apache.hadoop - hadoop-auth - ${hadoop.scope} - - - - org.apache.hadoop - hadoop-client - - - javax.servlet - servlet-api - - - ${hadoop.scope} - - - - org.apache.httpcomponents - httpclient - test - - - - org.eclipse.jetty - jetty-server - - - - org.eclipse.jetty - jetty-servlet - - - - org.json4s - json4s-ast_${scala.binary.version} - - - - org.json4s - json4s-core_${scala.binary.version} - - - - org.json4s - json4s-jackson_${scala.binary.version} - - - - org.scala-lang - scala-library - - - - org.scalatra - scalatra_${scala.binary.version} - - - - org.scalatra - scalatra-json_${scala.binary.version} - - - - org.scalatra - scalatra-metrics_${scala.binary.version} - 2.4.0.M3 - - - com.typesafe.akka - akka-actor_${scala.binary.version} - - - - - - org.slf4j - slf4j-api - - - - com.cloudera.livy - livy-repl_${scala.binary.version} - ${project.version} - test - - - - org.apache.spark - spark-core_${scala.binary.version} - test - - - - org.apache.spark - spark-repl_${scala.binary.version} - test - - - - org.apache.spark - spark-sql_${scala.binary.version} - test - - - - org.apache.spark - spark-streaming_${scala.binary.version} - test - - - - org.apache.spark - spark-hive_${scala.binary.version} - test - - - - org.scalatra - scalatra-test_${scala.binary.version} - test - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - com.cloudera.livy.server.Main - - - ${project.build.directory}/jars - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - package - - copy-dependencies - - - - - - - + 4.0.0 + + com.cloudera.livy + livy-main + ../pom.xml + 0.4.0-SNAPSHOT + + + livy-server + 0.4.0-SNAPSHOT + jar + + + + + ${project.groupId} + livy-core_${scala.binary.version} + ${project.version} + + + + ${project.groupId} + livy-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + ${project.groupId} + livy-rsc + ${project.version} + + + + com.cloudera.livy + livy-test-lib + ${project.version} + test + + + + com.fasterxml.jackson.core + jackson-core + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + + + + io.dropwizard.metrics + metrics-core + + + + io.dropwizard.metrics + metrics-healthchecks + + + + javax.servlet + javax.servlet-api + + + + org.apache.hadoop + hadoop-auth + ${hadoop.scope} + + + + org.apache.hadoop + hadoop-client + + + javax.servlet + servlet-api + + + ${hadoop.scope} + + + + org.apache.httpcomponents + httpclient + test + + + + org.eclipse.jetty + jetty-server + + + + org.eclipse.jetty + jetty-servlet + + + + org.json4s + json4s-ast_${scala.binary.version} + + + + org.json4s + json4s-core_${scala.binary.version} + + + + org.json4s + json4s-jackson_${scala.binary.version} + + + + org.scala-lang + scala-library + + + + org.scalatra + scalatra_${scala.binary.version} + + + + org.scalatra + scalatra-json_${scala.binary.version} + + + + org.scalatra + scalatra-metrics_${scala.binary.version} + 2.4.0.M3 + + + com.typesafe.akka + akka-actor_${scala.binary.version} + + + + + + org.slf4j + slf4j-api + + + + com.cloudera.livy + livy-repl_${scala.binary.version} + ${project.version} + test + + + + org.apache.spark + spark-core_${scala.binary.version} + test + + + + org.apache.spark + spark-repl_${scala.binary.version} + test + + + + org.apache.spark + spark-sql_${scala.binary.version} + test + + + + org.apache.spark + spark-streaming_${scala.binary.version} + test + + + + org.apache.spark + spark-hive_${scala.binary.version} + test + + + + org.scalatra + scalatra-test_${scala.binary.version} + test + + + + + + + + ${project.basedir}/src/main/resources + + + + ${project.build.directory}/extra-resources + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + com.cloudera.livy.server.Main + + + ${project.build.directory}/jars + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + package + + copy-dependencies + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + generate-resources + + + + + + + + + + + + run + + + + + + + diff --git a/server/src/main/resources/com/cloudera/livy/server/ui/static/all-sessions.js b/server/src/main/resources/com/cloudera/livy/server/ui/static/all-sessions.js new file mode 100644 index 000000000..b0c690460 --- /dev/null +++ b/server/src/main/resources/com/cloudera/livy/server/ui/static/all-sessions.js @@ -0,0 +1,79 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +function tdWrap(str) { + return "" + str + ""; +} + +function loadSessionsTable(sessions) { + $.each(sessions, function(index, session) { + $("#interactive-sessions .sessions-table-body").append( + "" + + tdWrap(session.id) + + tdWrap(session.appId) + + tdWrap(session.owner) + + tdWrap(session.proxyUser) + + tdWrap(session.kind) + + tdWrap(session.state) + + "" + ); + }); +} + +function loadBatchesTable(sessions) { + $.each(sessions, function(index, session) { + $("#batches .sessions-table-body").append( + "" + + tdWrap(session.id) + + tdWrap(session.appId) + + tdWrap(session.state) + + "" + ); + }); +} + +var numSessions = 0; +var numBatches = 0; + +$(document).ready(function () { + var sessionsReq = $.getJSON(location.origin + "/sessions", function(response) { + if (response && response.total > 0) { + $("#interactive-sessions").load("/static/sessions-table.html .sessions-template", function() { + loadSessionsTable(response.sessions); + $('#interactive-sessions [data-toggle="tooltip"]').tooltip(); + }); + } + numSessions = response.total; + }); + + var batchesReq = $.getJSON(location.origin + "/batches", function(response) { + if (response && response.total > 0) { + $("#batches").load("/static/batches-table.html .sessions-template", function() { + loadBatchesTable(response.sessions); + $('#batches [data-toggle="tooltip"]').tooltip(); + }); + } + numBatches = response.total; + }); + + $.when(sessionsReq, batchesReq).done(function () { + if (numSessions + numBatches == 0) { + $("#all-sessions").append('

No Sessions or Batches have been created yet.

'); + } + }); +}); \ No newline at end of file diff --git a/server/src/main/resources/com/cloudera/livy/server/ui/static/batches-table.html b/server/src/main/resources/com/cloudera/livy/server/ui/static/batches-table.html new file mode 100644 index 000000000..45efbd655 --- /dev/null +++ b/server/src/main/resources/com/cloudera/livy/server/ui/static/batches-table.html @@ -0,0 +1,37 @@ + + +

Batch Sessions

+ + + + + + + + + + + +
Batch IdApplication Id + + State + +
\ No newline at end of file diff --git a/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.css b/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.css new file mode 100755 index 000000000..72cc3ab1b --- /dev/null +++ b/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.css @@ -0,0 +1,14 @@ +/*! + * Bootstrap v3.3.7 (http://getbootstrap.com) + * Copyright 2011-2017 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ + +/*! + * Generated using the Bootstrap Customizer (http://getbootstrap.com/customize/?id=794c6d582814e1084501e746ef56e328) + * Config saved to config.json and https://gist.github.com/794c6d582814e1084501e746ef56e328 + *//*! + * Bootstrap v3.3.7 (http://getbootstrap.com) + * Copyright 2011-2016 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + *//*! normalize.css v3.0.3 | MIT License | github.com/necolas/normalize.css */html{font-family:sans-serif;-ms-text-size-adjust:100%;-webkit-text-size-adjust:100%}body{margin:0}article,aside,details,figcaption,figure,footer,header,hgroup,main,menu,nav,section,summary{display:block}audio,canvas,progress,video{display:inline-block;vertical-align:baseline}audio:not([controls]){display:none;height:0}[hidden],template{display:none}a{background-color:transparent}a:active,a:hover{outline:0}abbr[title]{border-bottom:1px dotted}b,strong{font-weight:bold}dfn{font-style:italic}h1{font-size:2em;margin:0.67em 0}mark{background:#ff0;color:#000}small{font-size:80%}sub,sup{font-size:75%;line-height:0;position:relative;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{border:0}svg:not(:root){overflow:hidden}figure{margin:1em 40px}hr{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;height:0}pre{overflow:auto}code,kbd,pre,samp{font-family:monospace, monospace;font-size:1em}button,input,optgroup,select,textarea{color:inherit;font:inherit;margin:0}button{overflow:visible}button,select{text-transform:none}button,html input[type="button"],input[type="reset"],input[type="submit"]{-webkit-appearance:button;cursor:pointer}button[disabled],html input[disabled]{cursor:default}button::-moz-focus-inner,input::-moz-focus-inner{border:0;padding:0}input{line-height:normal}input[type="checkbox"],input[type="radio"]{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;padding:0}input[type="number"]::-webkit-inner-spin-button,input[type="number"]::-webkit-outer-spin-button{height:auto}input[type="search"]{-webkit-appearance:textfield;-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box}input[type="search"]::-webkit-search-cancel-button,input[type="search"]::-webkit-search-decoration{-webkit-appearance:none}fieldset{border:1px solid #c0c0c0;margin:0 2px;padding:0.35em 0.625em 0.75em}legend{border:0;padding:0}textarea{overflow:auto}optgroup{font-weight:bold}table{border-collapse:collapse;border-spacing:0}td,th{padding:0}/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */@media print{*,*:before,*:after{background:transparent !important;color:#000 !important;-webkit-box-shadow:none !important;box-shadow:none !important;text-shadow:none !important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}a[href^="#"]:after,a[href^="javascript:"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100% !important}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}.navbar{display:none}.btn>.caret,.dropup>.btn>.caret{border-top-color:#000 !important}.label{border:1px solid #000}.table{border-collapse:collapse !important}.table td,.table th{background-color:#fff !important}.table-bordered th,.table-bordered td{border:1px solid #ddd !important}}*{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}*:before,*:after{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}html{font-size:10px;-webkit-tap-highlight-color:rgba(0,0,0,0)}body{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:1.42857143;color:#333;background-color:#fff}input,button,select,textarea{font-family:inherit;font-size:inherit;line-height:inherit}a{color:#337ab7;text-decoration:none}a:hover,a:focus{color:#23527c;text-decoration:underline}a:focus{outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}figure{margin:0}img{vertical-align:middle}.img-responsive,.thumbnail>img,.thumbnail a>img,.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;max-width:100%;height:auto}.img-rounded{border-radius:6px}.img-thumbnail{padding:4px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out;display:inline-block;max-width:100%;height:auto}.img-circle{border-radius:50%}hr{margin-top:20px;margin-bottom:20px;border:0;border-top:1px solid #eee}.sr-only{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0, 0, 0, 0);border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;margin:0;overflow:visible;clip:auto}[role="button"]{cursor:pointer}h1,h2,h3,h4,h5,h6,.h1,.h2,.h3,.h4,.h5,.h6{font-family:inherit;font-weight:500;line-height:1.1;color:inherit}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small,.h1 small,.h2 small,.h3 small,.h4 small,.h5 small,.h6 small,h1 .small,h2 .small,h3 .small,h4 .small,h5 .small,h6 .small,.h1 .small,.h2 .small,.h3 .small,.h4 .small,.h5 .small,.h6 .small{font-weight:normal;line-height:1;color:#777}h1,.h1,h2,.h2,h3,.h3{margin-top:20px;margin-bottom:10px}h1 small,.h1 small,h2 small,.h2 small,h3 small,.h3 small,h1 .small,.h1 .small,h2 .small,.h2 .small,h3 .small,.h3 .small{font-size:65%}h4,.h4,h5,.h5,h6,.h6{margin-top:10px;margin-bottom:10px}h4 small,.h4 small,h5 small,.h5 small,h6 small,.h6 small,h4 .small,.h4 .small,h5 .small,.h5 .small,h6 .small,.h6 .small{font-size:75%}h1,.h1{font-size:36px}h2,.h2{font-size:30px}h3,.h3{font-size:24px}h4,.h4{font-size:18px}h5,.h5{font-size:14px}h6,.h6{font-size:12px}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:16px;font-weight:300;line-height:1.4}@media (min-width:768px){.lead{font-size:21px}}small,.small{font-size:85%}mark,.mark{background-color:#fcf8e3;padding:.2em}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}.text-justify{text-align:justify}.text-nowrap{white-space:nowrap}.text-lowercase{text-transform:lowercase}.text-uppercase{text-transform:uppercase}.text-capitalize{text-transform:capitalize}.text-muted{color:#777}.text-primary{color:#337ab7}a.text-primary:hover,a.text-primary:focus{color:#286090}.text-success{color:#3c763d}a.text-success:hover,a.text-success:focus{color:#2b542c}.text-info{color:#31708f}a.text-info:hover,a.text-info:focus{color:#245269}.text-warning{color:#8a6d3b}a.text-warning:hover,a.text-warning:focus{color:#66512c}.text-danger{color:#a94442}a.text-danger:hover,a.text-danger:focus{color:#843534}.bg-primary{color:#fff;background-color:#337ab7}a.bg-primary:hover,a.bg-primary:focus{background-color:#286090}.bg-success{background-color:#dff0d8}a.bg-success:hover,a.bg-success:focus{background-color:#c1e2b3}.bg-info{background-color:#d9edf7}a.bg-info:hover,a.bg-info:focus{background-color:#afd9ee}.bg-warning{background-color:#fcf8e3}a.bg-warning:hover,a.bg-warning:focus{background-color:#f7ecb5}.bg-danger{background-color:#f2dede}a.bg-danger:hover,a.bg-danger:focus{background-color:#e4b9b9}.page-header{padding-bottom:9px;margin:40px 0 20px;border-bottom:1px solid #eee}ul,ol{margin-top:0;margin-bottom:10px}ul ul,ol ul,ul ol,ol ol{margin-bottom:0}.list-unstyled{padding-left:0;list-style:none}.list-inline{padding-left:0;list-style:none;margin-left:-5px}.list-inline>li{display:inline-block;padding-left:5px;padding-right:5px}dl{margin-top:0;margin-bottom:20px}dt,dd{line-height:1.42857143}dt{font-weight:bold}dd{margin-left:0}@media (min-width:768px){.dl-horizontal dt{float:left;width:160px;clear:left;text-align:right;overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #777}.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:10px 20px;margin:0 0 20px;font-size:17.5px;border-left:5px solid #eee}blockquote p:last-child,blockquote ul:last-child,blockquote ol:last-child{margin-bottom:0}blockquote footer,blockquote small,blockquote .small{display:block;font-size:80%;line-height:1.42857143;color:#777}blockquote footer:before,blockquote small:before,blockquote .small:before{content:'\2014 \00A0'}.blockquote-reverse,blockquote.pull-right{padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0;text-align:right}.blockquote-reverse footer:before,blockquote.pull-right footer:before,.blockquote-reverse small:before,blockquote.pull-right small:before,.blockquote-reverse .small:before,blockquote.pull-right .small:before{content:''}.blockquote-reverse footer:after,blockquote.pull-right footer:after,.blockquote-reverse small:after,blockquote.pull-right small:after,.blockquote-reverse .small:after,blockquote.pull-right .small:after{content:'\00A0 \2014'}address{margin-bottom:20px;font-style:normal;line-height:1.42857143}code,kbd,pre,samp{font-family:Menlo,Monaco,Consolas,"Courier New",monospace}code{padding:2px 4px;font-size:90%;color:#c7254e;background-color:#f9f2f4;border-radius:4px}kbd{padding:2px 4px;font-size:90%;color:#fff;background-color:#333;border-radius:3px;-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.25);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.25)}kbd kbd{padding:0;font-size:100%;font-weight:bold;-webkit-box-shadow:none;box-shadow:none}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:1.42857143;word-break:break-all;word-wrap:break-word;color:#333;background-color:#f5f5f5;border:1px solid #ccc;border-radius:4px}pre code{padding:0;font-size:inherit;color:inherit;white-space:pre-wrap;background-color:transparent;border-radius:0}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}@media (min-width:768px){.container{width:750px}}@media (min-width:992px){.container{width:970px}}@media (min-width:1200px){.container{width:1170px}}.container-fluid{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}.row{margin-left:-15px;margin-right:-15px}.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12{position:relative;min-height:1px;padding-left:15px;padding-right:15px}.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12{float:left}.col-xs-12{width:100%}.col-xs-11{width:91.66666667%}.col-xs-10{width:83.33333333%}.col-xs-9{width:75%}.col-xs-8{width:66.66666667%}.col-xs-7{width:58.33333333%}.col-xs-6{width:50%}.col-xs-5{width:41.66666667%}.col-xs-4{width:33.33333333%}.col-xs-3{width:25%}.col-xs-2{width:16.66666667%}.col-xs-1{width:8.33333333%}.col-xs-pull-12{right:100%}.col-xs-pull-11{right:91.66666667%}.col-xs-pull-10{right:83.33333333%}.col-xs-pull-9{right:75%}.col-xs-pull-8{right:66.66666667%}.col-xs-pull-7{right:58.33333333%}.col-xs-pull-6{right:50%}.col-xs-pull-5{right:41.66666667%}.col-xs-pull-4{right:33.33333333%}.col-xs-pull-3{right:25%}.col-xs-pull-2{right:16.66666667%}.col-xs-pull-1{right:8.33333333%}.col-xs-pull-0{right:auto}.col-xs-push-12{left:100%}.col-xs-push-11{left:91.66666667%}.col-xs-push-10{left:83.33333333%}.col-xs-push-9{left:75%}.col-xs-push-8{left:66.66666667%}.col-xs-push-7{left:58.33333333%}.col-xs-push-6{left:50%}.col-xs-push-5{left:41.66666667%}.col-xs-push-4{left:33.33333333%}.col-xs-push-3{left:25%}.col-xs-push-2{left:16.66666667%}.col-xs-push-1{left:8.33333333%}.col-xs-push-0{left:auto}.col-xs-offset-12{margin-left:100%}.col-xs-offset-11{margin-left:91.66666667%}.col-xs-offset-10{margin-left:83.33333333%}.col-xs-offset-9{margin-left:75%}.col-xs-offset-8{margin-left:66.66666667%}.col-xs-offset-7{margin-left:58.33333333%}.col-xs-offset-6{margin-left:50%}.col-xs-offset-5{margin-left:41.66666667%}.col-xs-offset-4{margin-left:33.33333333%}.col-xs-offset-3{margin-left:25%}.col-xs-offset-2{margin-left:16.66666667%}.col-xs-offset-1{margin-left:8.33333333%}.col-xs-offset-0{margin-left:0}@media (min-width:768px){.col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12{float:left}.col-sm-12{width:100%}.col-sm-11{width:91.66666667%}.col-sm-10{width:83.33333333%}.col-sm-9{width:75%}.col-sm-8{width:66.66666667%}.col-sm-7{width:58.33333333%}.col-sm-6{width:50%}.col-sm-5{width:41.66666667%}.col-sm-4{width:33.33333333%}.col-sm-3{width:25%}.col-sm-2{width:16.66666667%}.col-sm-1{width:8.33333333%}.col-sm-pull-12{right:100%}.col-sm-pull-11{right:91.66666667%}.col-sm-pull-10{right:83.33333333%}.col-sm-pull-9{right:75%}.col-sm-pull-8{right:66.66666667%}.col-sm-pull-7{right:58.33333333%}.col-sm-pull-6{right:50%}.col-sm-pull-5{right:41.66666667%}.col-sm-pull-4{right:33.33333333%}.col-sm-pull-3{right:25%}.col-sm-pull-2{right:16.66666667%}.col-sm-pull-1{right:8.33333333%}.col-sm-pull-0{right:auto}.col-sm-push-12{left:100%}.col-sm-push-11{left:91.66666667%}.col-sm-push-10{left:83.33333333%}.col-sm-push-9{left:75%}.col-sm-push-8{left:66.66666667%}.col-sm-push-7{left:58.33333333%}.col-sm-push-6{left:50%}.col-sm-push-5{left:41.66666667%}.col-sm-push-4{left:33.33333333%}.col-sm-push-3{left:25%}.col-sm-push-2{left:16.66666667%}.col-sm-push-1{left:8.33333333%}.col-sm-push-0{left:auto}.col-sm-offset-12{margin-left:100%}.col-sm-offset-11{margin-left:91.66666667%}.col-sm-offset-10{margin-left:83.33333333%}.col-sm-offset-9{margin-left:75%}.col-sm-offset-8{margin-left:66.66666667%}.col-sm-offset-7{margin-left:58.33333333%}.col-sm-offset-6{margin-left:50%}.col-sm-offset-5{margin-left:41.66666667%}.col-sm-offset-4{margin-left:33.33333333%}.col-sm-offset-3{margin-left:25%}.col-sm-offset-2{margin-left:16.66666667%}.col-sm-offset-1{margin-left:8.33333333%}.col-sm-offset-0{margin-left:0}}@media (min-width:992px){.col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12{float:left}.col-md-12{width:100%}.col-md-11{width:91.66666667%}.col-md-10{width:83.33333333%}.col-md-9{width:75%}.col-md-8{width:66.66666667%}.col-md-7{width:58.33333333%}.col-md-6{width:50%}.col-md-5{width:41.66666667%}.col-md-4{width:33.33333333%}.col-md-3{width:25%}.col-md-2{width:16.66666667%}.col-md-1{width:8.33333333%}.col-md-pull-12{right:100%}.col-md-pull-11{right:91.66666667%}.col-md-pull-10{right:83.33333333%}.col-md-pull-9{right:75%}.col-md-pull-8{right:66.66666667%}.col-md-pull-7{right:58.33333333%}.col-md-pull-6{right:50%}.col-md-pull-5{right:41.66666667%}.col-md-pull-4{right:33.33333333%}.col-md-pull-3{right:25%}.col-md-pull-2{right:16.66666667%}.col-md-pull-1{right:8.33333333%}.col-md-pull-0{right:auto}.col-md-push-12{left:100%}.col-md-push-11{left:91.66666667%}.col-md-push-10{left:83.33333333%}.col-md-push-9{left:75%}.col-md-push-8{left:66.66666667%}.col-md-push-7{left:58.33333333%}.col-md-push-6{left:50%}.col-md-push-5{left:41.66666667%}.col-md-push-4{left:33.33333333%}.col-md-push-3{left:25%}.col-md-push-2{left:16.66666667%}.col-md-push-1{left:8.33333333%}.col-md-push-0{left:auto}.col-md-offset-12{margin-left:100%}.col-md-offset-11{margin-left:91.66666667%}.col-md-offset-10{margin-left:83.33333333%}.col-md-offset-9{margin-left:75%}.col-md-offset-8{margin-left:66.66666667%}.col-md-offset-7{margin-left:58.33333333%}.col-md-offset-6{margin-left:50%}.col-md-offset-5{margin-left:41.66666667%}.col-md-offset-4{margin-left:33.33333333%}.col-md-offset-3{margin-left:25%}.col-md-offset-2{margin-left:16.66666667%}.col-md-offset-1{margin-left:8.33333333%}.col-md-offset-0{margin-left:0}}@media (min-width:1200px){.col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12{float:left}.col-lg-12{width:100%}.col-lg-11{width:91.66666667%}.col-lg-10{width:83.33333333%}.col-lg-9{width:75%}.col-lg-8{width:66.66666667%}.col-lg-7{width:58.33333333%}.col-lg-6{width:50%}.col-lg-5{width:41.66666667%}.col-lg-4{width:33.33333333%}.col-lg-3{width:25%}.col-lg-2{width:16.66666667%}.col-lg-1{width:8.33333333%}.col-lg-pull-12{right:100%}.col-lg-pull-11{right:91.66666667%}.col-lg-pull-10{right:83.33333333%}.col-lg-pull-9{right:75%}.col-lg-pull-8{right:66.66666667%}.col-lg-pull-7{right:58.33333333%}.col-lg-pull-6{right:50%}.col-lg-pull-5{right:41.66666667%}.col-lg-pull-4{right:33.33333333%}.col-lg-pull-3{right:25%}.col-lg-pull-2{right:16.66666667%}.col-lg-pull-1{right:8.33333333%}.col-lg-pull-0{right:auto}.col-lg-push-12{left:100%}.col-lg-push-11{left:91.66666667%}.col-lg-push-10{left:83.33333333%}.col-lg-push-9{left:75%}.col-lg-push-8{left:66.66666667%}.col-lg-push-7{left:58.33333333%}.col-lg-push-6{left:50%}.col-lg-push-5{left:41.66666667%}.col-lg-push-4{left:33.33333333%}.col-lg-push-3{left:25%}.col-lg-push-2{left:16.66666667%}.col-lg-push-1{left:8.33333333%}.col-lg-push-0{left:auto}.col-lg-offset-12{margin-left:100%}.col-lg-offset-11{margin-left:91.66666667%}.col-lg-offset-10{margin-left:83.33333333%}.col-lg-offset-9{margin-left:75%}.col-lg-offset-8{margin-left:66.66666667%}.col-lg-offset-7{margin-left:58.33333333%}.col-lg-offset-6{margin-left:50%}.col-lg-offset-5{margin-left:41.66666667%}.col-lg-offset-4{margin-left:33.33333333%}.col-lg-offset-3{margin-left:25%}.col-lg-offset-2{margin-left:16.66666667%}.col-lg-offset-1{margin-left:8.33333333%}.col-lg-offset-0{margin-left:0}}table{background-color:transparent}caption{padding-top:8px;padding-bottom:8px;color:#777;text-align:left}th{text-align:left}.table{width:100%;max-width:100%;margin-bottom:20px}.table>thead>tr>th,.table>tbody>tr>th,.table>tfoot>tr>th,.table>thead>tr>td,.table>tbody>tr>td,.table>tfoot>tr>td{padding:8px;line-height:1.42857143;vertical-align:top;border-top:1px solid #ddd}.table>thead>tr>th{vertical-align:bottom;border-bottom:2px solid #ddd}.table>caption+thead>tr:first-child>th,.table>colgroup+thead>tr:first-child>th,.table>thead:first-child>tr:first-child>th,.table>caption+thead>tr:first-child>td,.table>colgroup+thead>tr:first-child>td,.table>thead:first-child>tr:first-child>td{border-top:0}.table>tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.table-condensed>thead>tr>th,.table-condensed>tbody>tr>th,.table-condensed>tfoot>tr>th,.table-condensed>thead>tr>td,.table-condensed>tbody>tr>td,.table-condensed>tfoot>tr>td{padding:5px}.table-bordered{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>tbody>tr>th,.table-bordered>tfoot>tr>th,.table-bordered>thead>tr>td,.table-bordered>tbody>tr>td,.table-bordered>tfoot>tr>td{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>thead>tr>td{border-bottom-width:2px}.table-striped>tbody>tr:nth-of-type(odd){background-color:#f9f9f9}.table-hover>tbody>tr:hover{background-color:#f5f5f5}table col[class*="col-"]{position:static;float:none;display:table-column}table td[class*="col-"],table th[class*="col-"]{position:static;float:none;display:table-cell}.table>thead>tr>td.active,.table>tbody>tr>td.active,.table>tfoot>tr>td.active,.table>thead>tr>th.active,.table>tbody>tr>th.active,.table>tfoot>tr>th.active,.table>thead>tr.active>td,.table>tbody>tr.active>td,.table>tfoot>tr.active>td,.table>thead>tr.active>th,.table>tbody>tr.active>th,.table>tfoot>tr.active>th{background-color:#f5f5f5}.table-hover>tbody>tr>td.active:hover,.table-hover>tbody>tr>th.active:hover,.table-hover>tbody>tr.active:hover>td,.table-hover>tbody>tr:hover>.active,.table-hover>tbody>tr.active:hover>th{background-color:#e8e8e8}.table>thead>tr>td.success,.table>tbody>tr>td.success,.table>tfoot>tr>td.success,.table>thead>tr>th.success,.table>tbody>tr>th.success,.table>tfoot>tr>th.success,.table>thead>tr.success>td,.table>tbody>tr.success>td,.table>tfoot>tr.success>td,.table>thead>tr.success>th,.table>tbody>tr.success>th,.table>tfoot>tr.success>th{background-color:#dff0d8}.table-hover>tbody>tr>td.success:hover,.table-hover>tbody>tr>th.success:hover,.table-hover>tbody>tr.success:hover>td,.table-hover>tbody>tr:hover>.success,.table-hover>tbody>tr.success:hover>th{background-color:#d0e9c6}.table>thead>tr>td.info,.table>tbody>tr>td.info,.table>tfoot>tr>td.info,.table>thead>tr>th.info,.table>tbody>tr>th.info,.table>tfoot>tr>th.info,.table>thead>tr.info>td,.table>tbody>tr.info>td,.table>tfoot>tr.info>td,.table>thead>tr.info>th,.table>tbody>tr.info>th,.table>tfoot>tr.info>th{background-color:#d9edf7}.table-hover>tbody>tr>td.info:hover,.table-hover>tbody>tr>th.info:hover,.table-hover>tbody>tr.info:hover>td,.table-hover>tbody>tr:hover>.info,.table-hover>tbody>tr.info:hover>th{background-color:#c4e3f3}.table>thead>tr>td.warning,.table>tbody>tr>td.warning,.table>tfoot>tr>td.warning,.table>thead>tr>th.warning,.table>tbody>tr>th.warning,.table>tfoot>tr>th.warning,.table>thead>tr.warning>td,.table>tbody>tr.warning>td,.table>tfoot>tr.warning>td,.table>thead>tr.warning>th,.table>tbody>tr.warning>th,.table>tfoot>tr.warning>th{background-color:#fcf8e3}.table-hover>tbody>tr>td.warning:hover,.table-hover>tbody>tr>th.warning:hover,.table-hover>tbody>tr.warning:hover>td,.table-hover>tbody>tr:hover>.warning,.table-hover>tbody>tr.warning:hover>th{background-color:#faf2cc}.table>thead>tr>td.danger,.table>tbody>tr>td.danger,.table>tfoot>tr>td.danger,.table>thead>tr>th.danger,.table>tbody>tr>th.danger,.table>tfoot>tr>th.danger,.table>thead>tr.danger>td,.table>tbody>tr.danger>td,.table>tfoot>tr.danger>td,.table>thead>tr.danger>th,.table>tbody>tr.danger>th,.table>tfoot>tr.danger>th{background-color:#f2dede}.table-hover>tbody>tr>td.danger:hover,.table-hover>tbody>tr>th.danger:hover,.table-hover>tbody>tr.danger:hover>td,.table-hover>tbody>tr:hover>.danger,.table-hover>tbody>tr.danger:hover>th{background-color:#ebcccc}.table-responsive{overflow-x:auto;min-height:0.01%}@media screen and (max-width:767px){.table-responsive{width:100%;margin-bottom:15px;overflow-y:hidden;-ms-overflow-style:-ms-autohiding-scrollbar;border:1px solid #ddd}.table-responsive>.table{margin-bottom:0}.table-responsive>.table>thead>tr>th,.table-responsive>.table>tbody>tr>th,.table-responsive>.table>tfoot>tr>th,.table-responsive>.table>thead>tr>td,.table-responsive>.table>tbody>tr>td,.table-responsive>.table>tfoot>tr>td{white-space:nowrap}.table-responsive>.table-bordered{border:0}.table-responsive>.table-bordered>thead>tr>th:first-child,.table-responsive>.table-bordered>tbody>tr>th:first-child,.table-responsive>.table-bordered>tfoot>tr>th:first-child,.table-responsive>.table-bordered>thead>tr>td:first-child,.table-responsive>.table-bordered>tbody>tr>td:first-child,.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.table-responsive>.table-bordered>thead>tr>th:last-child,.table-responsive>.table-bordered>tbody>tr>th:last-child,.table-responsive>.table-bordered>tfoot>tr>th:last-child,.table-responsive>.table-bordered>thead>tr>td:last-child,.table-responsive>.table-bordered>tbody>tr>td:last-child,.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.table-responsive>.table-bordered>tbody>tr:last-child>th,.table-responsive>.table-bordered>tfoot>tr:last-child>th,.table-responsive>.table-bordered>tbody>tr:last-child>td,.table-responsive>.table-bordered>tfoot>tr:last-child>td{border-bottom:0}}fieldset{padding:0;margin:0;border:0;min-width:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:inherit;color:#333;border:0;border-bottom:1px solid #e5e5e5}label{display:inline-block;max-width:100%;margin-bottom:5px;font-weight:bold}input[type="search"]{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;line-height:normal}input[type="file"]{display:block}input[type="range"]{display:block;width:100%}select[multiple],select[size]{height:auto}input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}output{display:block;padding-top:7px;font-size:14px;line-height:1.42857143;color:#555}.form-control{display:block;width:100%;height:34px;padding:6px 12px;font-size:14px;line-height:1.42857143;color:#555;background-color:#fff;background-image:none;border:1px solid #ccc;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border-color ease-in-out .15s, -webkit-box-shadow ease-in-out .15s;-o-transition:border-color ease-in-out .15s, box-shadow ease-in-out .15s;transition:border-color ease-in-out .15s, box-shadow ease-in-out .15s}.form-control:focus{border-color:#66afe9;outline:0;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6)}.form-control::-moz-placeholder{color:#999;opacity:1}.form-control:-ms-input-placeholder{color:#999}.form-control::-webkit-input-placeholder{color:#999}.form-control::-ms-expand{border:0;background-color:transparent}.form-control[disabled],.form-control[readonly],fieldset[disabled] .form-control{background-color:#eee;opacity:1}.form-control[disabled],fieldset[disabled] .form-control{cursor:not-allowed}textarea.form-control{height:auto}input[type="search"]{-webkit-appearance:none}@media screen and (-webkit-min-device-pixel-ratio:0){input[type="date"].form-control,input[type="time"].form-control,input[type="datetime-local"].form-control,input[type="month"].form-control{line-height:34px}input[type="date"].input-sm,input[type="time"].input-sm,input[type="datetime-local"].input-sm,input[type="month"].input-sm,.input-group-sm input[type="date"],.input-group-sm input[type="time"],.input-group-sm input[type="datetime-local"],.input-group-sm input[type="month"]{line-height:30px}input[type="date"].input-lg,input[type="time"].input-lg,input[type="datetime-local"].input-lg,input[type="month"].input-lg,.input-group-lg input[type="date"],.input-group-lg input[type="time"],.input-group-lg input[type="datetime-local"],.input-group-lg input[type="month"]{line-height:46px}}.form-group{margin-bottom:15px}.radio,.checkbox{position:relative;display:block;margin-top:10px;margin-bottom:10px}.radio label,.checkbox label{min-height:20px;padding-left:20px;margin-bottom:0;font-weight:normal;cursor:pointer}.radio input[type="radio"],.radio-inline input[type="radio"],.checkbox input[type="checkbox"],.checkbox-inline input[type="checkbox"]{position:absolute;margin-left:-20px;margin-top:4px \9}.radio+.radio,.checkbox+.checkbox{margin-top:-5px}.radio-inline,.checkbox-inline{position:relative;display:inline-block;padding-left:20px;margin-bottom:0;vertical-align:middle;font-weight:normal;cursor:pointer}.radio-inline+.radio-inline,.checkbox-inline+.checkbox-inline{margin-top:0;margin-left:10px}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"].disabled,input[type="checkbox"].disabled,fieldset[disabled] input[type="radio"],fieldset[disabled] input[type="checkbox"]{cursor:not-allowed}.radio-inline.disabled,.checkbox-inline.disabled,fieldset[disabled] .radio-inline,fieldset[disabled] .checkbox-inline{cursor:not-allowed}.radio.disabled label,.checkbox.disabled label,fieldset[disabled] .radio label,fieldset[disabled] .checkbox label{cursor:not-allowed}.form-control-static{padding-top:7px;padding-bottom:7px;margin-bottom:0;min-height:34px}.form-control-static.input-lg,.form-control-static.input-sm{padding-left:0;padding-right:0}.input-sm{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-sm{height:30px;line-height:30px}textarea.input-sm,select[multiple].input-sm{height:auto}.form-group-sm .form-control{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.form-group-sm select.form-control{height:30px;line-height:30px}.form-group-sm textarea.form-control,.form-group-sm select[multiple].form-control{height:auto}.form-group-sm .form-control-static{height:30px;min-height:32px;padding:6px 10px;font-size:12px;line-height:1.5}.input-lg{height:46px;padding:10px 16px;font-size:18px;line-height:1.3333333;border-radius:6px}select.input-lg{height:46px;line-height:46px}textarea.input-lg,select[multiple].input-lg{height:auto}.form-group-lg .form-control{height:46px;padding:10px 16px;font-size:18px;line-height:1.3333333;border-radius:6px}.form-group-lg select.form-control{height:46px;line-height:46px}.form-group-lg textarea.form-control,.form-group-lg select[multiple].form-control{height:auto}.form-group-lg .form-control-static{height:46px;min-height:38px;padding:11px 16px;font-size:18px;line-height:1.3333333}.has-feedback{position:relative}.has-feedback .form-control{padding-right:42.5px}.form-control-feedback{position:absolute;top:0;right:0;z-index:2;display:block;width:34px;height:34px;line-height:34px;text-align:center;pointer-events:none}.input-lg+.form-control-feedback,.input-group-lg+.form-control-feedback,.form-group-lg .form-control+.form-control-feedback{width:46px;height:46px;line-height:46px}.input-sm+.form-control-feedback,.input-group-sm+.form-control-feedback,.form-group-sm .form-control+.form-control-feedback{width:30px;height:30px;line-height:30px}.has-success .help-block,.has-success .control-label,.has-success .radio,.has-success .checkbox,.has-success .radio-inline,.has-success .checkbox-inline,.has-success.radio label,.has-success.checkbox label,.has-success.radio-inline label,.has-success.checkbox-inline label{color:#3c763d}.has-success .form-control{border-color:#3c763d;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-success .form-control:focus{border-color:#2b542c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #67b168;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #67b168}.has-success .input-group-addon{color:#3c763d;border-color:#3c763d;background-color:#dff0d8}.has-success .form-control-feedback{color:#3c763d}.has-warning .help-block,.has-warning .control-label,.has-warning .radio,.has-warning .checkbox,.has-warning .radio-inline,.has-warning .checkbox-inline,.has-warning.radio label,.has-warning.checkbox label,.has-warning.radio-inline label,.has-warning.checkbox-inline label{color:#8a6d3b}.has-warning .form-control{border-color:#8a6d3b;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-warning .form-control:focus{border-color:#66512c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #c0a16b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #c0a16b}.has-warning .input-group-addon{color:#8a6d3b;border-color:#8a6d3b;background-color:#fcf8e3}.has-warning .form-control-feedback{color:#8a6d3b}.has-error .help-block,.has-error .control-label,.has-error .radio,.has-error .checkbox,.has-error .radio-inline,.has-error .checkbox-inline,.has-error.radio label,.has-error.checkbox label,.has-error.radio-inline label,.has-error.checkbox-inline label{color:#a94442}.has-error .form-control{border-color:#a94442;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-error .form-control:focus{border-color:#843534;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #ce8483;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #ce8483}.has-error .input-group-addon{color:#a94442;border-color:#a94442;background-color:#f2dede}.has-error .form-control-feedback{color:#a94442}.has-feedback label~.form-control-feedback{top:25px}.has-feedback label.sr-only~.form-control-feedback{top:0}.help-block{display:block;margin-top:5px;margin-bottom:10px;color:#737373}@media (min-width:768px){.form-inline .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .form-control-static{display:inline-block}.form-inline .input-group{display:inline-table;vertical-align:middle}.form-inline .input-group .input-group-addon,.form-inline .input-group .input-group-btn,.form-inline .input-group .form-control{width:auto}.form-inline .input-group>.form-control{width:100%}.form-inline .control-label{margin-bottom:0;vertical-align:middle}.form-inline .radio,.form-inline .checkbox{display:inline-block;margin-top:0;margin-bottom:0;vertical-align:middle}.form-inline .radio label,.form-inline .checkbox label{padding-left:0}.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{position:relative;margin-left:0}.form-inline .has-feedback .form-control-feedback{top:0}}.form-horizontal .radio,.form-horizontal .checkbox,.form-horizontal .radio-inline,.form-horizontal .checkbox-inline{margin-top:0;margin-bottom:0;padding-top:7px}.form-horizontal .radio,.form-horizontal .checkbox{min-height:27px}.form-horizontal .form-group{margin-left:-15px;margin-right:-15px}@media (min-width:768px){.form-horizontal .control-label{text-align:right;margin-bottom:0;padding-top:7px}}.form-horizontal .has-feedback .form-control-feedback{right:15px}@media (min-width:768px){.form-horizontal .form-group-lg .control-label{padding-top:11px;font-size:18px}}@media (min-width:768px){.form-horizontal .form-group-sm .control-label{padding-top:6px;font-size:12px}}.btn{display:inline-block;margin-bottom:0;font-weight:normal;text-align:center;vertical-align:middle;-ms-touch-action:manipulation;touch-action:manipulation;cursor:pointer;background-image:none;border:1px solid transparent;white-space:nowrap;padding:6px 12px;font-size:14px;line-height:1.42857143;border-radius:4px;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}.btn:focus,.btn:active:focus,.btn.active:focus,.btn.focus,.btn:active.focus,.btn.active.focus{outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn:hover,.btn:focus,.btn.focus{color:#333;text-decoration:none}.btn:active,.btn.active{outline:0;background-image:none;-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,0.125);box-shadow:inset 0 3px 5px rgba(0,0,0,0.125)}.btn.disabled,.btn[disabled],fieldset[disabled] .btn{cursor:not-allowed;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;box-shadow:none}a.btn.disabled,fieldset[disabled] a.btn{pointer-events:none}.btn-default{color:#333;background-color:#fff;border-color:#ccc}.btn-default:focus,.btn-default.focus{color:#333;background-color:#e6e6e6;border-color:#8c8c8c}.btn-default:hover{color:#333;background-color:#e6e6e6;border-color:#adadad}.btn-default:active,.btn-default.active,.open>.dropdown-toggle.btn-default{color:#333;background-color:#e6e6e6;border-color:#adadad}.btn-default:active:hover,.btn-default.active:hover,.open>.dropdown-toggle.btn-default:hover,.btn-default:active:focus,.btn-default.active:focus,.open>.dropdown-toggle.btn-default:focus,.btn-default:active.focus,.btn-default.active.focus,.open>.dropdown-toggle.btn-default.focus{color:#333;background-color:#d4d4d4;border-color:#8c8c8c}.btn-default:active,.btn-default.active,.open>.dropdown-toggle.btn-default{background-image:none}.btn-default.disabled:hover,.btn-default[disabled]:hover,fieldset[disabled] .btn-default:hover,.btn-default.disabled:focus,.btn-default[disabled]:focus,fieldset[disabled] .btn-default:focus,.btn-default.disabled.focus,.btn-default[disabled].focus,fieldset[disabled] .btn-default.focus{background-color:#fff;border-color:#ccc}.btn-default .badge{color:#fff;background-color:#333}.btn-primary{color:#fff;background-color:#337ab7;border-color:#2e6da4}.btn-primary:focus,.btn-primary.focus{color:#fff;background-color:#286090;border-color:#122b40}.btn-primary:hover{color:#fff;background-color:#286090;border-color:#204d74}.btn-primary:active,.btn-primary.active,.open>.dropdown-toggle.btn-primary{color:#fff;background-color:#286090;border-color:#204d74}.btn-primary:active:hover,.btn-primary.active:hover,.open>.dropdown-toggle.btn-primary:hover,.btn-primary:active:focus,.btn-primary.active:focus,.open>.dropdown-toggle.btn-primary:focus,.btn-primary:active.focus,.btn-primary.active.focus,.open>.dropdown-toggle.btn-primary.focus{color:#fff;background-color:#204d74;border-color:#122b40}.btn-primary:active,.btn-primary.active,.open>.dropdown-toggle.btn-primary{background-image:none}.btn-primary.disabled:hover,.btn-primary[disabled]:hover,fieldset[disabled] .btn-primary:hover,.btn-primary.disabled:focus,.btn-primary[disabled]:focus,fieldset[disabled] .btn-primary:focus,.btn-primary.disabled.focus,.btn-primary[disabled].focus,fieldset[disabled] .btn-primary.focus{background-color:#337ab7;border-color:#2e6da4}.btn-primary .badge{color:#337ab7;background-color:#fff}.btn-success{color:#fff;background-color:#5cb85c;border-color:#4cae4c}.btn-success:focus,.btn-success.focus{color:#fff;background-color:#449d44;border-color:#255625}.btn-success:hover{color:#fff;background-color:#449d44;border-color:#398439}.btn-success:active,.btn-success.active,.open>.dropdown-toggle.btn-success{color:#fff;background-color:#449d44;border-color:#398439}.btn-success:active:hover,.btn-success.active:hover,.open>.dropdown-toggle.btn-success:hover,.btn-success:active:focus,.btn-success.active:focus,.open>.dropdown-toggle.btn-success:focus,.btn-success:active.focus,.btn-success.active.focus,.open>.dropdown-toggle.btn-success.focus{color:#fff;background-color:#398439;border-color:#255625}.btn-success:active,.btn-success.active,.open>.dropdown-toggle.btn-success{background-image:none}.btn-success.disabled:hover,.btn-success[disabled]:hover,fieldset[disabled] .btn-success:hover,.btn-success.disabled:focus,.btn-success[disabled]:focus,fieldset[disabled] .btn-success:focus,.btn-success.disabled.focus,.btn-success[disabled].focus,fieldset[disabled] .btn-success.focus{background-color:#5cb85c;border-color:#4cae4c}.btn-success .badge{color:#5cb85c;background-color:#fff}.btn-info{color:#fff;background-color:#5bc0de;border-color:#46b8da}.btn-info:focus,.btn-info.focus{color:#fff;background-color:#31b0d5;border-color:#1b6d85}.btn-info:hover{color:#fff;background-color:#31b0d5;border-color:#269abc}.btn-info:active,.btn-info.active,.open>.dropdown-toggle.btn-info{color:#fff;background-color:#31b0d5;border-color:#269abc}.btn-info:active:hover,.btn-info.active:hover,.open>.dropdown-toggle.btn-info:hover,.btn-info:active:focus,.btn-info.active:focus,.open>.dropdown-toggle.btn-info:focus,.btn-info:active.focus,.btn-info.active.focus,.open>.dropdown-toggle.btn-info.focus{color:#fff;background-color:#269abc;border-color:#1b6d85}.btn-info:active,.btn-info.active,.open>.dropdown-toggle.btn-info{background-image:none}.btn-info.disabled:hover,.btn-info[disabled]:hover,fieldset[disabled] .btn-info:hover,.btn-info.disabled:focus,.btn-info[disabled]:focus,fieldset[disabled] .btn-info:focus,.btn-info.disabled.focus,.btn-info[disabled].focus,fieldset[disabled] .btn-info.focus{background-color:#5bc0de;border-color:#46b8da}.btn-info .badge{color:#5bc0de;background-color:#fff}.btn-warning{color:#fff;background-color:#f0ad4e;border-color:#eea236}.btn-warning:focus,.btn-warning.focus{color:#fff;background-color:#ec971f;border-color:#985f0d}.btn-warning:hover{color:#fff;background-color:#ec971f;border-color:#d58512}.btn-warning:active,.btn-warning.active,.open>.dropdown-toggle.btn-warning{color:#fff;background-color:#ec971f;border-color:#d58512}.btn-warning:active:hover,.btn-warning.active:hover,.open>.dropdown-toggle.btn-warning:hover,.btn-warning:active:focus,.btn-warning.active:focus,.open>.dropdown-toggle.btn-warning:focus,.btn-warning:active.focus,.btn-warning.active.focus,.open>.dropdown-toggle.btn-warning.focus{color:#fff;background-color:#d58512;border-color:#985f0d}.btn-warning:active,.btn-warning.active,.open>.dropdown-toggle.btn-warning{background-image:none}.btn-warning.disabled:hover,.btn-warning[disabled]:hover,fieldset[disabled] .btn-warning:hover,.btn-warning.disabled:focus,.btn-warning[disabled]:focus,fieldset[disabled] .btn-warning:focus,.btn-warning.disabled.focus,.btn-warning[disabled].focus,fieldset[disabled] .btn-warning.focus{background-color:#f0ad4e;border-color:#eea236}.btn-warning .badge{color:#f0ad4e;background-color:#fff}.btn-danger{color:#fff;background-color:#d9534f;border-color:#d43f3a}.btn-danger:focus,.btn-danger.focus{color:#fff;background-color:#c9302c;border-color:#761c19}.btn-danger:hover{color:#fff;background-color:#c9302c;border-color:#ac2925}.btn-danger:active,.btn-danger.active,.open>.dropdown-toggle.btn-danger{color:#fff;background-color:#c9302c;border-color:#ac2925}.btn-danger:active:hover,.btn-danger.active:hover,.open>.dropdown-toggle.btn-danger:hover,.btn-danger:active:focus,.btn-danger.active:focus,.open>.dropdown-toggle.btn-danger:focus,.btn-danger:active.focus,.btn-danger.active.focus,.open>.dropdown-toggle.btn-danger.focus{color:#fff;background-color:#ac2925;border-color:#761c19}.btn-danger:active,.btn-danger.active,.open>.dropdown-toggle.btn-danger{background-image:none}.btn-danger.disabled:hover,.btn-danger[disabled]:hover,fieldset[disabled] .btn-danger:hover,.btn-danger.disabled:focus,.btn-danger[disabled]:focus,fieldset[disabled] .btn-danger:focus,.btn-danger.disabled.focus,.btn-danger[disabled].focus,fieldset[disabled] .btn-danger.focus{background-color:#d9534f;border-color:#d43f3a}.btn-danger .badge{color:#d9534f;background-color:#fff}.btn-link{color:#337ab7;font-weight:normal;border-radius:0}.btn-link,.btn-link:active,.btn-link.active,.btn-link[disabled],fieldset[disabled] .btn-link{background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.btn-link,.btn-link:hover,.btn-link:focus,.btn-link:active{border-color:transparent}.btn-link:hover,.btn-link:focus{color:#23527c;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,fieldset[disabled] .btn-link:hover,.btn-link[disabled]:focus,fieldset[disabled] .btn-link:focus{color:#777;text-decoration:none}.btn-lg,.btn-group-lg>.btn{padding:10px 16px;font-size:18px;line-height:1.3333333;border-radius:6px}.btn-sm,.btn-group-sm>.btn{padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.btn-xs,.btn-group-xs>.btn{padding:1px 5px;font-size:12px;line-height:1.5;border-radius:3px}.btn-block{display:block;width:100%}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.fade{opacity:0;-webkit-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{display:none}.collapse.in{display:block}tr.collapse.in{display:table-row}tbody.collapse.in{display:table-row-group}.collapsing{position:relative;height:0;overflow:hidden;-webkit-transition-property:height, visibility;-o-transition-property:height, visibility;transition-property:height, visibility;-webkit-transition-duration:.35s;-o-transition-duration:.35s;transition-duration:.35s;-webkit-transition-timing-function:ease;-o-transition-timing-function:ease;transition-timing-function:ease}.caret{display:inline-block;width:0;height:0;margin-left:2px;vertical-align:middle;border-top:4px dashed;border-top:4px solid \9;border-right:4px solid transparent;border-left:4px solid transparent}.dropup,.dropdown{position:relative}.dropdown-toggle:focus{outline:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;font-size:14px;text-align:left;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);border-radius:4px;-webkit-box-shadow:0 6px 12px rgba(0,0,0,0.175);box-shadow:0 6px 12px rgba(0,0,0,0.175);-webkit-background-clip:padding-box;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:1.42857143;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus{text-decoration:none;color:#262626;background-color:#f5f5f5}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;outline:0;background-color:#337ab7}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#777}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);cursor:not-allowed}.open>.dropdown-menu{display:block}.open>a{outline:0}.dropdown-menu-right{left:auto;right:0}.dropdown-menu-left{left:0;right:auto}.dropdown-header{display:block;padding:3px 20px;font-size:12px;line-height:1.42857143;color:#777;white-space:nowrap}.dropdown-backdrop{position:fixed;left:0;right:0;bottom:0;top:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px dashed;border-bottom:4px solid \9;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:2px}@media (min-width:768px){.navbar-right .dropdown-menu{left:auto;right:0}.navbar-right .dropdown-menu-left{left:0;right:auto}}.btn-group,.btn-group-vertical{position:relative;display:inline-block;vertical-align:middle}.btn-group>.btn,.btn-group-vertical>.btn{position:relative;float:left}.btn-group>.btn:hover,.btn-group-vertical>.btn:hover,.btn-group>.btn:focus,.btn-group-vertical>.btn:focus,.btn-group>.btn:active,.btn-group-vertical>.btn:active,.btn-group>.btn.active,.btn-group-vertical>.btn.active{z-index:2}.btn-group .btn+.btn,.btn-group .btn+.btn-group,.btn-group .btn-group+.btn,.btn-group .btn-group+.btn-group{margin-left:-1px}.btn-toolbar{margin-left:-5px}.btn-toolbar .btn,.btn-toolbar .btn-group,.btn-toolbar .input-group{float:left}.btn-toolbar>.btn,.btn-toolbar>.btn-group,.btn-toolbar>.input-group{margin-left:5px}.btn-group>.btn:not(:first-child):not(:last-child):not(.dropdown-toggle){border-radius:0}.btn-group>.btn:first-child{margin-left:0}.btn-group>.btn:first-child:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn:last-child:not(:first-child),.btn-group>.dropdown-toggle:not(:first-child){border-bottom-left-radius:0;border-top-left-radius:0}.btn-group>.btn-group{float:left}.btn-group>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group>.btn-group:first-child:not(:last-child)>.btn:last-child,.btn-group>.btn-group:first-child:not(:last-child)>.dropdown-toggle{border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn-group:last-child:not(:first-child)>.btn:first-child{border-bottom-left-radius:0;border-top-left-radius:0}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{padding-left:8px;padding-right:8px}.btn-group>.btn-lg+.dropdown-toggle{padding-left:12px;padding-right:12px}.btn-group.open .dropdown-toggle{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,0.125);box-shadow:inset 0 3px 5px rgba(0,0,0,0.125)}.btn-group.open .dropdown-toggle.btn-link{-webkit-box-shadow:none;box-shadow:none}.btn .caret{margin-left:0}.btn-lg .caret{border-width:5px 5px 0;border-bottom-width:0}.dropup .btn-lg .caret{border-width:0 5px 5px}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group,.btn-group-vertical>.btn-group>.btn{display:block;float:none;width:100%;max-width:100%}.btn-group-vertical>.btn-group>.btn{float:none}.btn-group-vertical>.btn+.btn,.btn-group-vertical>.btn+.btn-group,.btn-group-vertical>.btn-group+.btn,.btn-group-vertical>.btn-group+.btn-group{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:not(:first-child):not(:last-child){border-radius:0}.btn-group-vertical>.btn:first-child:not(:last-child){border-top-right-radius:4px;border-top-left-radius:4px;border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn:last-child:not(:first-child){border-top-right-radius:0;border-top-left-radius:0;border-bottom-right-radius:4px;border-bottom-left-radius:4px}.btn-group-vertical>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group-vertical>.btn-group:first-child:not(:last-child)>.btn:last-child,.btn-group-vertical>.btn-group:first-child:not(:last-child)>.dropdown-toggle{border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:last-child:not(:first-child)>.btn:first-child{border-top-right-radius:0;border-top-left-radius:0}.btn-group-justified{display:table;width:100%;table-layout:fixed;border-collapse:separate}.btn-group-justified>.btn,.btn-group-justified>.btn-group{float:none;display:table-cell;width:1%}.btn-group-justified>.btn-group .btn{width:100%}.btn-group-justified>.btn-group .dropdown-menu{left:auto}[data-toggle="buttons"]>.btn input[type="radio"],[data-toggle="buttons"]>.btn-group>.btn input[type="radio"],[data-toggle="buttons"]>.btn input[type="checkbox"],[data-toggle="buttons"]>.btn-group>.btn input[type="checkbox"]{position:absolute;clip:rect(0, 0, 0, 0);pointer-events:none}.input-group{position:relative;display:table;border-collapse:separate}.input-group[class*="col-"]{float:none;padding-left:0;padding-right:0}.input-group .form-control{position:relative;z-index:2;float:left;width:100%;margin-bottom:0}.input-group .form-control:focus{z-index:3}.input-group-lg>.form-control,.input-group-lg>.input-group-addon,.input-group-lg>.input-group-btn>.btn{height:46px;padding:10px 16px;font-size:18px;line-height:1.3333333;border-radius:6px}select.input-group-lg>.form-control,select.input-group-lg>.input-group-addon,select.input-group-lg>.input-group-btn>.btn{height:46px;line-height:46px}textarea.input-group-lg>.form-control,textarea.input-group-lg>.input-group-addon,textarea.input-group-lg>.input-group-btn>.btn,select[multiple].input-group-lg>.form-control,select[multiple].input-group-lg>.input-group-addon,select[multiple].input-group-lg>.input-group-btn>.btn{height:auto}.input-group-sm>.form-control,.input-group-sm>.input-group-addon,.input-group-sm>.input-group-btn>.btn{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-group-sm>.form-control,select.input-group-sm>.input-group-addon,select.input-group-sm>.input-group-btn>.btn{height:30px;line-height:30px}textarea.input-group-sm>.form-control,textarea.input-group-sm>.input-group-addon,textarea.input-group-sm>.input-group-btn>.btn,select[multiple].input-group-sm>.form-control,select[multiple].input-group-sm>.input-group-addon,select[multiple].input-group-sm>.input-group-btn>.btn{height:auto}.input-group-addon,.input-group-btn,.input-group .form-control{display:table-cell}.input-group-addon:not(:first-child):not(:last-child),.input-group-btn:not(:first-child):not(:last-child),.input-group .form-control:not(:first-child):not(:last-child){border-radius:0}.input-group-addon,.input-group-btn{width:1%;white-space:nowrap;vertical-align:middle}.input-group-addon{padding:6px 12px;font-size:14px;font-weight:normal;line-height:1;color:#555;text-align:center;background-color:#eee;border:1px solid #ccc;border-radius:4px}.input-group-addon.input-sm{padding:5px 10px;font-size:12px;border-radius:3px}.input-group-addon.input-lg{padding:10px 16px;font-size:18px;border-radius:6px}.input-group-addon input[type="radio"],.input-group-addon input[type="checkbox"]{margin-top:0}.input-group .form-control:first-child,.input-group-addon:first-child,.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group>.btn,.input-group-btn:first-child>.dropdown-toggle,.input-group-btn:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group-btn:last-child>.btn-group:not(:last-child)>.btn{border-bottom-right-radius:0;border-top-right-radius:0}.input-group-addon:first-child{border-right:0}.input-group .form-control:last-child,.input-group-addon:last-child,.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group>.btn,.input-group-btn:last-child>.dropdown-toggle,.input-group-btn:first-child>.btn:not(:first-child),.input-group-btn:first-child>.btn-group:not(:first-child)>.btn{border-bottom-left-radius:0;border-top-left-radius:0}.input-group-addon:last-child{border-left:0}.input-group-btn{position:relative;font-size:0;white-space:nowrap}.input-group-btn>.btn{position:relative}.input-group-btn>.btn+.btn{margin-left:-1px}.input-group-btn>.btn:hover,.input-group-btn>.btn:focus,.input-group-btn>.btn:active{z-index:2}.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group{margin-right:-1px}.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group{z-index:2;margin-left:-1px}.nav{margin-bottom:0;padding-left:0;list-style:none}.nav>li{position:relative;display:block}.nav>li>a{position:relative;display:block;padding:10px 15px}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li.disabled>a{color:#777}.nav>li.disabled>a:hover,.nav>li.disabled>a:focus{color:#777;text-decoration:none;background-color:transparent;cursor:not-allowed}.nav .open>a,.nav .open>a:hover,.nav .open>a:focus{background-color:#eee;border-color:#337ab7}.nav .nav-divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.nav>li>a>img{max-width:none}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{float:left;margin-bottom:-1px}.nav-tabs>li>a{margin-right:2px;line-height:1.42857143;border:1px solid transparent;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>li.active>a,.nav-tabs>li.active>a:hover,.nav-tabs>li.active>a:focus{color:#555;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent;cursor:default}.nav-tabs.nav-justified{width:100%;border-bottom:0}.nav-tabs.nav-justified>li{float:none}.nav-tabs.nav-justified>li>a{text-align:center;margin-bottom:5px}.nav-tabs.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-tabs.nav-justified>li{display:table-cell;width:1%}.nav-tabs.nav-justified>li>a{margin-bottom:0}}.nav-tabs.nav-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs.nav-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border-bottom-color:#fff}}.nav-pills>li{float:left}.nav-pills>li>a{border-radius:4px}.nav-pills>li+li{margin-left:2px}.nav-pills>li.active>a,.nav-pills>li.active>a:hover,.nav-pills>li.active>a:focus{color:#fff;background-color:#337ab7}.nav-stacked>li{float:none}.nav-stacked>li+li{margin-top:2px;margin-left:0}.nav-justified{width:100%}.nav-justified>li{float:none}.nav-justified>li>a{text-align:center;margin-bottom:5px}.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-justified>li{display:table-cell;width:1%}.nav-justified>li>a{margin-bottom:0}}.nav-tabs-justified{border-bottom:0}.nav-tabs-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border-bottom-color:#fff}}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-right-radius:0;border-top-left-radius:0}.navbar{position:relative;min-height:50px;margin-bottom:20px;border:1px solid transparent}@media (min-width:768px){.navbar{border-radius:4px}}@media (min-width:768px){.navbar-header{float:left}}.navbar-collapse{overflow-x:visible;padding-right:15px;padding-left:15px;border-top:1px solid transparent;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1);-webkit-overflow-scrolling:touch}.navbar-collapse.in{overflow-y:auto}@media (min-width:768px){.navbar-collapse{width:auto;border-top:0;-webkit-box-shadow:none;box-shadow:none}.navbar-collapse.collapse{display:block !important;height:auto !important;padding-bottom:0;overflow:visible !important}.navbar-collapse.in{overflow-y:visible}.navbar-fixed-top .navbar-collapse,.navbar-static-top .navbar-collapse,.navbar-fixed-bottom .navbar-collapse{padding-left:0;padding-right:0}}.navbar-fixed-top .navbar-collapse,.navbar-fixed-bottom .navbar-collapse{max-height:340px}@media (max-device-width:480px) and (orientation:landscape){.navbar-fixed-top .navbar-collapse,.navbar-fixed-bottom .navbar-collapse{max-height:200px}}.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:-15px;margin-left:-15px}@media (min-width:768px){.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:0;margin-left:0}}.navbar-static-top{z-index:1000;border-width:0 0 1px}@media (min-width:768px){.navbar-static-top{border-radius:0}}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030}@media (min-width:768px){.navbar-fixed-top,.navbar-fixed-bottom{border-radius:0}}.navbar-fixed-top{top:0;border-width:0 0 1px}.navbar-fixed-bottom{bottom:0;margin-bottom:0;border-width:1px 0 0}.navbar-brand{float:left;padding:15px 15px;font-size:18px;line-height:20px;height:50px}.navbar-brand:hover,.navbar-brand:focus{text-decoration:none}.navbar-brand>img{display:block}@media (min-width:768px){.navbar>.container .navbar-brand,.navbar>.container-fluid .navbar-brand{margin-left:-15px}}.navbar-toggle{position:relative;float:right;margin-right:15px;padding:9px 10px;margin-top:8px;margin-bottom:8px;background-color:transparent;background-image:none;border:1px solid transparent;border-radius:4px}.navbar-toggle:focus{outline:0}.navbar-toggle .icon-bar{display:block;width:22px;height:2px;border-radius:1px}.navbar-toggle .icon-bar+.icon-bar{margin-top:4px}@media (min-width:768px){.navbar-toggle{display:none}}.navbar-nav{margin:7.5px -15px}.navbar-nav>li>a{padding-top:10px;padding-bottom:10px;line-height:20px}@media (max-width:767px){.navbar-nav .open .dropdown-menu{position:static;float:none;width:auto;margin-top:0;background-color:transparent;border:0;-webkit-box-shadow:none;box-shadow:none}.navbar-nav .open .dropdown-menu>li>a,.navbar-nav .open .dropdown-menu .dropdown-header{padding:5px 15px 5px 25px}.navbar-nav .open .dropdown-menu>li>a{line-height:20px}.navbar-nav .open .dropdown-menu>li>a:hover,.navbar-nav .open .dropdown-menu>li>a:focus{background-image:none}}@media (min-width:768px){.navbar-nav{float:left;margin:0}.navbar-nav>li{float:left}.navbar-nav>li>a{padding-top:15px;padding-bottom:15px}}.navbar-form{margin-left:-15px;margin-right:-15px;padding:10px 15px;border-top:1px solid transparent;border-bottom:1px solid transparent;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);margin-top:8px;margin-bottom:8px}@media (min-width:768px){.navbar-form .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.navbar-form .form-control{display:inline-block;width:auto;vertical-align:middle}.navbar-form .form-control-static{display:inline-block}.navbar-form .input-group{display:inline-table;vertical-align:middle}.navbar-form .input-group .input-group-addon,.navbar-form .input-group .input-group-btn,.navbar-form .input-group .form-control{width:auto}.navbar-form .input-group>.form-control{width:100%}.navbar-form .control-label{margin-bottom:0;vertical-align:middle}.navbar-form .radio,.navbar-form .checkbox{display:inline-block;margin-top:0;margin-bottom:0;vertical-align:middle}.navbar-form .radio label,.navbar-form .checkbox label{padding-left:0}.navbar-form .radio input[type="radio"],.navbar-form .checkbox input[type="checkbox"]{position:relative;margin-left:0}.navbar-form .has-feedback .form-control-feedback{top:0}}@media (max-width:767px){.navbar-form .form-group{margin-bottom:5px}.navbar-form .form-group:last-child{margin-bottom:0}}@media (min-width:768px){.navbar-form{width:auto;border:0;margin-left:0;margin-right:0;padding-top:0;padding-bottom:0;-webkit-box-shadow:none;box-shadow:none}}.navbar-nav>li>.dropdown-menu{margin-top:0;border-top-right-radius:0;border-top-left-radius:0}.navbar-fixed-bottom .navbar-nav>li>.dropdown-menu{margin-bottom:0;border-top-right-radius:4px;border-top-left-radius:4px;border-bottom-right-radius:0;border-bottom-left-radius:0}.navbar-btn{margin-top:8px;margin-bottom:8px}.navbar-btn.btn-sm{margin-top:10px;margin-bottom:10px}.navbar-btn.btn-xs{margin-top:14px;margin-bottom:14px}.navbar-text{margin-top:15px;margin-bottom:15px}@media (min-width:768px){.navbar-text{float:left;margin-left:15px;margin-right:15px}}@media (min-width:768px){.navbar-left{float:left !important}.navbar-right{float:right !important;margin-right:-15px}.navbar-right~.navbar-right{margin-right:0}}.navbar-default{background-color:#f8f8f8;border-color:#e7e7e7}.navbar-default .navbar-brand{color:#777}.navbar-default .navbar-brand:hover,.navbar-default .navbar-brand:focus{color:#5e5e5e;background-color:transparent}.navbar-default .navbar-text{color:#777}.navbar-default .navbar-nav>li>a{color:#777}.navbar-default .navbar-nav>li>a:hover,.navbar-default .navbar-nav>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav>.active>a,.navbar-default .navbar-nav>.active>a:hover,.navbar-default .navbar-nav>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav>.disabled>a,.navbar-default .navbar-nav>.disabled>a:hover,.navbar-default .navbar-nav>.disabled>a:focus{color:#ccc;background-color:transparent}.navbar-default .navbar-toggle{border-color:#ddd}.navbar-default .navbar-toggle:hover,.navbar-default .navbar-toggle:focus{background-color:#ddd}.navbar-default .navbar-toggle .icon-bar{background-color:#888}.navbar-default .navbar-collapse,.navbar-default .navbar-form{border-color:#e7e7e7}.navbar-default .navbar-nav>.open>a,.navbar-default .navbar-nav>.open>a:hover,.navbar-default .navbar-nav>.open>a:focus{background-color:#e7e7e7;color:#555}@media (max-width:767px){.navbar-default .navbar-nav .open .dropdown-menu>li>a{color:#777}.navbar-default .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav .open .dropdown-menu>.active>a,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#ccc;background-color:transparent}}.navbar-default .navbar-link{color:#777}.navbar-default .navbar-link:hover{color:#333}.navbar-default .btn-link{color:#777}.navbar-default .btn-link:hover,.navbar-default .btn-link:focus{color:#333}.navbar-default .btn-link[disabled]:hover,fieldset[disabled] .navbar-default .btn-link:hover,.navbar-default .btn-link[disabled]:focus,fieldset[disabled] .navbar-default .btn-link:focus{color:#ccc}.navbar-inverse{background-color:#222;border-color:#080808}.navbar-inverse .navbar-brand{color:#9d9d9d}.navbar-inverse .navbar-brand:hover,.navbar-inverse .navbar-brand:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-text{color:#9d9d9d}.navbar-inverse .navbar-nav>li>a{color:#9d9d9d}.navbar-inverse .navbar-nav>li>a:hover,.navbar-inverse .navbar-nav>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav>.active>a,.navbar-inverse .navbar-nav>.active>a:hover,.navbar-inverse .navbar-nav>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav>.disabled>a,.navbar-inverse .navbar-nav>.disabled>a:hover,.navbar-inverse .navbar-nav>.disabled>a:focus{color:#444;background-color:transparent}.navbar-inverse .navbar-toggle{border-color:#333}.navbar-inverse .navbar-toggle:hover,.navbar-inverse .navbar-toggle:focus{background-color:#333}.navbar-inverse .navbar-toggle .icon-bar{background-color:#fff}.navbar-inverse .navbar-collapse,.navbar-inverse .navbar-form{border-color:#101010}.navbar-inverse .navbar-nav>.open>a,.navbar-inverse .navbar-nav>.open>a:hover,.navbar-inverse .navbar-nav>.open>a:focus{background-color:#080808;color:#fff}@media (max-width:767px){.navbar-inverse .navbar-nav .open .dropdown-menu>.dropdown-header{border-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu .divider{background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a{color:#9d9d9d}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#444;background-color:transparent}}.navbar-inverse .navbar-link{color:#9d9d9d}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .btn-link{color:#9d9d9d}.navbar-inverse .btn-link:hover,.navbar-inverse .btn-link:focus{color:#fff}.navbar-inverse .btn-link[disabled]:hover,fieldset[disabled] .navbar-inverse .btn-link:hover,.navbar-inverse .btn-link[disabled]:focus,fieldset[disabled] .navbar-inverse .btn-link:focus{color:#444}.breadcrumb{padding:8px 15px;margin-bottom:20px;list-style:none;background-color:#f5f5f5;border-radius:4px}.breadcrumb>li{display:inline-block}.breadcrumb>li+li:before{content:"/\00a0";padding:0 5px;color:#ccc}.breadcrumb>.active{color:#777}.pagination{display:inline-block;padding-left:0;margin:20px 0;border-radius:4px}.pagination>li{display:inline}.pagination>li>a,.pagination>li>span{position:relative;float:left;padding:6px 12px;line-height:1.42857143;text-decoration:none;color:#337ab7;background-color:#fff;border:1px solid #ddd;margin-left:-1px}.pagination>li:first-child>a,.pagination>li:first-child>span{margin-left:0;border-bottom-left-radius:4px;border-top-left-radius:4px}.pagination>li:last-child>a,.pagination>li:last-child>span{border-bottom-right-radius:4px;border-top-right-radius:4px}.pagination>li>a:hover,.pagination>li>span:hover,.pagination>li>a:focus,.pagination>li>span:focus{z-index:2;color:#23527c;background-color:#eee;border-color:#ddd}.pagination>.active>a,.pagination>.active>span,.pagination>.active>a:hover,.pagination>.active>span:hover,.pagination>.active>a:focus,.pagination>.active>span:focus{z-index:3;color:#fff;background-color:#337ab7;border-color:#337ab7;cursor:default}.pagination>.disabled>span,.pagination>.disabled>span:hover,.pagination>.disabled>span:focus,.pagination>.disabled>a,.pagination>.disabled>a:hover,.pagination>.disabled>a:focus{color:#777;background-color:#fff;border-color:#ddd;cursor:not-allowed}.pagination-lg>li>a,.pagination-lg>li>span{padding:10px 16px;font-size:18px;line-height:1.3333333}.pagination-lg>li:first-child>a,.pagination-lg>li:first-child>span{border-bottom-left-radius:6px;border-top-left-radius:6px}.pagination-lg>li:last-child>a,.pagination-lg>li:last-child>span{border-bottom-right-radius:6px;border-top-right-radius:6px}.pagination-sm>li>a,.pagination-sm>li>span{padding:5px 10px;font-size:12px;line-height:1.5}.pagination-sm>li:first-child>a,.pagination-sm>li:first-child>span{border-bottom-left-radius:3px;border-top-left-radius:3px}.pagination-sm>li:last-child>a,.pagination-sm>li:last-child>span{border-bottom-right-radius:3px;border-top-right-radius:3px}.pager{padding-left:0;margin:20px 0;list-style:none;text-align:center}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#eee}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#777;background-color:#fff;cursor:not-allowed}.label{display:inline;padding:.2em .6em .3em;font-size:75%;font-weight:bold;line-height:1;color:#fff;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:.25em}a.label:hover,a.label:focus{color:#fff;text-decoration:none;cursor:pointer}.label:empty{display:none}.btn .label{position:relative;top:-1px}.label-default{background-color:#777}.label-default[href]:hover,.label-default[href]:focus{background-color:#5e5e5e}.label-primary{background-color:#337ab7}.label-primary[href]:hover,.label-primary[href]:focus{background-color:#286090}.label-success{background-color:#5cb85c}.label-success[href]:hover,.label-success[href]:focus{background-color:#449d44}.label-info{background-color:#5bc0de}.label-info[href]:hover,.label-info[href]:focus{background-color:#31b0d5}.label-warning{background-color:#f0ad4e}.label-warning[href]:hover,.label-warning[href]:focus{background-color:#ec971f}.label-danger{background-color:#d9534f}.label-danger[href]:hover,.label-danger[href]:focus{background-color:#c9302c}.badge{display:inline-block;min-width:10px;padding:3px 7px;font-size:12px;font-weight:bold;color:#fff;line-height:1;vertical-align:middle;white-space:nowrap;text-align:center;background-color:#777;border-radius:10px}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.btn-xs .badge,.btn-group-xs>.btn .badge{top:0;padding:1px 5px}a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}.list-group-item.active>.badge,.nav-pills>.active>a>.badge{color:#337ab7;background-color:#fff}.list-group-item>.badge{float:right}.list-group-item>.badge+.badge{margin-right:5px}.nav-pills>li>a>.badge{margin-left:3px}.jumbotron{padding-top:30px;padding-bottom:30px;margin-bottom:30px;color:inherit;background-color:#eee}.jumbotron h1,.jumbotron .h1{color:inherit}.jumbotron p{margin-bottom:15px;font-size:21px;font-weight:200}.jumbotron>hr{border-top-color:#d5d5d5}.container .jumbotron,.container-fluid .jumbotron{border-radius:6px;padding-left:15px;padding-right:15px}.jumbotron .container{max-width:100%}@media screen and (min-width:768px){.jumbotron{padding-top:48px;padding-bottom:48px}.container .jumbotron,.container-fluid .jumbotron{padding-left:60px;padding-right:60px}.jumbotron h1,.jumbotron .h1{font-size:63px}}.thumbnail{display:block;padding:4px;margin-bottom:20px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:border .2s ease-in-out;-o-transition:border .2s ease-in-out;transition:border .2s ease-in-out}.thumbnail>img,.thumbnail a>img{margin-left:auto;margin-right:auto}a.thumbnail:hover,a.thumbnail:focus,a.thumbnail.active{border-color:#337ab7}.thumbnail .caption{padding:9px;color:#333}.alert{padding:15px;margin-bottom:20px;border:1px solid transparent;border-radius:4px}.alert h4{margin-top:0;color:inherit}.alert .alert-link{font-weight:bold}.alert>p,.alert>ul{margin-bottom:0}.alert>p+p{margin-top:5px}.alert-dismissable,.alert-dismissible{padding-right:35px}.alert-dismissable .close,.alert-dismissible .close{position:relative;top:-2px;right:-21px;color:inherit}.alert-success{background-color:#dff0d8;border-color:#d6e9c6;color:#3c763d}.alert-success hr{border-top-color:#c9e2b3}.alert-success .alert-link{color:#2b542c}.alert-info{background-color:#d9edf7;border-color:#bce8f1;color:#31708f}.alert-info hr{border-top-color:#a6e1ec}.alert-info .alert-link{color:#245269}.alert-warning{background-color:#fcf8e3;border-color:#faebcc;color:#8a6d3b}.alert-warning hr{border-top-color:#f7e1b5}.alert-warning .alert-link{color:#66512c}.alert-danger{background-color:#f2dede;border-color:#ebccd1;color:#a94442}.alert-danger hr{border-top-color:#e4b9c0}.alert-danger .alert-link{color:#843534}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{overflow:hidden;height:20px;margin-bottom:20px;background-color:#f5f5f5;border-radius:4px;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress-bar{float:left;width:0%;height:100%;font-size:12px;line-height:20px;color:#fff;text-align:center;background-color:#337ab7;-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress-striped .progress-bar,.progress-bar-striped{background-image:-webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);-webkit-background-size:40px 40px;background-size:40px 40px}.progress.active .progress-bar,.progress-bar.active{-webkit-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-bar-success{background-color:#5cb85c}.progress-striped .progress-bar-success{background-image:-webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)}.progress-bar-info{background-color:#5bc0de}.progress-striped .progress-bar-info{background-image:-webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)}.progress-bar-warning{background-color:#f0ad4e}.progress-striped .progress-bar-warning{background-image:-webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)}.progress-bar-danger{background-color:#d9534f}.progress-striped .progress-bar-danger{background-image:-webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)}.media{margin-top:15px}.media:first-child{margin-top:0}.media,.media-body{zoom:1;overflow:hidden}.media-body{width:10000px}.media-object{display:block}.media-object.img-thumbnail{max-width:none}.media-right,.media>.pull-right{padding-left:10px}.media-left,.media>.pull-left{padding-right:10px}.media-left,.media-right,.media-body{display:table-cell;vertical-align:top}.media-middle{vertical-align:middle}.media-bottom{vertical-align:bottom}.media-heading{margin-top:0;margin-bottom:5px}.media-list{padding-left:0;list-style:none}.list-group{margin-bottom:20px;padding-left:0}.list-group-item{position:relative;display:block;padding:10px 15px;margin-bottom:-1px;background-color:#fff;border:1px solid #ddd}.list-group-item:first-child{border-top-right-radius:4px;border-top-left-radius:4px}.list-group-item:last-child{margin-bottom:0;border-bottom-right-radius:4px;border-bottom-left-radius:4px}a.list-group-item,button.list-group-item{color:#555}a.list-group-item .list-group-item-heading,button.list-group-item .list-group-item-heading{color:#333}a.list-group-item:hover,button.list-group-item:hover,a.list-group-item:focus,button.list-group-item:focus{text-decoration:none;color:#555;background-color:#f5f5f5}button.list-group-item{width:100%;text-align:left}.list-group-item.disabled,.list-group-item.disabled:hover,.list-group-item.disabled:focus{background-color:#eee;color:#777;cursor:not-allowed}.list-group-item.disabled .list-group-item-heading,.list-group-item.disabled:hover .list-group-item-heading,.list-group-item.disabled:focus .list-group-item-heading{color:inherit}.list-group-item.disabled .list-group-item-text,.list-group-item.disabled:hover .list-group-item-text,.list-group-item.disabled:focus .list-group-item-text{color:#777}.list-group-item.active,.list-group-item.active:hover,.list-group-item.active:focus{z-index:2;color:#fff;background-color:#337ab7;border-color:#337ab7}.list-group-item.active .list-group-item-heading,.list-group-item.active:hover .list-group-item-heading,.list-group-item.active:focus .list-group-item-heading,.list-group-item.active .list-group-item-heading>small,.list-group-item.active:hover .list-group-item-heading>small,.list-group-item.active:focus .list-group-item-heading>small,.list-group-item.active .list-group-item-heading>.small,.list-group-item.active:hover .list-group-item-heading>.small,.list-group-item.active:focus .list-group-item-heading>.small{color:inherit}.list-group-item.active .list-group-item-text,.list-group-item.active:hover .list-group-item-text,.list-group-item.active:focus .list-group-item-text{color:#c7ddef}.list-group-item-success{color:#3c763d;background-color:#dff0d8}a.list-group-item-success,button.list-group-item-success{color:#3c763d}a.list-group-item-success .list-group-item-heading,button.list-group-item-success .list-group-item-heading{color:inherit}a.list-group-item-success:hover,button.list-group-item-success:hover,a.list-group-item-success:focus,button.list-group-item-success:focus{color:#3c763d;background-color:#d0e9c6}a.list-group-item-success.active,button.list-group-item-success.active,a.list-group-item-success.active:hover,button.list-group-item-success.active:hover,a.list-group-item-success.active:focus,button.list-group-item-success.active:focus{color:#fff;background-color:#3c763d;border-color:#3c763d}.list-group-item-info{color:#31708f;background-color:#d9edf7}a.list-group-item-info,button.list-group-item-info{color:#31708f}a.list-group-item-info .list-group-item-heading,button.list-group-item-info .list-group-item-heading{color:inherit}a.list-group-item-info:hover,button.list-group-item-info:hover,a.list-group-item-info:focus,button.list-group-item-info:focus{color:#31708f;background-color:#c4e3f3}a.list-group-item-info.active,button.list-group-item-info.active,a.list-group-item-info.active:hover,button.list-group-item-info.active:hover,a.list-group-item-info.active:focus,button.list-group-item-info.active:focus{color:#fff;background-color:#31708f;border-color:#31708f}.list-group-item-warning{color:#8a6d3b;background-color:#fcf8e3}a.list-group-item-warning,button.list-group-item-warning{color:#8a6d3b}a.list-group-item-warning .list-group-item-heading,button.list-group-item-warning .list-group-item-heading{color:inherit}a.list-group-item-warning:hover,button.list-group-item-warning:hover,a.list-group-item-warning:focus,button.list-group-item-warning:focus{color:#8a6d3b;background-color:#faf2cc}a.list-group-item-warning.active,button.list-group-item-warning.active,a.list-group-item-warning.active:hover,button.list-group-item-warning.active:hover,a.list-group-item-warning.active:focus,button.list-group-item-warning.active:focus{color:#fff;background-color:#8a6d3b;border-color:#8a6d3b}.list-group-item-danger{color:#a94442;background-color:#f2dede}a.list-group-item-danger,button.list-group-item-danger{color:#a94442}a.list-group-item-danger .list-group-item-heading,button.list-group-item-danger .list-group-item-heading{color:inherit}a.list-group-item-danger:hover,button.list-group-item-danger:hover,a.list-group-item-danger:focus,button.list-group-item-danger:focus{color:#a94442;background-color:#ebcccc}a.list-group-item-danger.active,button.list-group-item-danger.active,a.list-group-item-danger.active:hover,button.list-group-item-danger.active:hover,a.list-group-item-danger.active:focus,button.list-group-item-danger.active:focus{color:#fff;background-color:#a94442;border-color:#a94442}.list-group-item-heading{margin-top:0;margin-bottom:5px}.list-group-item-text{margin-bottom:0;line-height:1.3}.panel{margin-bottom:20px;background-color:#fff;border:1px solid transparent;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,0.05);box-shadow:0 1px 1px rgba(0,0,0,0.05)}.panel-body{padding:15px}.panel-heading{padding:10px 15px;border-bottom:1px solid transparent;border-top-right-radius:3px;border-top-left-radius:3px}.panel-heading>.dropdown .dropdown-toggle{color:inherit}.panel-title{margin-top:0;margin-bottom:0;font-size:16px;color:inherit}.panel-title>a,.panel-title>small,.panel-title>.small,.panel-title>small>a,.panel-title>.small>a{color:inherit}.panel-footer{padding:10px 15px;background-color:#f5f5f5;border-top:1px solid #ddd;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.list-group,.panel>.panel-collapse>.list-group{margin-bottom:0}.panel>.list-group .list-group-item,.panel>.panel-collapse>.list-group .list-group-item{border-width:1px 0;border-radius:0}.panel>.list-group:first-child .list-group-item:first-child,.panel>.panel-collapse>.list-group:first-child .list-group-item:first-child{border-top:0;border-top-right-radius:3px;border-top-left-radius:3px}.panel>.list-group:last-child .list-group-item:last-child,.panel>.panel-collapse>.list-group:last-child .list-group-item:last-child{border-bottom:0;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.panel-heading+.panel-collapse>.list-group .list-group-item:first-child{border-top-right-radius:0;border-top-left-radius:0}.panel-heading+.list-group .list-group-item:first-child{border-top-width:0}.list-group+.panel-footer{border-top-width:0}.panel>.table,.panel>.table-responsive>.table,.panel>.panel-collapse>.table{margin-bottom:0}.panel>.table caption,.panel>.table-responsive>.table caption,.panel>.panel-collapse>.table caption{padding-left:15px;padding-right:15px}.panel>.table:first-child,.panel>.table-responsive:first-child>.table:first-child{border-top-right-radius:3px;border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child{border-top-left-radius:3px;border-top-right-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:first-child{border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:last-child{border-top-right-radius:3px}.panel>.table:last-child,.panel>.table-responsive:last-child>.table:last-child{border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child{border-bottom-left-radius:3px;border-bottom-right-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:first-child{border-bottom-left-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:last-child{border-bottom-right-radius:3px}.panel>.panel-body+.table,.panel>.panel-body+.table-responsive,.panel>.table+.panel-body,.panel>.table-responsive+.panel-body{border-top:1px solid #ddd}.panel>.table>tbody:first-child>tr:first-child th,.panel>.table>tbody:first-child>tr:first-child td{border-top:0}.panel>.table-bordered,.panel>.table-responsive>.table-bordered{border:0}.panel>.table-bordered>thead>tr>th:first-child,.panel>.table-responsive>.table-bordered>thead>tr>th:first-child,.panel>.table-bordered>tbody>tr>th:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:first-child,.panel>.table-bordered>tfoot>tr>th:first-child,.panel>.table-responsive>.table-bordered>tfoot>tr>th:first-child,.panel>.table-bordered>thead>tr>td:first-child,.panel>.table-responsive>.table-bordered>thead>tr>td:first-child,.panel>.table-bordered>tbody>tr>td:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:first-child,.panel>.table-bordered>tfoot>tr>td:first-child,.panel>.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.panel>.table-bordered>thead>tr>th:last-child,.panel>.table-responsive>.table-bordered>thead>tr>th:last-child,.panel>.table-bordered>tbody>tr>th:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:last-child,.panel>.table-bordered>tfoot>tr>th:last-child,.panel>.table-responsive>.table-bordered>tfoot>tr>th:last-child,.panel>.table-bordered>thead>tr>td:last-child,.panel>.table-responsive>.table-bordered>thead>tr>td:last-child,.panel>.table-bordered>tbody>tr>td:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:last-child,.panel>.table-bordered>tfoot>tr>td:last-child,.panel>.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.panel>.table-bordered>thead>tr:first-child>td,.panel>.table-responsive>.table-bordered>thead>tr:first-child>td,.panel>.table-bordered>tbody>tr:first-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>td,.panel>.table-bordered>thead>tr:first-child>th,.panel>.table-responsive>.table-bordered>thead>tr:first-child>th,.panel>.table-bordered>tbody>tr:first-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>th{border-bottom:0}.panel>.table-bordered>tbody>tr:last-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>td,.panel>.table-bordered>tfoot>tr:last-child>td,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>td,.panel>.table-bordered>tbody>tr:last-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>th,.panel>.table-bordered>tfoot>tr:last-child>th,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>th{border-bottom:0}.panel>.table-responsive{border:0;margin-bottom:0}.panel-group{margin-bottom:20px}.panel-group .panel{margin-bottom:0;border-radius:4px}.panel-group .panel+.panel{margin-top:5px}.panel-group .panel-heading{border-bottom:0}.panel-group .panel-heading+.panel-collapse>.panel-body,.panel-group .panel-heading+.panel-collapse>.list-group{border-top:1px solid #ddd}.panel-group .panel-footer{border-top:0}.panel-group .panel-footer+.panel-collapse .panel-body{border-bottom:1px solid #ddd}.panel-default{border-color:#ddd}.panel-default>.panel-heading{color:#333;background-color:#f5f5f5;border-color:#ddd}.panel-default>.panel-heading+.panel-collapse>.panel-body{border-top-color:#ddd}.panel-default>.panel-heading .badge{color:#f5f5f5;background-color:#333}.panel-default>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#ddd}.panel-primary{border-color:#337ab7}.panel-primary>.panel-heading{color:#fff;background-color:#337ab7;border-color:#337ab7}.panel-primary>.panel-heading+.panel-collapse>.panel-body{border-top-color:#337ab7}.panel-primary>.panel-heading .badge{color:#337ab7;background-color:#fff}.panel-primary>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#337ab7}.panel-success{border-color:#d6e9c6}.panel-success>.panel-heading{color:#3c763d;background-color:#dff0d8;border-color:#d6e9c6}.panel-success>.panel-heading+.panel-collapse>.panel-body{border-top-color:#d6e9c6}.panel-success>.panel-heading .badge{color:#dff0d8;background-color:#3c763d}.panel-success>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#d6e9c6}.panel-info{border-color:#bce8f1}.panel-info>.panel-heading{color:#31708f;background-color:#d9edf7;border-color:#bce8f1}.panel-info>.panel-heading+.panel-collapse>.panel-body{border-top-color:#bce8f1}.panel-info>.panel-heading .badge{color:#d9edf7;background-color:#31708f}.panel-info>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#bce8f1}.panel-warning{border-color:#faebcc}.panel-warning>.panel-heading{color:#8a6d3b;background-color:#fcf8e3;border-color:#faebcc}.panel-warning>.panel-heading+.panel-collapse>.panel-body{border-top-color:#faebcc}.panel-warning>.panel-heading .badge{color:#fcf8e3;background-color:#8a6d3b}.panel-warning>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#faebcc}.panel-danger{border-color:#ebccd1}.panel-danger>.panel-heading{color:#a94442;background-color:#f2dede;border-color:#ebccd1}.panel-danger>.panel-heading+.panel-collapse>.panel-body{border-top-color:#ebccd1}.panel-danger>.panel-heading .badge{color:#f2dede;background-color:#a94442}.panel-danger>.panel-footer+.panel-collapse>.panel-body{border-bottom-color:#ebccd1}.embed-responsive{position:relative;display:block;height:0;padding:0;overflow:hidden}.embed-responsive .embed-responsive-item,.embed-responsive iframe,.embed-responsive embed,.embed-responsive object,.embed-responsive video{position:absolute;top:0;left:0;bottom:0;height:100%;width:100%;border:0}.embed-responsive-16by9{padding-bottom:56.25%}.embed-responsive-4by3{padding-bottom:75%}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-lg{padding:24px;border-radius:6px}.well-sm{padding:9px;border-radius:3px}.close{float:right;font-size:21px;font-weight:bold;line-height:1;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.5;filter:alpha(opacity=50)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.modal-open{overflow:hidden}.modal{display:none;overflow:hidden;position:fixed;top:0;right:0;bottom:0;left:0;z-index:1050;-webkit-overflow-scrolling:touch;outline:0}.modal.fade .modal-dialog{-webkit-transform:translate(0, -25%);-ms-transform:translate(0, -25%);-o-transform:translate(0, -25%);transform:translate(0, -25%);-webkit-transition:-webkit-transform 0.3s ease-out;-o-transition:-o-transform 0.3s ease-out;transition:transform 0.3s ease-out}.modal.in .modal-dialog{-webkit-transform:translate(0, 0);-ms-transform:translate(0, 0);-o-transform:translate(0, 0);transform:translate(0, 0)}.modal-open .modal{overflow-x:hidden;overflow-y:auto}.modal-dialog{position:relative;width:auto;margin:10px}.modal-content{position:relative;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.2);border-radius:6px;-webkit-box-shadow:0 3px 9px rgba(0,0,0,0.5);box-shadow:0 3px 9px rgba(0,0,0,0.5);-webkit-background-clip:padding-box;background-clip:padding-box;outline:0}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0;filter:alpha(opacity=0)}.modal-backdrop.in{opacity:.5;filter:alpha(opacity=50)}.modal-header{padding:15px;border-bottom:1px solid #e5e5e5}.modal-header .close{margin-top:-2px}.modal-title{margin:0;line-height:1.42857143}.modal-body{position:relative;padding:15px}.modal-footer{padding:15px;text-align:right;border-top:1px solid #e5e5e5}.modal-footer .btn+.btn{margin-left:5px;margin-bottom:0}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}.modal-scrollbar-measure{position:absolute;top:-9999px;width:50px;height:50px;overflow:scroll}@media (min-width:768px){.modal-dialog{width:600px;margin:30px auto}.modal-content{-webkit-box-shadow:0 5px 15px rgba(0,0,0,0.5);box-shadow:0 5px 15px rgba(0,0,0,0.5)}.modal-sm{width:300px}}@media (min-width:992px){.modal-lg{width:900px}}.tooltip{position:absolute;z-index:1070;display:block;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-style:normal;font-weight:normal;letter-spacing:normal;line-break:auto;line-height:1.42857143;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;white-space:normal;word-break:normal;word-spacing:normal;word-wrap:normal;font-size:12px;opacity:0;filter:alpha(opacity=0)}.tooltip.in{opacity:.9;filter:alpha(opacity=90)}.tooltip.top{margin-top:-3px;padding:5px 0}.tooltip.right{margin-left:3px;padding:0 5px}.tooltip.bottom{margin-top:3px;padding:5px 0}.tooltip.left{margin-left:-3px;padding:0 5px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;background-color:#000;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.top-left .tooltip-arrow{bottom:0;right:5px;margin-bottom:-5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.top-right .tooltip-arrow{bottom:0;left:5px;margin-bottom:-5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-width:5px 5px 5px 0;border-right-color:#000}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-width:5px 0 5px 5px;border-left-color:#000}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-width:0 5px 5px;border-bottom-color:#000}.tooltip.bottom-left .tooltip-arrow{top:0;right:5px;margin-top:-5px;border-width:0 5px 5px;border-bottom-color:#000}.tooltip.bottom-right .tooltip-arrow{top:0;left:5px;margin-top:-5px;border-width:0 5px 5px;border-bottom-color:#000}.popover{position:absolute;top:0;left:0;z-index:1060;display:none;max-width:276px;padding:1px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-style:normal;font-weight:normal;letter-spacing:normal;line-break:auto;line-height:1.42857143;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;white-space:normal;word-break:normal;word-spacing:normal;word-wrap:normal;font-size:14px;background-color:#fff;-webkit-background-clip:padding-box;background-clip:padding-box;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2)}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{margin:0;padding:8px 14px;font-size:14px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover>.arrow,.popover>.arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover>.arrow{border-width:11px}.popover>.arrow:after{border-width:10px;content:""}.popover.top>.arrow{left:50%;margin-left:-11px;border-bottom-width:0;border-top-color:#999;border-top-color:rgba(0,0,0,0.25);bottom:-11px}.popover.top>.arrow:after{content:" ";bottom:1px;margin-left:-10px;border-bottom-width:0;border-top-color:#fff}.popover.right>.arrow{top:50%;left:-11px;margin-top:-11px;border-left-width:0;border-right-color:#999;border-right-color:rgba(0,0,0,0.25)}.popover.right>.arrow:after{content:" ";left:1px;bottom:-10px;border-left-width:0;border-right-color:#fff}.popover.bottom>.arrow{left:50%;margin-left:-11px;border-top-width:0;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,0.25);top:-11px}.popover.bottom>.arrow:after{content:" ";top:1px;margin-left:-10px;border-top-width:0;border-bottom-color:#fff}.popover.left>.arrow{top:50%;right:-11px;margin-top:-11px;border-right-width:0;border-left-color:#999;border-left-color:rgba(0,0,0,0.25)}.popover.left>.arrow:after{content:" ";right:1px;border-right-width:0;border-left-color:#fff;bottom:-10px}.carousel{position:relative}.carousel-inner{position:relative;overflow:hidden;width:100%}.carousel-inner>.item{display:none;position:relative;-webkit-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{line-height:1}@media all and (transform-3d),(-webkit-transform-3d){.carousel-inner>.item{-webkit-transition:-webkit-transform 0.6s ease-in-out;-o-transition:-o-transform 0.6s ease-in-out;transition:transform 0.6s ease-in-out;-webkit-backface-visibility:hidden;backface-visibility:hidden;-webkit-perspective:1000px;perspective:1000px}.carousel-inner>.item.next,.carousel-inner>.item.active.right{-webkit-transform:translate3d(100%, 0, 0);transform:translate3d(100%, 0, 0);left:0}.carousel-inner>.item.prev,.carousel-inner>.item.active.left{-webkit-transform:translate3d(-100%, 0, 0);transform:translate3d(-100%, 0, 0);left:0}.carousel-inner>.item.next.left,.carousel-inner>.item.prev.right,.carousel-inner>.item.active{-webkit-transform:translate3d(0, 0, 0);transform:translate3d(0, 0, 0);left:0}}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.active.right{left:100%}.carousel-control{position:absolute;top:0;left:0;bottom:0;width:15%;opacity:.5;filter:alpha(opacity=50);font-size:20px;color:#fff;text-align:center;text-shadow:0 1px 2px rgba(0,0,0,0.6);background-color:rgba(0,0,0,0)}.carousel-control.left{background-image:-webkit-linear-gradient(left, rgba(0,0,0,0.5) 0, rgba(0,0,0,0.0001) 100%);background-image:-o-linear-gradient(left, rgba(0,0,0,0.5) 0, rgba(0,0,0,0.0001) 100%);background-image:-webkit-gradient(linear, left top, right top, color-stop(0, rgba(0,0,0,0.5)), to(rgba(0,0,0,0.0001)));background-image:linear-gradient(to right, rgba(0,0,0,0.5) 0, rgba(0,0,0,0.0001) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1)}.carousel-control.right{left:auto;right:0;background-image:-webkit-linear-gradient(left, rgba(0,0,0,0.0001) 0, rgba(0,0,0,0.5) 100%);background-image:-o-linear-gradient(left, rgba(0,0,0,0.0001) 0, rgba(0,0,0,0.5) 100%);background-image:-webkit-gradient(linear, left top, right top, color-stop(0, rgba(0,0,0,0.0001)), to(rgba(0,0,0,0.5)));background-image:linear-gradient(to right, rgba(0,0,0,0.0001) 0, rgba(0,0,0,0.5) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1)}.carousel-control:hover,.carousel-control:focus{outline:0;color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-control .icon-prev,.carousel-control .icon-next,.carousel-control .glyphicon-chevron-left,.carousel-control .glyphicon-chevron-right{position:absolute;top:50%;margin-top:-10px;z-index:5;display:inline-block}.carousel-control .icon-prev,.carousel-control .glyphicon-chevron-left{left:50%;margin-left:-10px}.carousel-control .icon-next,.carousel-control .glyphicon-chevron-right{right:50%;margin-right:-10px}.carousel-control .icon-prev,.carousel-control .icon-next{width:20px;height:20px;line-height:1;font-family:serif}.carousel-control .icon-prev:before{content:'\2039'}.carousel-control .icon-next:before{content:'\203a'}.carousel-indicators{position:absolute;bottom:10px;left:50%;z-index:15;width:60%;margin-left:-30%;padding-left:0;list-style:none;text-align:center}.carousel-indicators li{display:inline-block;width:10px;height:10px;margin:1px;text-indent:-999px;border:1px solid #fff;border-radius:10px;cursor:pointer;background-color:#000 \9;background-color:rgba(0,0,0,0)}.carousel-indicators .active{margin:0;width:12px;height:12px;background-color:#fff}.carousel-caption{position:absolute;left:15%;right:15%;bottom:20px;z-index:10;padding-top:20px;padding-bottom:20px;color:#fff;text-align:center;text-shadow:0 1px 2px rgba(0,0,0,0.6)}.carousel-caption .btn{text-shadow:none}@media screen and (min-width:768px){.carousel-control .glyphicon-chevron-left,.carousel-control .glyphicon-chevron-right,.carousel-control .icon-prev,.carousel-control .icon-next{width:30px;height:30px;margin-top:-10px;font-size:30px}.carousel-control .glyphicon-chevron-left,.carousel-control .icon-prev{margin-left:-10px}.carousel-control .glyphicon-chevron-right,.carousel-control .icon-next{margin-right:-10px}.carousel-caption{left:20%;right:20%;padding-bottom:30px}.carousel-indicators{bottom:20px}}.clearfix:before,.clearfix:after,.dl-horizontal dd:before,.dl-horizontal dd:after,.container:before,.container:after,.container-fluid:before,.container-fluid:after,.row:before,.row:after,.form-horizontal .form-group:before,.form-horizontal .form-group:after,.btn-toolbar:before,.btn-toolbar:after,.btn-group-vertical>.btn-group:before,.btn-group-vertical>.btn-group:after,.nav:before,.nav:after,.navbar:before,.navbar:after,.navbar-header:before,.navbar-header:after,.navbar-collapse:before,.navbar-collapse:after,.pager:before,.pager:after,.panel-body:before,.panel-body:after,.modal-header:before,.modal-header:after,.modal-footer:before,.modal-footer:after{content:" ";display:table}.clearfix:after,.dl-horizontal dd:after,.container:after,.container-fluid:after,.row:after,.form-horizontal .form-group:after,.btn-toolbar:after,.btn-group-vertical>.btn-group:after,.nav:after,.navbar:after,.navbar-header:after,.navbar-collapse:after,.pager:after,.panel-body:after,.modal-header:after,.modal-footer:after{clear:both}.center-block{display:block;margin-left:auto;margin-right:auto}.pull-right{float:right !important}.pull-left{float:left !important}.hide{display:none !important}.show{display:block !important}.invisible{visibility:hidden}.text-hide{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.hidden{display:none !important}.affix{position:fixed}@-ms-viewport{width:device-width}.visible-xs,.visible-sm,.visible-md,.visible-lg{display:none !important}.visible-xs-block,.visible-xs-inline,.visible-xs-inline-block,.visible-sm-block,.visible-sm-inline,.visible-sm-inline-block,.visible-md-block,.visible-md-inline,.visible-md-inline-block,.visible-lg-block,.visible-lg-inline,.visible-lg-inline-block{display:none !important}@media (max-width:767px){.visible-xs{display:block !important}table.visible-xs{display:table !important}tr.visible-xs{display:table-row !important}th.visible-xs,td.visible-xs{display:table-cell !important}}@media (max-width:767px){.visible-xs-block{display:block !important}}@media (max-width:767px){.visible-xs-inline{display:inline !important}}@media (max-width:767px){.visible-xs-inline-block{display:inline-block !important}}@media (min-width:768px) and (max-width:991px){.visible-sm{display:block !important}table.visible-sm{display:table !important}tr.visible-sm{display:table-row !important}th.visible-sm,td.visible-sm{display:table-cell !important}}@media (min-width:768px) and (max-width:991px){.visible-sm-block{display:block !important}}@media (min-width:768px) and (max-width:991px){.visible-sm-inline{display:inline !important}}@media (min-width:768px) and (max-width:991px){.visible-sm-inline-block{display:inline-block !important}}@media (min-width:992px) and (max-width:1199px){.visible-md{display:block !important}table.visible-md{display:table !important}tr.visible-md{display:table-row !important}th.visible-md,td.visible-md{display:table-cell !important}}@media (min-width:992px) and (max-width:1199px){.visible-md-block{display:block !important}}@media (min-width:992px) and (max-width:1199px){.visible-md-inline{display:inline !important}}@media (min-width:992px) and (max-width:1199px){.visible-md-inline-block{display:inline-block !important}}@media (min-width:1200px){.visible-lg{display:block !important}table.visible-lg{display:table !important}tr.visible-lg{display:table-row !important}th.visible-lg,td.visible-lg{display:table-cell !important}}@media (min-width:1200px){.visible-lg-block{display:block !important}}@media (min-width:1200px){.visible-lg-inline{display:inline !important}}@media (min-width:1200px){.visible-lg-inline-block{display:inline-block !important}}@media (max-width:767px){.hidden-xs{display:none !important}}@media (min-width:768px) and (max-width:991px){.hidden-sm{display:none !important}}@media (min-width:992px) and (max-width:1199px){.hidden-md{display:none !important}}@media (min-width:1200px){.hidden-lg{display:none !important}}.visible-print{display:none !important}@media print{.visible-print{display:block !important}table.visible-print{display:table !important}tr.visible-print{display:table-row !important}th.visible-print,td.visible-print{display:table-cell !important}}.visible-print-block{display:none !important}@media print{.visible-print-block{display:block !important}}.visible-print-inline{display:none !important}@media print{.visible-print-inline{display:inline !important}}.visible-print-inline-block{display:none !important}@media print{.visible-print-inline-block{display:inline-block !important}}@media print{.hidden-print{display:none !important}} \ No newline at end of file diff --git a/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.js b/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.js new file mode 100644 index 000000000..9bcd2fcca --- /dev/null +++ b/server/src/main/resources/com/cloudera/livy/server/ui/static/bootstrap.min.js @@ -0,0 +1,7 @@ +/*! + * Bootstrap v3.3.7 (http://getbootstrap.com) + * Copyright 2011-2016 Twitter, Inc. + * Licensed under the MIT license + */ +if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";var b=a.fn.jquery.split(" ")[0].split(".");if(b[0]<2&&b[1]<9||1==b[0]&&9==b[1]&&b[2]<1||b[0]>3)throw new Error("Bootstrap's JavaScript requires jQuery version 1.9.1 or higher, but lower than version 4")}(jQuery),+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one("bsTransitionEnd",function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b(),a.support.transition&&(a.event.special.bsTransitionEnd={bindType:a.support.transition.end,delegateType:a.support.transition.end,handle:function(b){if(a(b.target).is(this))return b.handleObj.handler.apply(this,arguments)}})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var c=a(this),e=c.data("bs.alert");e||c.data("bs.alert",e=new d(this)),"string"==typeof b&&e[b].call(c)})}var c='[data-dismiss="alert"]',d=function(b){a(b).on("click",c,this.close)};d.VERSION="3.3.7",d.TRANSITION_DURATION=150,d.prototype.close=function(b){function c(){g.detach().trigger("closed.bs.alert").remove()}var e=a(this),f=e.attr("data-target");f||(f=e.attr("href"),f=f&&f.replace(/.*(?=#[^\s]*$)/,""));var g=a("#"===f?[]:f);b&&b.preventDefault(),g.length||(g=e.closest(".alert")),g.trigger(b=a.Event("close.bs.alert")),b.isDefaultPrevented()||(g.removeClass("in"),a.support.transition&&g.hasClass("fade")?g.one("bsTransitionEnd",c).emulateTransitionEnd(d.TRANSITION_DURATION):c())};var e=a.fn.alert;a.fn.alert=b,a.fn.alert.Constructor=d,a.fn.alert.noConflict=function(){return a.fn.alert=e,this},a(document).on("click.bs.alert.data-api",c,d.prototype.close)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof b&&b;e||d.data("bs.button",e=new c(this,f)),"toggle"==b?e.toggle():b&&e.setState(b)})}var c=function(b,d){this.$element=a(b),this.options=a.extend({},c.DEFAULTS,d),this.isLoading=!1};c.VERSION="3.3.7",c.DEFAULTS={loadingText:"loading..."},c.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",null==f.resetText&&d.data("resetText",d[e]()),setTimeout(a.proxy(function(){d[e](null==f[b]?this.options[b]:f[b]),"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c).prop(c,!0)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).removeAttr(c).prop(c,!1))},this),0)},c.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")?(c.prop("checked")&&(a=!1),b.find(".active").removeClass("active"),this.$element.addClass("active")):"checkbox"==c.prop("type")&&(c.prop("checked")!==this.$element.hasClass("active")&&(a=!1),this.$element.toggleClass("active")),c.prop("checked",this.$element.hasClass("active")),a&&c.trigger("change")}else this.$element.attr("aria-pressed",!this.$element.hasClass("active")),this.$element.toggleClass("active")};var d=a.fn.button;a.fn.button=b,a.fn.button.Constructor=c,a.fn.button.noConflict=function(){return a.fn.button=d,this},a(document).on("click.bs.button.data-api",'[data-toggle^="button"]',function(c){var d=a(c.target).closest(".btn");b.call(d,"toggle"),a(c.target).is('input[type="radio"], input[type="checkbox"]')||(c.preventDefault(),d.is("input,button")?d.trigger("focus"):d.find("input:visible,button:visible").first().trigger("focus"))}).on("focus.bs.button.data-api blur.bs.button.data-api",'[data-toggle^="button"]',function(b){a(b.target).closest(".btn").toggleClass("focus",/^focus(in)?$/.test(b.type))})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},c.DEFAULTS,d.data(),"object"==typeof b&&b),g="string"==typeof b?b:f.slide;e||d.data("bs.carousel",e=new c(this,f)),"number"==typeof b?e.to(b):g?e[g]():f.interval&&e.pause().cycle()})}var c=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=null,this.sliding=null,this.interval=null,this.$active=null,this.$items=null,this.options.keyboard&&this.$element.on("keydown.bs.carousel",a.proxy(this.keydown,this)),"hover"==this.options.pause&&!("ontouchstart"in document.documentElement)&&this.$element.on("mouseenter.bs.carousel",a.proxy(this.pause,this)).on("mouseleave.bs.carousel",a.proxy(this.cycle,this))};c.VERSION="3.3.7",c.TRANSITION_DURATION=600,c.DEFAULTS={interval:5e3,pause:"hover",wrap:!0,keyboard:!0},c.prototype.keydown=function(a){if(!/input|textarea/i.test(a.target.tagName)){switch(a.which){case 37:this.prev();break;case 39:this.next();break;default:return}a.preventDefault()}},c.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},c.prototype.getItemIndex=function(a){return this.$items=a.parent().children(".item"),this.$items.index(a||this.$active)},c.prototype.getItemForDirection=function(a,b){var c=this.getItemIndex(b),d="prev"==a&&0===c||"next"==a&&c==this.$items.length-1;if(d&&!this.options.wrap)return b;var e="prev"==a?-1:1,f=(c+e)%this.$items.length;return this.$items.eq(f)},c.prototype.to=function(a){var b=this,c=this.getItemIndex(this.$active=this.$element.find(".item.active"));if(!(a>this.$items.length-1||a<0))return this.sliding?this.$element.one("slid.bs.carousel",function(){b.to(a)}):c==a?this.pause().cycle():this.slide(a>c?"next":"prev",this.$items.eq(a))},c.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},c.prototype.next=function(){if(!this.sliding)return this.slide("next")},c.prototype.prev=function(){if(!this.sliding)return this.slide("prev")},c.prototype.slide=function(b,d){var e=this.$element.find(".item.active"),f=d||this.getItemForDirection(b,e),g=this.interval,h="next"==b?"left":"right",i=this;if(f.hasClass("active"))return this.sliding=!1;var j=f[0],k=a.Event("slide.bs.carousel",{relatedTarget:j,direction:h});if(this.$element.trigger(k),!k.isDefaultPrevented()){if(this.sliding=!0,g&&this.pause(),this.$indicators.length){this.$indicators.find(".active").removeClass("active");var l=a(this.$indicators.children()[this.getItemIndex(f)]);l&&l.addClass("active")}var m=a.Event("slid.bs.carousel",{relatedTarget:j,direction:h});return a.support.transition&&this.$element.hasClass("slide")?(f.addClass(b),f[0].offsetWidth,e.addClass(h),f.addClass(h),e.one("bsTransitionEnd",function(){f.removeClass([b,h].join(" ")).addClass("active"),e.removeClass(["active",h].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger(m)},0)}).emulateTransitionEnd(c.TRANSITION_DURATION)):(e.removeClass("active"),f.addClass("active"),this.sliding=!1,this.$element.trigger(m)),g&&this.cycle(),this}};var d=a.fn.carousel;a.fn.carousel=b,a.fn.carousel.Constructor=c,a.fn.carousel.noConflict=function(){return a.fn.carousel=d,this};var e=function(c){var d,e=a(this),f=a(e.attr("data-target")||(d=e.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""));if(f.hasClass("carousel")){var g=a.extend({},f.data(),e.data()),h=e.attr("data-slide-to");h&&(g.interval=!1),b.call(f,g),h&&f.data("bs.carousel").to(h),c.preventDefault()}};a(document).on("click.bs.carousel.data-api","[data-slide]",e).on("click.bs.carousel.data-api","[data-slide-to]",e),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var c=a(this);b.call(c,c.data())})})}(jQuery),+function(a){"use strict";function b(b){var c,d=b.attr("data-target")||(c=b.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"");return a(d)}function c(b){return this.each(function(){var c=a(this),e=c.data("bs.collapse"),f=a.extend({},d.DEFAULTS,c.data(),"object"==typeof b&&b);!e&&f.toggle&&/show|hide/.test(b)&&(f.toggle=!1),e||c.data("bs.collapse",e=new d(this,f)),"string"==typeof b&&e[b]()})}var d=function(b,c){this.$element=a(b),this.options=a.extend({},d.DEFAULTS,c),this.$trigger=a('[data-toggle="collapse"][href="#'+b.id+'"],[data-toggle="collapse"][data-target="#'+b.id+'"]'),this.transitioning=null,this.options.parent?this.$parent=this.getParent():this.addAriaAndCollapsedClass(this.$element,this.$trigger),this.options.toggle&&this.toggle()};d.VERSION="3.3.7",d.TRANSITION_DURATION=350,d.DEFAULTS={toggle:!0},d.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},d.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b,e=this.$parent&&this.$parent.children(".panel").children(".in, .collapsing");if(!(e&&e.length&&(b=e.data("bs.collapse"),b&&b.transitioning))){var f=a.Event("show.bs.collapse");if(this.$element.trigger(f),!f.isDefaultPrevented()){e&&e.length&&(c.call(e,"hide"),b||e.data("bs.collapse",null));var g=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[g](0).attr("aria-expanded",!0),this.$trigger.removeClass("collapsed").attr("aria-expanded",!0),this.transitioning=1;var h=function(){this.$element.removeClass("collapsing").addClass("collapse in")[g](""),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return h.call(this);var i=a.camelCase(["scroll",g].join("-"));this.$element.one("bsTransitionEnd",a.proxy(h,this)).emulateTransitionEnd(d.TRANSITION_DURATION)[g](this.$element[0][i])}}}},d.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse in").attr("aria-expanded",!1),this.$trigger.addClass("collapsed").attr("aria-expanded",!1),this.transitioning=1;var e=function(){this.transitioning=0,this.$element.removeClass("collapsing").addClass("collapse").trigger("hidden.bs.collapse")};return a.support.transition?void this.$element[c](0).one("bsTransitionEnd",a.proxy(e,this)).emulateTransitionEnd(d.TRANSITION_DURATION):e.call(this)}}},d.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()},d.prototype.getParent=function(){return a(this.options.parent).find('[data-toggle="collapse"][data-parent="'+this.options.parent+'"]').each(a.proxy(function(c,d){var e=a(d);this.addAriaAndCollapsedClass(b(e),e)},this)).end()},d.prototype.addAriaAndCollapsedClass=function(a,b){var c=a.hasClass("in");a.attr("aria-expanded",c),b.toggleClass("collapsed",!c).attr("aria-expanded",c)};var e=a.fn.collapse;a.fn.collapse=c,a.fn.collapse.Constructor=d,a.fn.collapse.noConflict=function(){return a.fn.collapse=e,this},a(document).on("click.bs.collapse.data-api",'[data-toggle="collapse"]',function(d){var e=a(this);e.attr("data-target")||d.preventDefault();var f=b(e),g=f.data("bs.collapse"),h=g?"toggle":e.data();c.call(f,h)})}(jQuery),+function(a){"use strict";function b(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}function c(c){c&&3===c.which||(a(e).remove(),a(f).each(function(){var d=a(this),e=b(d),f={relatedTarget:this};e.hasClass("open")&&(c&&"click"==c.type&&/input|textarea/i.test(c.target.tagName)&&a.contains(e[0],c.target)||(e.trigger(c=a.Event("hide.bs.dropdown",f)),c.isDefaultPrevented()||(d.attr("aria-expanded","false"),e.removeClass("open").trigger(a.Event("hidden.bs.dropdown",f)))))}))}function d(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new g(this)),"string"==typeof b&&d[b].call(c)})}var e=".dropdown-backdrop",f='[data-toggle="dropdown"]',g=function(b){a(b).on("click.bs.dropdown",this.toggle)};g.VERSION="3.3.7",g.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=b(e),g=f.hasClass("open");if(c(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a(document.createElement("div")).addClass("dropdown-backdrop").insertAfter(a(this)).on("click",c);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;e.trigger("focus").attr("aria-expanded","true"),f.toggleClass("open").trigger(a.Event("shown.bs.dropdown",h))}return!1}},g.prototype.keydown=function(c){if(/(38|40|27|32)/.test(c.which)&&!/input|textarea/i.test(c.target.tagName)){var d=a(this);if(c.preventDefault(),c.stopPropagation(),!d.is(".disabled, :disabled")){var e=b(d),g=e.hasClass("open");if(!g&&27!=c.which||g&&27==c.which)return 27==c.which&&e.find(f).trigger("focus"),d.trigger("click");var h=" li:not(.disabled):visible a",i=e.find(".dropdown-menu"+h);if(i.length){var j=i.index(c.target);38==c.which&&j>0&&j--,40==c.which&&jdocument.documentElement.clientHeight;this.$element.css({paddingLeft:!this.bodyIsOverflowing&&a?this.scrollbarWidth:"",paddingRight:this.bodyIsOverflowing&&!a?this.scrollbarWidth:""})},c.prototype.resetAdjustments=function(){this.$element.css({paddingLeft:"",paddingRight:""})},c.prototype.checkScrollbar=function(){var a=window.innerWidth;if(!a){var b=document.documentElement.getBoundingClientRect();a=b.right-Math.abs(b.left)}this.bodyIsOverflowing=document.body.clientWidth
',trigger:"hover focus",title:"",delay:0,html:!1,container:!1,viewport:{selector:"body",padding:0}},c.prototype.init=function(b,c,d){if(this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d),this.$viewport=this.options.viewport&&a(a.isFunction(this.options.viewport)?this.options.viewport.call(this,this.$element):this.options.viewport.selector||this.options.viewport),this.inState={click:!1,hover:!1,focus:!1},this.$element[0]instanceof document.constructor&&!this.options.selector)throw new Error("`selector` option must be specified when initializing "+this.type+" on the window.document object!");for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},c.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},c.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusin"==b.type?"focus":"hover"]=!0),c.tip().hasClass("in")||"in"==c.hoverState?void(c.hoverState="in"):(clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show())},c.prototype.isInStateTrue=function(){for(var a in this.inState)if(this.inState[a])return!0;return!1},c.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);if(c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusout"==b.type?"focus":"hover"]=!1),!c.isInStateTrue())return clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide()},c.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){this.$element.trigger(b);var d=a.contains(this.$element[0].ownerDocument.documentElement,this.$element[0]);if(b.isDefaultPrevented()||!d)return;var e=this,f=this.tip(),g=this.getUID(this.type);this.setContent(),f.attr("id",g),this.$element.attr("aria-describedby",g),this.options.animation&&f.addClass("fade");var h="function"==typeof this.options.placement?this.options.placement.call(this,f[0],this.$element[0]):this.options.placement,i=/\s?auto?\s?/i,j=i.test(h);j&&(h=h.replace(i,"")||"top"),f.detach().css({top:0,left:0,display:"block"}).addClass(h).data("bs."+this.type,this),this.options.container?f.appendTo(this.options.container):f.insertAfter(this.$element),this.$element.trigger("inserted.bs."+this.type);var k=this.getPosition(),l=f[0].offsetWidth,m=f[0].offsetHeight;if(j){var n=h,o=this.getPosition(this.$viewport);h="bottom"==h&&k.bottom+m>o.bottom?"top":"top"==h&&k.top-mo.width?"left":"left"==h&&k.left-lg.top+g.height&&(e.top=g.top+g.height-i)}else{var j=b.left-f,k=b.left+f+c;jg.right&&(e.left=g.left+g.width-k)}return e},c.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},c.prototype.getUID=function(a){do a+=~~(1e6*Math.random());while(document.getElementById(a));return a},c.prototype.tip=function(){if(!this.$tip&&(this.$tip=a(this.options.template),1!=this.$tip.length))throw new Error(this.type+" `template` option must consist of exactly 1 top-level element!");return this.$tip},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},c.prototype.enable=function(){this.enabled=!0},c.prototype.disable=function(){this.enabled=!1},c.prototype.toggleEnabled=function(){this.enabled=!this.enabled},c.prototype.toggle=function(b){var c=this;b&&(c=a(b.currentTarget).data("bs."+this.type),c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c))),b?(c.inState.click=!c.inState.click,c.isInStateTrue()?c.enter(c):c.leave(c)):c.tip().hasClass("in")?c.leave(c):c.enter(c)},c.prototype.destroy=function(){var a=this;clearTimeout(this.timeout),this.hide(function(){a.$element.off("."+a.type).removeData("bs."+a.type),a.$tip&&a.$tip.detach(),a.$tip=null,a.$arrow=null,a.$viewport=null,a.$element=null})};var d=a.fn.tooltip;a.fn.tooltip=b,a.fn.tooltip.Constructor=c,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=d,this}}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof b&&b;!e&&/destroy|hide/.test(b)||(e||d.data("bs.popover",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");c.VERSION="3.3.7",c.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:''}),c.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),c.prototype.constructor=c,c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content").children().detach().end()[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},c.prototype.hasContent=function(){return this.getTitle()||this.getContent()},c.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")};var d=a.fn.popover;a.fn.popover=b,a.fn.popover.Constructor=c,a.fn.popover.noConflict=function(){return a.fn.popover=d,this}}(jQuery),+function(a){"use strict";function b(c,d){this.$body=a(document.body),this.$scrollElement=a(a(c).is(document.body)?window:c),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||"")+" .nav li > a",this.offsets=[],this.targets=[],this.activeTarget=null,this.scrollHeight=0,this.$scrollElement.on("scroll.bs.scrollspy",a.proxy(this.process,this)),this.refresh(),this.process()}function c(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})}b.VERSION="3.3.7",b.DEFAULTS={offset:10},b.prototype.getScrollHeight=function(){return this.$scrollElement[0].scrollHeight||Math.max(this.$body[0].scrollHeight,document.documentElement.scrollHeight)},b.prototype.refresh=function(){var b=this,c="offset",d=0;this.offsets=[],this.targets=[],this.scrollHeight=this.getScrollHeight(),a.isWindow(this.$scrollElement[0])||(c="position",d=this.$scrollElement.scrollTop()),this.$body.find(this.selector).map(function(){var b=a(this),e=b.data("target")||b.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.length&&f.is(":visible")&&[[f[c]().top+d,e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){b.offsets.push(this[0]),b.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.getScrollHeight(),d=this.options.offset+c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(this.scrollHeight!=c&&this.refresh(),b>=d)return g!=(a=f[f.length-1])&&this.activate(a);if(g&&b=e[a]&&(void 0===e[a+1]||b .dropdown-menu > .active").removeClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!1),b.addClass("active").find('[data-toggle="tab"]').attr("aria-expanded",!0),h?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu").length&&b.closest("li.dropdown").addClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!0),e&&e()}var g=d.find("> .active"),h=e&&a.support.transition&&(g.length&&g.hasClass("fade")||!!d.find("> .fade").length);g.length&&h?g.one("bsTransitionEnd",f).emulateTransitionEnd(c.TRANSITION_DURATION):f(),g.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this};var e=function(c){c.preventDefault(),b.call(a(this),"show")};a(document).on("click.bs.tab.data-api",'[data-toggle="tab"]',e).on("click.bs.tab.data-api",'[data-toggle="pill"]',e)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=null,this.unpin=null,this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.3.7",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getState=function(a,b,c,d){var e=this.$target.scrollTop(),f=this.$element.offset(),g=this.$target.height();if(null!=c&&"top"==this.affixed)return e=a-d&&"bottom"},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=this.$element.height(),d=this.options.offset,e=d.top,f=d.bottom,g=Math.max(a(document).height(),a(document.body).height());"object"!=typeof d&&(f=e=d),"function"==typeof e&&(e=d.top(this.$element)),"function"==typeof f&&(f=d.bottom(this.$element));var h=this.getState(g,b,e,f);if(this.affixed!=h){null!=this.unpin&&this.$element.css("top","");var i="affix"+(h?"-"+h:""),j=a.Event(i+".bs.affix");if(this.$element.trigger(j),j.isDefaultPrevented())return;this.affixed=h,this.unpin="bottom"==h?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(i).trigger(i.replace("affix","affixed")+".bs.affix")}"bottom"==h&&this.$element.offset({top:g-b-f})}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},null!=d.offsetBottom&&(d.offset.bottom=d.offsetBottom),null!=d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery); \ No newline at end of file diff --git a/server/src/main/resources/com/cloudera/livy/server/ui/static/jquery-3.2.1.min.js b/server/src/main/resources/com/cloudera/livy/server/ui/static/jquery-3.2.1.min.js new file mode 100644 index 000000000..644d35e27 --- /dev/null +++ b/server/src/main/resources/com/cloudera/livy/server/ui/static/jquery-3.2.1.min.js @@ -0,0 +1,4 @@ +/*! jQuery v3.2.1 | (c) JS Foundation and other contributors | jquery.org/license */ +!function(a,b){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){"use strict";var c=[],d=a.document,e=Object.getPrototypeOf,f=c.slice,g=c.concat,h=c.push,i=c.indexOf,j={},k=j.toString,l=j.hasOwnProperty,m=l.toString,n=m.call(Object),o={};function p(a,b){b=b||d;var c=b.createElement("script");c.text=a,b.head.appendChild(c).parentNode.removeChild(c)}var q="3.2.1",r=function(a,b){return new r.fn.init(a,b)},s=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,t=/^-ms-/,u=/-([a-z])/g,v=function(a,b){return b.toUpperCase()};r.fn=r.prototype={jquery:q,constructor:r,length:0,toArray:function(){return f.call(this)},get:function(a){return null==a?f.call(this):a<0?this[a+this.length]:this[a]},pushStack:function(a){var b=r.merge(this.constructor(),a);return b.prevObject=this,b},each:function(a){return r.each(this,a)},map:function(a){return this.pushStack(r.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(f.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(a<0?b:0);return this.pushStack(c>=0&&c0&&b-1 in a)}var x=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ha(),z=ha(),A=ha(),B=function(a,b){return a===b&&(l=!0),0},C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=function(a,b){for(var c=0,d=a.length;c+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(N),U=new RegExp("^"+L+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L+"|[*])"),ATTR:new RegExp("^"+M),PSEUDO:new RegExp("^"+N),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),aa=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:d<0?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},ba=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ca=function(a,b){return b?"\0"===a?"\ufffd":a.slice(0,-1)+"\\"+a.charCodeAt(a.length-1).toString(16)+" ":"\\"+a},da=function(){m()},ea=ta(function(a){return a.disabled===!0&&("form"in a||"label"in a)},{dir:"parentNode",next:"legend"});try{G.apply(D=H.call(v.childNodes),v.childNodes),D[v.childNodes.length].nodeType}catch(fa){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function ga(a,b,d,e){var f,h,j,k,l,o,r,s=b&&b.ownerDocument,w=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==w&&9!==w&&11!==w)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==w&&(l=Z.exec(a)))if(f=l[1]){if(9===w){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(s&&(j=s.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(l[2])return G.apply(d,b.getElementsByTagName(a)),d;if((f=l[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==w)s=b,r=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(ba,ca):b.setAttribute("id",k=u),o=g(a),h=o.length;while(h--)o[h]="#"+k+" "+sa(o[h]);r=o.join(","),s=$.test(a)&&qa(b.parentNode)||b}if(r)try{return G.apply(d,s.querySelectorAll(r)),d}catch(x){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(P,"$1"),b,d,e)}function ha(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ia(a){return a[u]=!0,a}function ja(a){var b=n.createElement("fieldset");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ka(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function la(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&a.sourceIndex-b.sourceIndex;if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function na(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function oa(a){return function(b){return"form"in b?b.parentNode&&b.disabled===!1?"label"in b?"label"in b.parentNode?b.parentNode.disabled===a:b.disabled===a:b.isDisabled===a||b.isDisabled!==!a&&ea(b)===a:b.disabled===a:"label"in b&&b.disabled===a}}function pa(a){return ia(function(b){return b=+b,ia(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function qa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=ga.support={},f=ga.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return!!b&&"HTML"!==b.nodeName},m=ga.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),v!==n&&(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ja(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ja(function(a){return a.appendChild(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Y.test(n.getElementsByClassName),c.getById=ja(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){return a.getAttribute("id")===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}}):(d.filter.ID=function(a){var b=a.replace(_,aa);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}},d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c,d,e,f=b.getElementById(a);if(f){if(c=f.getAttributeNode("id"),c&&c.value===a)return[f];e=b.getElementsByName(a),d=0;while(f=e[d++])if(c=f.getAttributeNode("id"),c&&c.value===a)return[f]}return[]}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){if("undefined"!=typeof b.getElementsByClassName&&p)return b.getElementsByClassName(a)},r=[],q=[],(c.qsa=Y.test(n.querySelectorAll))&&(ja(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ja(function(a){a.innerHTML="";var b=n.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+K+"*[*^$|!~]?="),2!==a.querySelectorAll(":enabled").length&&q.push(":enabled",":disabled"),o.appendChild(a).disabled=!0,2!==a.querySelectorAll(":disabled").length&&q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Y.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ja(function(a){c.disconnectedMatch=s.call(a,"*"),s.call(a,"[s!='']:x"),r.push("!=",N)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Y.test(o.compareDocumentPosition),t=b||Y.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?I(k,a)-I(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?I(k,a)-I(k,b):0;if(e===f)return la(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?la(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},ga.matches=function(a,b){return ga(a,null,null,b)},ga.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(S,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return ga(b,n,null,[a]).length>0},ga.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},ga.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},ga.escape=function(a){return(a+"").replace(ba,ca)},ga.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},ga.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=ga.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=ga.selectors={cacheLength:50,createPseudo:ia,match:V,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(_,aa),a[3]=(a[3]||a[4]||a[5]||"").replace(_,aa),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||ga.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&ga.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return V.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&T.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(_,aa).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=ga.attr(d,a);return null==e?"!="===b:!b||(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(O," ")+" ").indexOf(c)>-1:"|="===b&&(e===c||e.slice(0,c.length+1)===c+"-"))}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||ga.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ia(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ia(function(a){var b=[],c=[],d=h(a.replace(P,"$1"));return d[u]?ia(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ia(function(a){return function(b){return ga(a,b).length>0}}),contains:ia(function(a){return a=a.replace(_,aa),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ia(function(a){return U.test(a||"")||ga.error("unsupported lang: "+a),a=a.replace(_,aa).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:oa(!1),disabled:oa(!0),checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:pa(function(){return[0]}),last:pa(function(a,b){return[b-1]}),eq:pa(function(a,b,c){return[c<0?c+b:c]}),even:pa(function(a,b){for(var c=0;c=0;)a.push(d);return a}),gt:pa(function(a,b,c){for(var d=c<0?c+b:c;++d1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function va(a,b,c){for(var d=0,e=b.length;d-1&&(f[j]=!(g[j]=l))}}else r=wa(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ya(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ta(function(a){return a===b},h,!0),l=ta(function(a){return I(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];i1&&ua(m),i>1&&sa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(P,"$1"),c,i0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=E.call(i));u=wa(u)}G.apply(i,u),k&&!f&&u.length>0&&r+b.length>1&&ga.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ia(f):f}return h=ga.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=ya(b[c]),f[u]?d.push(f):e.push(f);f=A(a,za(e,d)),f.selector=a}return f},i=ga.select=function(a,b,c,e){var f,i,j,k,l,m="function"==typeof a&&a,n=!e&&g(a=m.selector||a);if(c=c||[],1===n.length){if(i=n[0]=n[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&9===b.nodeType&&p&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(_,aa),b)||[])[0],!b)return c;m&&(b=b.parentNode),a=a.slice(i.shift().value.length)}f=V.needsContext.test(a)?0:i.length;while(f--){if(j=i[f],d.relative[k=j.type])break;if((l=d.find[k])&&(e=l(j.matches[0].replace(_,aa),$.test(i[0].type)&&qa(b.parentNode)||b))){if(i.splice(f,1),a=e.length&&sa(i),!a)return G.apply(c,e),c;break}}}return(m||h(a,n))(e,b,!p,c,!b||$.test(a)&&qa(b.parentNode)||b),c},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ja(function(a){return 1&a.compareDocumentPosition(n.createElement("fieldset"))}),ja(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||ka("type|href|height|width",function(a,b,c){if(!c)return a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ja(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ka("value",function(a,b,c){if(!c&&"input"===a.nodeName.toLowerCase())return a.defaultValue}),ja(function(a){return null==a.getAttribute("disabled")})||ka(J,function(a,b,c){var d;if(!c)return a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),ga}(a);r.find=x,r.expr=x.selectors,r.expr[":"]=r.expr.pseudos,r.uniqueSort=r.unique=x.uniqueSort,r.text=x.getText,r.isXMLDoc=x.isXML,r.contains=x.contains,r.escapeSelector=x.escape;var y=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&r(a).is(c))break;d.push(a)}return d},z=function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},A=r.expr.match.needsContext;function B(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()}var C=/^<([a-z][^\/\0>:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i,D=/^.[^:#\[\.,]*$/;function E(a,b,c){return r.isFunction(b)?r.grep(a,function(a,d){return!!b.call(a,d,a)!==c}):b.nodeType?r.grep(a,function(a){return a===b!==c}):"string"!=typeof b?r.grep(a,function(a){return i.call(b,a)>-1!==c}):D.test(b)?r.filter(b,a,c):(b=r.filter(b,a),r.grep(a,function(a){return i.call(b,a)>-1!==c&&1===a.nodeType}))}r.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?r.find.matchesSelector(d,a)?[d]:[]:r.find.matches(a,r.grep(b,function(a){return 1===a.nodeType}))},r.fn.extend({find:function(a){var b,c,d=this.length,e=this;if("string"!=typeof a)return this.pushStack(r(a).filter(function(){for(b=0;b1?r.uniqueSort(c):c},filter:function(a){return this.pushStack(E(this,a||[],!1))},not:function(a){return this.pushStack(E(this,a||[],!0))},is:function(a){return!!E(this,"string"==typeof a&&A.test(a)?r(a):a||[],!1).length}});var F,G=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]+))$/,H=r.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||F,"string"==typeof a){if(e="<"===a[0]&&">"===a[a.length-1]&&a.length>=3?[null,a,null]:G.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof r?b[0]:b,r.merge(this,r.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),C.test(e[1])&&r.isPlainObject(b))for(e in b)r.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}return f=d.getElementById(e[2]),f&&(this[0]=f,this.length=1),this}return a.nodeType?(this[0]=a,this.length=1,this):r.isFunction(a)?void 0!==c.ready?c.ready(a):a(r):r.makeArray(a,this)};H.prototype=r.fn,F=r(d);var I=/^(?:parents|prev(?:Until|All))/,J={children:!0,contents:!0,next:!0,prev:!0};r.fn.extend({has:function(a){var b=r(a,this),c=b.length;return this.filter(function(){for(var a=0;a-1:1===c.nodeType&&r.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?r.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?i.call(r(a),this[0]):i.call(this,a.jquery?a[0]:a):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(r.uniqueSort(r.merge(this.get(),r(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function K(a,b){while((a=a[b])&&1!==a.nodeType);return a}r.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return y(a,"parentNode")},parentsUntil:function(a,b,c){return y(a,"parentNode",c)},next:function(a){return K(a,"nextSibling")},prev:function(a){return K(a,"previousSibling")},nextAll:function(a){return y(a,"nextSibling")},prevAll:function(a){return y(a,"previousSibling")},nextUntil:function(a,b,c){return y(a,"nextSibling",c)},prevUntil:function(a,b,c){return y(a,"previousSibling",c)},siblings:function(a){return z((a.parentNode||{}).firstChild,a)},children:function(a){return z(a.firstChild)},contents:function(a){return B(a,"iframe")?a.contentDocument:(B(a,"template")&&(a=a.content||a),r.merge([],a.childNodes))}},function(a,b){r.fn[a]=function(c,d){var e=r.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=r.filter(d,e)),this.length>1&&(J[a]||r.uniqueSort(e),I.test(a)&&e.reverse()),this.pushStack(e)}});var L=/[^\x20\t\r\n\f]+/g;function M(a){var b={};return r.each(a.match(L)||[],function(a,c){b[c]=!0}),b}r.Callbacks=function(a){a="string"==typeof a?M(a):r.extend({},a);var b,c,d,e,f=[],g=[],h=-1,i=function(){for(e=e||a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h-1)f.splice(c,1),c<=h&&h--}),this},has:function(a){return a?r.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=g=[],c||b||(f=c=""),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j};function N(a){return a}function O(a){throw a}function P(a,b,c,d){var e;try{a&&r.isFunction(e=a.promise)?e.call(a).done(b).fail(c):a&&r.isFunction(e=a.then)?e.call(a,b,c):b.apply(void 0,[a].slice(d))}catch(a){c.apply(void 0,[a])}}r.extend({Deferred:function(b){var c=[["notify","progress",r.Callbacks("memory"),r.Callbacks("memory"),2],["resolve","done",r.Callbacks("once memory"),r.Callbacks("once memory"),0,"resolved"],["reject","fail",r.Callbacks("once memory"),r.Callbacks("once memory"),1,"rejected"]],d="pending",e={state:function(){return d},always:function(){return f.done(arguments).fail(arguments),this},"catch":function(a){return e.then(null,a)},pipe:function(){var a=arguments;return r.Deferred(function(b){r.each(c,function(c,d){var e=r.isFunction(a[d[4]])&&a[d[4]];f[d[1]](function(){var a=e&&e.apply(this,arguments);a&&r.isFunction(a.promise)?a.promise().progress(b.notify).done(b.resolve).fail(b.reject):b[d[0]+"With"](this,e?[a]:arguments)})}),a=null}).promise()},then:function(b,d,e){var f=0;function g(b,c,d,e){return function(){var h=this,i=arguments,j=function(){var a,j;if(!(b=f&&(d!==O&&(h=void 0,i=[a]),c.rejectWith(h,i))}};b?k():(r.Deferred.getStackHook&&(k.stackTrace=r.Deferred.getStackHook()),a.setTimeout(k))}}return r.Deferred(function(a){c[0][3].add(g(0,a,r.isFunction(e)?e:N,a.notifyWith)),c[1][3].add(g(0,a,r.isFunction(b)?b:N)),c[2][3].add(g(0,a,r.isFunction(d)?d:O))}).promise()},promise:function(a){return null!=a?r.extend(a,e):e}},f={};return r.each(c,function(a,b){var g=b[2],h=b[5];e[b[1]]=g.add,h&&g.add(function(){d=h},c[3-a][2].disable,c[0][2].lock),g.add(b[3].fire),f[b[0]]=function(){return f[b[0]+"With"](this===f?void 0:this,arguments),this},f[b[0]+"With"]=g.fireWith}),e.promise(f),b&&b.call(f,f),f},when:function(a){var b=arguments.length,c=b,d=Array(c),e=f.call(arguments),g=r.Deferred(),h=function(a){return function(c){d[a]=this,e[a]=arguments.length>1?f.call(arguments):c,--b||g.resolveWith(d,e)}};if(b<=1&&(P(a,g.done(h(c)).resolve,g.reject,!b),"pending"===g.state()||r.isFunction(e[c]&&e[c].then)))return g.then();while(c--)P(e[c],h(c),g.reject);return g.promise()}});var Q=/^(Eval|Internal|Range|Reference|Syntax|Type|URI)Error$/;r.Deferred.exceptionHook=function(b,c){a.console&&a.console.warn&&b&&Q.test(b.name)&&a.console.warn("jQuery.Deferred exception: "+b.message,b.stack,c)},r.readyException=function(b){a.setTimeout(function(){throw b})};var R=r.Deferred();r.fn.ready=function(a){return R.then(a)["catch"](function(a){r.readyException(a)}),this},r.extend({isReady:!1,readyWait:1,ready:function(a){(a===!0?--r.readyWait:r.isReady)||(r.isReady=!0,a!==!0&&--r.readyWait>0||R.resolveWith(d,[r]))}}),r.ready.then=R.then;function S(){d.removeEventListener("DOMContentLoaded",S), +a.removeEventListener("load",S),r.ready()}"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll?a.setTimeout(r.ready):(d.addEventListener("DOMContentLoaded",S),a.addEventListener("load",S));var T=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===r.type(c)){e=!0;for(h in c)T(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,r.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(r(a),c)})),b))for(;h1,null,!0)},removeData:function(a){return this.each(function(){X.remove(this,a)})}}),r.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=W.get(a,b),c&&(!d||Array.isArray(c)?d=W.access(a,b,r.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=r.queue(a,b),d=c.length,e=c.shift(),f=r._queueHooks(a,b),g=function(){r.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return W.get(a,c)||W.access(a,c,{empty:r.Callbacks("once memory").add(function(){W.remove(a,[b+"queue",c])})})}}),r.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.length\x20\t\r\n\f]+)/i,la=/^$|\/(?:java|ecma)script/i,ma={option:[1,""],thead:[1,"","
"],col:[2,"","
"],tr:[2,"","
"],td:[3,"","
"],_default:[0,"",""]};ma.optgroup=ma.option,ma.tbody=ma.tfoot=ma.colgroup=ma.caption=ma.thead,ma.th=ma.td;function na(a,b){var c;return c="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):[],void 0===b||b&&B(a,b)?r.merge([a],c):c}function oa(a,b){for(var c=0,d=a.length;c-1)e&&e.push(f);else if(j=r.contains(f.ownerDocument,f),g=na(l.appendChild(f),"script"),j&&oa(g),c){k=0;while(f=g[k++])la.test(f.type||"")&&c.push(f)}return l}!function(){var a=d.createDocumentFragment(),b=a.appendChild(d.createElement("div")),c=d.createElement("input");c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),b.appendChild(c),o.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,b.innerHTML="",o.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue}();var ra=d.documentElement,sa=/^key/,ta=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,ua=/^([^.]*)(?:\.(.+)|)/;function va(){return!0}function wa(){return!1}function xa(){try{return d.activeElement}catch(a){}}function ya(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)ya(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=wa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return r().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=r.guid++)),a.each(function(){r.event.add(this,b,e,d,c)})}r.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.get(a);if(q){c.handler&&(f=c,c=f.handler,e=f.selector),e&&r.find.matchesSelector(ra,e),c.guid||(c.guid=r.guid++),(i=q.events)||(i=q.events={}),(g=q.handle)||(g=q.handle=function(b){return"undefined"!=typeof r&&r.event.triggered!==b.type?r.event.dispatch.apply(a,arguments):void 0}),b=(b||"").match(L)||[""],j=b.length;while(j--)h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n&&(l=r.event.special[n]||{},n=(e?l.delegateType:l.bindType)||n,l=r.event.special[n]||{},k=r.extend({type:n,origType:p,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&r.expr.match.needsContext.test(e),namespace:o.join(".")},f),(m=i[n])||(m=i[n]=[],m.delegateCount=0,l.setup&&l.setup.call(a,d,o,g)!==!1||a.addEventListener&&a.addEventListener(n,g)),l.add&&(l.add.call(a,k),k.handler.guid||(k.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,k):m.push(k),r.event.global[n]=!0)}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,p,q=W.hasData(a)&&W.get(a);if(q&&(i=q.events)){b=(b||"").match(L)||[""],j=b.length;while(j--)if(h=ua.exec(b[j])||[],n=p=h[1],o=(h[2]||"").split(".").sort(),n){l=r.event.special[n]||{},n=(d?l.delegateType:l.bindType)||n,m=i[n]||[],h=h[2]&&new RegExp("(^|\\.)"+o.join("\\.(?:.*\\.|)")+"(\\.|$)"),g=f=m.length;while(f--)k=m[f],!e&&p!==k.origType||c&&c.guid!==k.guid||h&&!h.test(k.namespace)||d&&d!==k.selector&&("**"!==d||!k.selector)||(m.splice(f,1),k.selector&&m.delegateCount--,l.remove&&l.remove.call(a,k));g&&!m.length&&(l.teardown&&l.teardown.call(a,o,q.handle)!==!1||r.removeEvent(a,n,q.handle),delete i[n])}else for(n in i)r.event.remove(a,n+b[j],c,d,!0);r.isEmptyObject(i)&&W.remove(a,"handle events")}},dispatch:function(a){var b=r.event.fix(a),c,d,e,f,g,h,i=new Array(arguments.length),j=(W.get(this,"events")||{})[b.type]||[],k=r.event.special[b.type]||{};for(i[0]=b,c=1;c=1))for(;j!==this;j=j.parentNode||this)if(1===j.nodeType&&("click"!==a.type||j.disabled!==!0)){for(f=[],g={},c=0;c-1:r.find(e,this,null,[j]).length),g[e]&&f.push(d);f.length&&h.push({elem:j,handlers:f})}return j=this,i\x20\t\r\n\f]*)[^>]*)\/>/gi,Aa=/\s*$/g;function Ea(a,b){return B(a,"table")&&B(11!==b.nodeType?b:b.firstChild,"tr")?r(">tbody",a)[0]||a:a}function Fa(a){return a.type=(null!==a.getAttribute("type"))+"/"+a.type,a}function Ga(a){var b=Ca.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Ha(a,b){var c,d,e,f,g,h,i,j;if(1===b.nodeType){if(W.hasData(a)&&(f=W.access(a),g=W.set(b,f),j=f.events)){delete g.handle,g.events={};for(e in j)for(c=0,d=j[e].length;c1&&"string"==typeof q&&!o.checkClone&&Ba.test(q))return a.each(function(e){var f=a.eq(e);s&&(b[0]=q.call(this,e,f.html())),Ja(f,b,c,d)});if(m&&(e=qa(b,a[0].ownerDocument,!1,a,d),f=e.firstChild,1===e.childNodes.length&&(e=f),f||d)){for(h=r.map(na(e,"script"),Fa),i=h.length;l")},clone:function(a,b,c){var d,e,f,g,h=a.cloneNode(!0),i=r.contains(a.ownerDocument,a);if(!(o.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||r.isXMLDoc(a)))for(g=na(h),f=na(a),d=0,e=f.length;d0&&oa(g,!i&&na(a,"script")),h},cleanData:function(a){for(var b,c,d,e=r.event.special,f=0;void 0!==(c=a[f]);f++)if(U(c)){if(b=c[W.expando]){if(b.events)for(d in b.events)e[d]?r.event.remove(c,d):r.removeEvent(c,d,b.handle);c[W.expando]=void 0}c[X.expando]&&(c[X.expando]=void 0)}}}),r.fn.extend({detach:function(a){return Ka(this,a,!0)},remove:function(a){return Ka(this,a)},text:function(a){return T(this,function(a){return void 0===a?r.text(this):this.empty().each(function(){1!==this.nodeType&&11!==this.nodeType&&9!==this.nodeType||(this.textContent=a)})},null,a,arguments.length)},append:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.appendChild(a)}})},prepend:function(){return Ja(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ea(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return Ja(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},empty:function(){for(var a,b=0;null!=(a=this[b]);b++)1===a.nodeType&&(r.cleanData(na(a,!1)),a.textContent="");return this},clone:function(a,b){return a=null!=a&&a,b=null==b?a:b,this.map(function(){return r.clone(this,a,b)})},html:function(a){return T(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a&&1===b.nodeType)return b.innerHTML;if("string"==typeof a&&!Aa.test(a)&&!ma[(ka.exec(a)||["",""])[1].toLowerCase()]){a=r.htmlPrefilter(a);try{for(;c1)}});function _a(a,b,c,d,e){return new _a.prototype.init(a,b,c,d,e)}r.Tween=_a,_a.prototype={constructor:_a,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||r.easing._default,this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(r.cssNumber[c]?"":"px")},cur:function(){var a=_a.propHooks[this.prop];return a&&a.get?a.get(this):_a.propHooks._default.get(this)},run:function(a){var b,c=_a.propHooks[this.prop];return this.options.duration?this.pos=b=r.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):this.pos=b=a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):_a.propHooks._default.set(this),this}},_a.prototype.init.prototype=_a.prototype,_a.propHooks={_default:{get:function(a){var b;return 1!==a.elem.nodeType||null!=a.elem[a.prop]&&null==a.elem.style[a.prop]?a.elem[a.prop]:(b=r.css(a.elem,a.prop,""),b&&"auto"!==b?b:0)},set:function(a){r.fx.step[a.prop]?r.fx.step[a.prop](a):1!==a.elem.nodeType||null==a.elem.style[r.cssProps[a.prop]]&&!r.cssHooks[a.prop]?a.elem[a.prop]=a.now:r.style(a.elem,a.prop,a.now+a.unit)}}},_a.propHooks.scrollTop=_a.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},r.easing={linear:function(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2},_default:"swing"},r.fx=_a.prototype.init,r.fx.step={};var ab,bb,cb=/^(?:toggle|show|hide)$/,db=/queueHooks$/;function eb(){bb&&(d.hidden===!1&&a.requestAnimationFrame?a.requestAnimationFrame(eb):a.setTimeout(eb,r.fx.interval),r.fx.tick())}function fb(){return a.setTimeout(function(){ab=void 0}),ab=r.now()}function gb(a,b){var c,d=0,e={height:a};for(b=b?1:0;d<4;d+=2-b)c=ca[d],e["margin"+c]=e["padding"+c]=a;return b&&(e.opacity=e.width=a),e}function hb(a,b,c){for(var d,e=(kb.tweeners[b]||[]).concat(kb.tweeners["*"]),f=0,g=e.length;f1)},removeAttr:function(a){return this.each(function(){r.removeAttr(this,a)})}}),r.extend({attr:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return"undefined"==typeof a.getAttribute?r.prop(a,b,c):(1===f&&r.isXMLDoc(a)||(e=r.attrHooks[b.toLowerCase()]||(r.expr.match.bool.test(b)?lb:void 0)),void 0!==c?null===c?void r.removeAttr(a,b):e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:(a.setAttribute(b,c+""),c):e&&"get"in e&&null!==(d=e.get(a,b))?d:(d=r.find.attr(a,b), +null==d?void 0:d))},attrHooks:{type:{set:function(a,b){if(!o.radioValue&&"radio"===b&&B(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}}},removeAttr:function(a,b){var c,d=0,e=b&&b.match(L);if(e&&1===a.nodeType)while(c=e[d++])a.removeAttribute(c)}}),lb={set:function(a,b,c){return b===!1?r.removeAttr(a,c):a.setAttribute(c,c),c}},r.each(r.expr.match.bool.source.match(/\w+/g),function(a,b){var c=mb[b]||r.find.attr;mb[b]=function(a,b,d){var e,f,g=b.toLowerCase();return d||(f=mb[g],mb[g]=e,e=null!=c(a,b,d)?g:null,mb[g]=f),e}});var nb=/^(?:input|select|textarea|button)$/i,ob=/^(?:a|area)$/i;r.fn.extend({prop:function(a,b){return T(this,r.prop,a,b,arguments.length>1)},removeProp:function(a){return this.each(function(){delete this[r.propFix[a]||a]})}}),r.extend({prop:function(a,b,c){var d,e,f=a.nodeType;if(3!==f&&8!==f&&2!==f)return 1===f&&r.isXMLDoc(a)||(b=r.propFix[b]||b,e=r.propHooks[b]),void 0!==c?e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:a[b]=c:e&&"get"in e&&null!==(d=e.get(a,b))?d:a[b]},propHooks:{tabIndex:{get:function(a){var b=r.find.attr(a,"tabindex");return b?parseInt(b,10):nb.test(a.nodeName)||ob.test(a.nodeName)&&a.href?0:-1}}},propFix:{"for":"htmlFor","class":"className"}}),o.optSelected||(r.propHooks.selected={get:function(a){var b=a.parentNode;return b&&b.parentNode&&b.parentNode.selectedIndex,null},set:function(a){var b=a.parentNode;b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex)}}),r.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"],function(){r.propFix[this.toLowerCase()]=this});function pb(a){var b=a.match(L)||[];return b.join(" ")}function qb(a){return a.getAttribute&&a.getAttribute("class")||""}r.fn.extend({addClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).addClass(a.call(this,b,qb(this)))});if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&&" "+pb(e)+" "){g=0;while(f=b[g++])d.indexOf(" "+f+" ")<0&&(d+=f+" ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},removeClass:function(a){var b,c,d,e,f,g,h,i=0;if(r.isFunction(a))return this.each(function(b){r(this).removeClass(a.call(this,b,qb(this)))});if(!arguments.length)return this.attr("class","");if("string"==typeof a&&a){b=a.match(L)||[];while(c=this[i++])if(e=qb(c),d=1===c.nodeType&&" "+pb(e)+" "){g=0;while(f=b[g++])while(d.indexOf(" "+f+" ")>-1)d=d.replace(" "+f+" "," ");h=pb(d),e!==h&&c.setAttribute("class",h)}}return this},toggleClass:function(a,b){var c=typeof a;return"boolean"==typeof b&&"string"===c?b?this.addClass(a):this.removeClass(a):r.isFunction(a)?this.each(function(c){r(this).toggleClass(a.call(this,c,qb(this),b),b)}):this.each(function(){var b,d,e,f;if("string"===c){d=0,e=r(this),f=a.match(L)||[];while(b=f[d++])e.hasClass(b)?e.removeClass(b):e.addClass(b)}else void 0!==a&&"boolean"!==c||(b=qb(this),b&&W.set(this,"__className__",b),this.setAttribute&&this.setAttribute("class",b||a===!1?"":W.get(this,"__className__")||""))})},hasClass:function(a){var b,c,d=0;b=" "+a+" ";while(c=this[d++])if(1===c.nodeType&&(" "+pb(qb(c))+" ").indexOf(b)>-1)return!0;return!1}});var rb=/\r/g;r.fn.extend({val:function(a){var b,c,d,e=this[0];{if(arguments.length)return d=r.isFunction(a),this.each(function(c){var e;1===this.nodeType&&(e=d?a.call(this,c,r(this).val()):a,null==e?e="":"number"==typeof e?e+="":Array.isArray(e)&&(e=r.map(e,function(a){return null==a?"":a+""})),b=r.valHooks[this.type]||r.valHooks[this.nodeName.toLowerCase()],b&&"set"in b&&void 0!==b.set(this,e,"value")||(this.value=e))});if(e)return b=r.valHooks[e.type]||r.valHooks[e.nodeName.toLowerCase()],b&&"get"in b&&void 0!==(c=b.get(e,"value"))?c:(c=e.value,"string"==typeof c?c.replace(rb,""):null==c?"":c)}}}),r.extend({valHooks:{option:{get:function(a){var b=r.find.attr(a,"value");return null!=b?b:pb(r.text(a))}},select:{get:function(a){var b,c,d,e=a.options,f=a.selectedIndex,g="select-one"===a.type,h=g?null:[],i=g?f+1:e.length;for(d=f<0?i:g?f:0;d-1)&&(c=!0);return c||(a.selectedIndex=-1),f}}}}),r.each(["radio","checkbox"],function(){r.valHooks[this]={set:function(a,b){if(Array.isArray(b))return a.checked=r.inArray(r(a).val(),b)>-1}},o.checkOn||(r.valHooks[this].get=function(a){return null===a.getAttribute("value")?"on":a.value})});var sb=/^(?:focusinfocus|focusoutblur)$/;r.extend(r.event,{trigger:function(b,c,e,f){var g,h,i,j,k,m,n,o=[e||d],p=l.call(b,"type")?b.type:b,q=l.call(b,"namespace")?b.namespace.split("."):[];if(h=i=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!sb.test(p+r.event.triggered)&&(p.indexOf(".")>-1&&(q=p.split("."),p=q.shift(),q.sort()),k=p.indexOf(":")<0&&"on"+p,b=b[r.expando]?b:new r.Event(p,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=q.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:r.makeArray(c,[b]),n=r.event.special[p]||{},f||!n.trigger||n.trigger.apply(e,c)!==!1)){if(!f&&!n.noBubble&&!r.isWindow(e)){for(j=n.delegateType||p,sb.test(j+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),i=h;i===(e.ownerDocument||d)&&o.push(i.defaultView||i.parentWindow||a)}g=0;while((h=o[g++])&&!b.isPropagationStopped())b.type=g>1?j:n.bindType||p,m=(W.get(h,"events")||{})[b.type]&&W.get(h,"handle"),m&&m.apply(h,c),m=k&&h[k],m&&m.apply&&U(h)&&(b.result=m.apply(h,c),b.result===!1&&b.preventDefault());return b.type=p,f||b.isDefaultPrevented()||n._default&&n._default.apply(o.pop(),c)!==!1||!U(e)||k&&r.isFunction(e[p])&&!r.isWindow(e)&&(i=e[k],i&&(e[k]=null),r.event.triggered=p,e[p](),r.event.triggered=void 0,i&&(e[k]=i)),b.result}},simulate:function(a,b,c){var d=r.extend(new r.Event,c,{type:a,isSimulated:!0});r.event.trigger(d,null,b)}}),r.fn.extend({trigger:function(a,b){return this.each(function(){r.event.trigger(a,b,this)})},triggerHandler:function(a,b){var c=this[0];if(c)return r.event.trigger(a,b,c,!0)}}),r.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(a,b){r.fn[b]=function(a,c){return arguments.length>0?this.on(b,null,a,c):this.trigger(b)}}),r.fn.extend({hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),o.focusin="onfocusin"in a,o.focusin||r.each({focus:"focusin",blur:"focusout"},function(a,b){var c=function(a){r.event.simulate(b,a.target,r.event.fix(a))};r.event.special[b]={setup:function(){var d=this.ownerDocument||this,e=W.access(d,b);e||d.addEventListener(a,c,!0),W.access(d,b,(e||0)+1)},teardown:function(){var d=this.ownerDocument||this,e=W.access(d,b)-1;e?W.access(d,b,e):(d.removeEventListener(a,c,!0),W.remove(d,b))}}});var tb=a.location,ub=r.now(),vb=/\?/;r.parseXML=function(b){var c;if(!b||"string"!=typeof b)return null;try{c=(new a.DOMParser).parseFromString(b,"text/xml")}catch(d){c=void 0}return c&&!c.getElementsByTagName("parsererror").length||r.error("Invalid XML: "+b),c};var wb=/\[\]$/,xb=/\r?\n/g,yb=/^(?:submit|button|image|reset|file)$/i,zb=/^(?:input|select|textarea|keygen)/i;function Ab(a,b,c,d){var e;if(Array.isArray(b))r.each(b,function(b,e){c||wb.test(a)?d(a,e):Ab(a+"["+("object"==typeof e&&null!=e?b:"")+"]",e,c,d)});else if(c||"object"!==r.type(b))d(a,b);else for(e in b)Ab(a+"["+e+"]",b[e],c,d)}r.param=function(a,b){var c,d=[],e=function(a,b){var c=r.isFunction(b)?b():b;d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(null==c?"":c)};if(Array.isArray(a)||a.jquery&&!r.isPlainObject(a))r.each(a,function(){e(this.name,this.value)});else for(c in a)Ab(c,a[c],b,e);return d.join("&")},r.fn.extend({serialize:function(){return r.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var a=r.prop(this,"elements");return a?r.makeArray(a):this}).filter(function(){var a=this.type;return this.name&&!r(this).is(":disabled")&&zb.test(this.nodeName)&&!yb.test(a)&&(this.checked||!ja.test(a))}).map(function(a,b){var c=r(this).val();return null==c?null:Array.isArray(c)?r.map(c,function(a){return{name:b.name,value:a.replace(xb,"\r\n")}}):{name:b.name,value:c.replace(xb,"\r\n")}}).get()}});var Bb=/%20/g,Cb=/#.*$/,Db=/([?&])_=[^&]*/,Eb=/^(.*?):[ \t]*([^\r\n]*)$/gm,Fb=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Gb=/^(?:GET|HEAD)$/,Hb=/^\/\//,Ib={},Jb={},Kb="*/".concat("*"),Lb=d.createElement("a");Lb.href=tb.href;function Mb(a){return function(b,c){"string"!=typeof b&&(c=b,b="*");var d,e=0,f=b.toLowerCase().match(L)||[];if(r.isFunction(c))while(d=f[e++])"+"===d[0]?(d=d.slice(1)||"*",(a[d]=a[d]||[]).unshift(c)):(a[d]=a[d]||[]).push(c)}}function Nb(a,b,c,d){var e={},f=a===Jb;function g(h){var i;return e[h]=!0,r.each(a[h]||[],function(a,h){var j=h(b,c,d);return"string"!=typeof j||f||e[j]?f?!(i=j):void 0:(b.dataTypes.unshift(j),g(j),!1)}),i}return g(b.dataTypes[0])||!e["*"]&&g("*")}function Ob(a,b){var c,d,e=r.ajaxSettings.flatOptions||{};for(c in b)void 0!==b[c]&&((e[c]?a:d||(d={}))[c]=b[c]);return d&&r.extend(!0,a,d),a}function Pb(a,b,c){var d,e,f,g,h=a.contents,i=a.dataTypes;while("*"===i[0])i.shift(),void 0===d&&(d=a.mimeType||b.getResponseHeader("Content-Type"));if(d)for(e in h)if(h[e]&&h[e].test(d)){i.unshift(e);break}if(i[0]in c)f=i[0];else{for(e in c){if(!i[0]||a.converters[e+" "+i[0]]){f=e;break}g||(g=e)}f=f||g}if(f)return f!==i[0]&&i.unshift(f),c[f]}function Qb(a,b,c,d){var e,f,g,h,i,j={},k=a.dataTypes.slice();if(k[1])for(g in a.converters)j[g.toLowerCase()]=a.converters[g];f=k.shift();while(f)if(a.responseFields[f]&&(c[a.responseFields[f]]=b),!i&&d&&a.dataFilter&&(b=a.dataFilter(b,a.dataType)),i=f,f=k.shift())if("*"===f)f=i;else if("*"!==i&&i!==f){if(g=j[i+" "+f]||j["* "+f],!g)for(e in j)if(h=e.split(" "),h[1]===f&&(g=j[i+" "+h[0]]||j["* "+h[0]])){g===!0?g=j[e]:j[e]!==!0&&(f=h[0],k.unshift(h[1]));break}if(g!==!0)if(g&&a["throws"])b=g(b);else try{b=g(b)}catch(l){return{state:"parsererror",error:g?l:"No conversion from "+i+" to "+f}}}return{state:"success",data:b}}r.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:tb.href,type:"GET",isLocal:Fb.test(tb.protocol),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Kb,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/\bxml\b/,html:/\bhtml/,json:/\bjson\b/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":JSON.parse,"text xml":r.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(a,b){return b?Ob(Ob(a,r.ajaxSettings),b):Ob(r.ajaxSettings,a)},ajaxPrefilter:Mb(Ib),ajaxTransport:Mb(Jb),ajax:function(b,c){"object"==typeof b&&(c=b,b=void 0),c=c||{};var e,f,g,h,i,j,k,l,m,n,o=r.ajaxSetup({},c),p=o.context||o,q=o.context&&(p.nodeType||p.jquery)?r(p):r.event,s=r.Deferred(),t=r.Callbacks("once memory"),u=o.statusCode||{},v={},w={},x="canceled",y={readyState:0,getResponseHeader:function(a){var b;if(k){if(!h){h={};while(b=Eb.exec(g))h[b[1].toLowerCase()]=b[2]}b=h[a.toLowerCase()]}return null==b?null:b},getAllResponseHeaders:function(){return k?g:null},setRequestHeader:function(a,b){return null==k&&(a=w[a.toLowerCase()]=w[a.toLowerCase()]||a,v[a]=b),this},overrideMimeType:function(a){return null==k&&(o.mimeType=a),this},statusCode:function(a){var b;if(a)if(k)y.always(a[y.status]);else for(b in a)u[b]=[u[b],a[b]];return this},abort:function(a){var b=a||x;return e&&e.abort(b),A(0,b),this}};if(s.promise(y),o.url=((b||o.url||tb.href)+"").replace(Hb,tb.protocol+"//"),o.type=c.method||c.type||o.method||o.type,o.dataTypes=(o.dataType||"*").toLowerCase().match(L)||[""],null==o.crossDomain){j=d.createElement("a");try{j.href=o.url,j.href=j.href,o.crossDomain=Lb.protocol+"//"+Lb.host!=j.protocol+"//"+j.host}catch(z){o.crossDomain=!0}}if(o.data&&o.processData&&"string"!=typeof o.data&&(o.data=r.param(o.data,o.traditional)),Nb(Ib,o,c,y),k)return y;l=r.event&&o.global,l&&0===r.active++&&r.event.trigger("ajaxStart"),o.type=o.type.toUpperCase(),o.hasContent=!Gb.test(o.type),f=o.url.replace(Cb,""),o.hasContent?o.data&&o.processData&&0===(o.contentType||"").indexOf("application/x-www-form-urlencoded")&&(o.data=o.data.replace(Bb,"+")):(n=o.url.slice(f.length),o.data&&(f+=(vb.test(f)?"&":"?")+o.data,delete o.data),o.cache===!1&&(f=f.replace(Db,"$1"),n=(vb.test(f)?"&":"?")+"_="+ub++ +n),o.url=f+n),o.ifModified&&(r.lastModified[f]&&y.setRequestHeader("If-Modified-Since",r.lastModified[f]),r.etag[f]&&y.setRequestHeader("If-None-Match",r.etag[f])),(o.data&&o.hasContent&&o.contentType!==!1||c.contentType)&&y.setRequestHeader("Content-Type",o.contentType),y.setRequestHeader("Accept",o.dataTypes[0]&&o.accepts[o.dataTypes[0]]?o.accepts[o.dataTypes[0]]+("*"!==o.dataTypes[0]?", "+Kb+"; q=0.01":""):o.accepts["*"]);for(m in o.headers)y.setRequestHeader(m,o.headers[m]);if(o.beforeSend&&(o.beforeSend.call(p,y,o)===!1||k))return y.abort();if(x="abort",t.add(o.complete),y.done(o.success),y.fail(o.error),e=Nb(Jb,o,c,y)){if(y.readyState=1,l&&q.trigger("ajaxSend",[y,o]),k)return y;o.async&&o.timeout>0&&(i=a.setTimeout(function(){y.abort("timeout")},o.timeout));try{k=!1,e.send(v,A)}catch(z){if(k)throw z;A(-1,z)}}else A(-1,"No Transport");function A(b,c,d,h){var j,m,n,v,w,x=c;k||(k=!0,i&&a.clearTimeout(i),e=void 0,g=h||"",y.readyState=b>0?4:0,j=b>=200&&b<300||304===b,d&&(v=Pb(o,y,d)),v=Qb(o,v,y,j),j?(o.ifModified&&(w=y.getResponseHeader("Last-Modified"),w&&(r.lastModified[f]=w),w=y.getResponseHeader("etag"),w&&(r.etag[f]=w)),204===b||"HEAD"===o.type?x="nocontent":304===b?x="notmodified":(x=v.state,m=v.data,n=v.error,j=!n)):(n=x,!b&&x||(x="error",b<0&&(b=0))),y.status=b,y.statusText=(c||x)+"",j?s.resolveWith(p,[m,x,y]):s.rejectWith(p,[y,x,n]),y.statusCode(u),u=void 0,l&&q.trigger(j?"ajaxSuccess":"ajaxError",[y,o,j?m:n]),t.fireWith(p,[y,x]),l&&(q.trigger("ajaxComplete",[y,o]),--r.active||r.event.trigger("ajaxStop")))}return y},getJSON:function(a,b,c){return r.get(a,b,c,"json")},getScript:function(a,b){return r.get(a,void 0,b,"script")}}),r.each(["get","post"],function(a,b){r[b]=function(a,c,d,e){return r.isFunction(c)&&(e=e||d,d=c,c=void 0),r.ajax(r.extend({url:a,type:b,dataType:e,data:c,success:d},r.isPlainObject(a)&&a))}}),r._evalUrl=function(a){return r.ajax({url:a,type:"GET",dataType:"script",cache:!0,async:!1,global:!1,"throws":!0})},r.fn.extend({wrapAll:function(a){var b;return this[0]&&(r.isFunction(a)&&(a=a.call(this[0])),b=r(a,this[0].ownerDocument).eq(0).clone(!0),this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstElementChild)a=a.firstElementChild;return a}).append(this)),this},wrapInner:function(a){return r.isFunction(a)?this.each(function(b){r(this).wrapInner(a.call(this,b))}):this.each(function(){var b=r(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=r.isFunction(a);return this.each(function(c){r(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(a){return this.parent(a).not("body").each(function(){r(this).replaceWith(this.childNodes)}),this}}),r.expr.pseudos.hidden=function(a){return!r.expr.pseudos.visible(a)},r.expr.pseudos.visible=function(a){return!!(a.offsetWidth||a.offsetHeight||a.getClientRects().length)},r.ajaxSettings.xhr=function(){try{return new a.XMLHttpRequest}catch(b){}};var Rb={0:200,1223:204},Sb=r.ajaxSettings.xhr();o.cors=!!Sb&&"withCredentials"in Sb,o.ajax=Sb=!!Sb,r.ajaxTransport(function(b){var c,d;if(o.cors||Sb&&!b.crossDomain)return{send:function(e,f){var g,h=b.xhr();if(h.open(b.type,b.url,b.async,b.username,b.password),b.xhrFields)for(g in b.xhrFields)h[g]=b.xhrFields[g];b.mimeType&&h.overrideMimeType&&h.overrideMimeType(b.mimeType),b.crossDomain||e["X-Requested-With"]||(e["X-Requested-With"]="XMLHttpRequest");for(g in e)h.setRequestHeader(g,e[g]);c=function(a){return function(){c&&(c=d=h.onload=h.onerror=h.onabort=h.onreadystatechange=null,"abort"===a?h.abort():"error"===a?"number"!=typeof h.status?f(0,"error"):f(h.status,h.statusText):f(Rb[h.status]||h.status,h.statusText,"text"!==(h.responseType||"text")||"string"!=typeof h.responseText?{binary:h.response}:{text:h.responseText},h.getAllResponseHeaders()))}},h.onload=c(),d=h.onerror=c("error"),void 0!==h.onabort?h.onabort=d:h.onreadystatechange=function(){4===h.readyState&&a.setTimeout(function(){c&&d()})},c=c("abort");try{h.send(b.hasContent&&b.data||null)}catch(i){if(c)throw i}},abort:function(){c&&c()}}}),r.ajaxPrefilter(function(a){a.crossDomain&&(a.contents.script=!1)}),r.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/\b(?:java|ecma)script\b/},converters:{"text script":function(a){return r.globalEval(a),a}}}),r.ajaxPrefilter("script",function(a){void 0===a.cache&&(a.cache=!1),a.crossDomain&&(a.type="GET")}),r.ajaxTransport("script",function(a){if(a.crossDomain){var b,c;return{send:function(e,f){b=r(" + + + {title} + + + def navBar(pageName: String): Seq[Node] = + + + def createPage(pageName: String, pageContents: Seq[Node]): Seq[Node] = + + {getHeader("Livy - " + pageName)} + +
+ {navBar(pageName)} + {pageContents} +
+ + + + get("/") { + val content = +
+
+
+
+ + createPage("Sessions", content) + } +} diff --git a/server/src/main/scala/com/cloudera/livy/sessions/Session.scala b/server/src/main/scala/com/cloudera/livy/sessions/Session.scala index faf342262..a0d0f8b10 100644 --- a/server/src/main/scala/com/cloudera/livy/sessions/Session.scala +++ b/server/src/main/scala/com/cloudera/livy/sessions/Session.scala @@ -197,7 +197,6 @@ abstract class Session(val id: Int, val owner: String, val livyConf: LivyConf) } } - val timeout: Long = TimeUnit.HOURS.toNanos(1) override def toString(): String = s"${this.getClass.getSimpleName} $id" diff --git a/server/src/main/scala/com/cloudera/livy/sessions/SessionManager.scala b/server/src/main/scala/com/cloudera/livy/sessions/SessionManager.scala index ac0561fb6..edf61c704 100644 --- a/server/src/main/scala/com/cloudera/livy/sessions/SessionManager.scala +++ b/server/src/main/scala/com/cloudera/livy/sessions/SessionManager.scala @@ -29,14 +29,13 @@ import scala.util.control.NonFatal import com.cloudera.livy.{LivyConf, Logging} import com.cloudera.livy.server.batch.{BatchRecoveryMetadata, BatchSession} -import com.cloudera.livy.server.interactive.{InteractiveRecoveryMetadata, InteractiveSession} +import com.cloudera.livy.server.interactive.{InteractiveRecoveryMetadata, InteractiveSession, SessionHeartbeatWatchdog} import com.cloudera.livy.server.recovery.SessionStore import com.cloudera.livy.sessions.Session.RecoveryMetadata object SessionManager { val SESSION_RECOVERY_MODE_OFF = "off" val SESSION_RECOVERY_MODE_RECOVERY = "recovery" - val SESSION_TIMEOUT = LivyConf.Entry("livy.server.session.timeout", "1h") } class BatchSessionManager( @@ -56,9 +55,13 @@ class InteractiveSessionManager( sessionStore, "interactive", mockSessions) + with SessionHeartbeatWatchdog[InteractiveSession, InteractiveRecoveryMetadata] + { + start() + } class SessionManager[S <: Session, R <: RecoveryMetadata : ClassTag]( - livyConf: LivyConf, + protected val livyConf: LivyConf, sessionRecovery: R => S, sessionStore: SessionStore, sessionType: String, @@ -72,8 +75,11 @@ class SessionManager[S <: Session, R <: RecoveryMetadata : ClassTag]( protected[this] final val idCounter = new AtomicInteger(0) protected[this] final val sessions = mutable.LinkedHashMap[Int, S]() + private[this] final val sessionTimeoutCheck = livyConf.getBoolean(LivyConf.SESSION_TIMEOUT_CHECK) private[this] final val sessionTimeout = - TimeUnit.MILLISECONDS.toNanos(livyConf.getTimeAsMs(SessionManager.SESSION_TIMEOUT)) + TimeUnit.MILLISECONDS.toNanos(livyConf.getTimeAsMs(LivyConf.SESSION_TIMEOUT)) + private[this] final val sessionStateRetainedInSec = + TimeUnit.MILLISECONDS.toNanos(livyConf.getTimeAsMs(LivyConf.SESSION_STATE_RETAIN_TIME)) mockSessions.getOrElse(recover()).foreach(register) new GarbageCollector().start() @@ -128,8 +134,20 @@ class SessionManager[S <: Session, R <: RecoveryMetadata : ClassTag]( def collectGarbage(): Future[Iterable[Unit]] = { def expired(session: Session): Boolean = { - val currentTime = System.nanoTime() - currentTime - session.lastActivity > math.max(sessionTimeout, session.timeout) + session.state match { + case s: FinishedSessionState => + val currentTime = System.nanoTime() + currentTime - s.time > sessionStateRetainedInSec + case _ => + if (!sessionTimeoutCheck) { + false + } else if (session.isInstanceOf[BatchSession]) { + false + } else { + val currentTime = System.nanoTime() + currentTime - session.lastActivity > sessionTimeout + } + } } Future.sequence(all().filter(expired).map(delete)) diff --git a/server/src/main/scala/com/cloudera/livy/utils/LineBufferedStream.scala b/server/src/main/scala/com/cloudera/livy/utils/LineBufferedStream.scala index d54bfb0bf..a8949aff0 100644 --- a/server/src/main/scala/com/cloudera/livy/utils/LineBufferedStream.scala +++ b/server/src/main/scala/com/cloudera/livy/utils/LineBufferedStream.scala @@ -39,7 +39,6 @@ class LineBufferedStream(inputStream: InputStream) extends Logging { for (line <- lines) { _lock.lock() try { - trace("stdout: ", line) _lines = _lines :+ line _condition.signalAll() } finally { @@ -47,6 +46,7 @@ class LineBufferedStream(inputStream: InputStream) extends Logging { } } + _lines.map { line => info("stdout: ", line) } _lock.lock() try { _finished = true diff --git a/server/src/main/scala/com/cloudera/livy/utils/LivySparkUtils.scala b/server/src/main/scala/com/cloudera/livy/utils/LivySparkUtils.scala index 2a7391f1f..766eb91ee 100644 --- a/server/src/main/scala/com/cloudera/livy/utils/LivySparkUtils.scala +++ b/server/src/main/scala/com/cloudera/livy/utils/LivySparkUtils.scala @@ -32,6 +32,8 @@ object LivySparkUtils extends Logging { // For each Spark version we supported, we need to add this mapping relation in case Scala // version cannot be detected from "spark-submit --version". private val _defaultSparkScalaVersion = SortedMap( + // Spark 2.1 + Scala 2.11 + (2, 1) -> "2.11", // Spark 2.0 + Scala 2.11 (2, 0) -> "2.11", // Spark 1.6 + Scala 2.10 @@ -40,7 +42,7 @@ object LivySparkUtils extends Logging { // Supported Spark version private val MIN_VERSION = (1, 6) - private val MAX_VERSION = (2, 1) + private val MAX_VERSION = (2, 2) private val sparkVersionRegex = """version (.*)""".r.unanchored private val scalaVersionRegex = """Scala version (.*), Java""".r.unanchored @@ -143,11 +145,11 @@ object LivySparkUtils extends Logging { * @return Two element tuple, one is major version and the other is minor version */ def formatSparkVersion(version: String): (Int, Int) = { - val versionPattern = """(\d)+\.(\d)+(?:[\.-]\d*)*""".r - version match { - case versionPattern(major, minor) => - (major.toInt, minor.toInt) - case _ => + val versionPattern = """^(\d+)\.(\d+)(\..*)?$""".r + versionPattern.findFirstMatchIn(version) match { + case Some(m) => + (m.group(1).toInt, m.group(2).toInt) + case None => throw new IllegalArgumentException(s"Fail to parse Spark version from $version") } } diff --git a/server/src/main/scala/com/cloudera/livy/utils/SparkYarnApp.scala b/server/src/main/scala/com/cloudera/livy/utils/SparkYarnApp.scala index 28ed82d10..fb47e5e4b 100644 --- a/server/src/main/scala/com/cloudera/livy/utils/SparkYarnApp.scala +++ b/server/src/main/scala/com/cloudera/livy/utils/SparkYarnApp.scala @@ -37,6 +37,15 @@ import com.cloudera.livy.{LivyConf, Logging, Utils} import com.cloudera.livy.util.LineBufferedProcess object SparkYarnApp extends Logging { + + def init(livyConf: LivyConf): Unit = { + sessionLeakageCheckInterval = livyConf.getTimeAsMs(LivyConf.YARN_APP_LEAKAGE_CHECK_INTERVAL) + sessionLeakageCheckTimeout = livyConf.getTimeAsMs(LivyConf.YARN_APP_LEAKAGE_CHECK_TIMEOUT) + leakedAppsGCThread.setDaemon(true) + leakedAppsGCThread.setName("LeakedAppsGCThread") + leakedAppsGCThread.start() + } + // YarnClient is thread safe. Create once, share it across threads. lazy val yarnClient = { val c = YarnClient.createYarnClient() @@ -50,6 +59,47 @@ object SparkYarnApp extends Logging { private def getYarnPollInterval(livyConf: LivyConf): FiniteDuration = livyConf.getTimeAsMs(LivyConf.YARN_POLL_INTERVAL) milliseconds + + private val appType = Set("SPARK").asJava + + private val leakedAppTags = new java.util.concurrent.ConcurrentHashMap[String, Long]() + + private var sessionLeakageCheckTimeout: Long = _ + + private var sessionLeakageCheckInterval: Long = _ + + private val leakedAppsGCThread = new Thread() { + override def run(): Unit = { + while (true) { + if (!leakedAppTags.isEmpty) { + // kill the app if found it and remove it if exceeding a threashold + val iter = leakedAppTags.entrySet().iterator() + var isRemoved = false + val now = System.currentTimeMillis() + val apps = yarnClient.getApplications(appType).asScala + while(iter.hasNext) { + val entry = iter.next() + apps.find(_.getApplicationTags.contains(entry.getKey)) + .foreach({ e => + info(s"Kill leaked app ${e.getApplicationId}") + yarnClient.killApplication(e.getApplicationId) + iter.remove() + isRemoved = true + }) + if (!isRemoved) { + if ((entry.getValue - now) > sessionLeakageCheckTimeout) { + iter.remove() + info(s"Remove leaked yarn app tag ${entry.getKey}") + } + } + } + } + Thread.sleep(sessionLeakageCheckInterval) + } + } + } + + } /** @@ -78,7 +128,9 @@ class SparkYarnApp private[utils] ( private var yarnDiagnostics: IndexedSeq[String] = IndexedSeq.empty[String] override def log(): IndexedSeq[String] = - process.map(_.inputLines).getOrElse(ArrayBuffer.empty[String]) ++ yarnDiagnostics + ("stdout: " +: process.map(_.inputLines).getOrElse(ArrayBuffer.empty[String])) ++ + ("\nstderr: " +: process.map(_.errorLines).getOrElse(ArrayBuffer.empty[String])) ++ + ("\nYARN Diagnostics: " +: yarnDiagnostics) override def kill(): Unit = synchronized { if (isRunning) { @@ -122,13 +174,16 @@ class SparkYarnApp private[utils] ( // FIXME Should not loop thru all YARN applications but YarnClient doesn't offer an API. // Consider calling rmClient in YarnClient directly. - val appType = Set("SPARK").asJava yarnClient.getApplications(appType).asScala.find(_.getApplicationTags.contains(appTagLowerCase)) match { case Some(app) => app.getApplicationId case None => if (deadline.isOverdue) { - throw new Exception(s"No YARN application is tagged with $appTagLowerCase.") + process.foreach(_.destroy()) + leakedAppTags.put(appTag, System.currentTimeMillis()) + throw new Exception(s"No YARN application is found with tag $appTagLowerCase in " + + livyConf.getTimeAsMs(LivyConf.YARN_APP_LOOKUP_TIMEOUT)/1000 + " seconds. " + + "Please check your cluster status, it is may be very busy.") } else { Clock.sleep(pollInterval.toMillis) getAppIdFromTag(appTagLowerCase, pollInterval, deadline) diff --git a/server/src/test/scala/com/cloudera/livy/server/interactive/BaseInteractiveServletSpec.scala b/server/src/test/scala/com/cloudera/livy/server/interactive/BaseInteractiveServletSpec.scala index 4aa16c1ce..f08e09ea4 100644 --- a/server/src/test/scala/com/cloudera/livy/server/interactive/BaseInteractiveServletSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/server/interactive/BaseInteractiveServletSpec.scala @@ -50,7 +50,7 @@ abstract class BaseInteractiveServletSpec } super.createConf() .set(LivyConf.SESSION_STAGING_DIR, tempDir.toURI().toString()) - .set(InteractiveSession.LIVY_REPL_JARS, "") + .set(LivyConf.REPL_JARS, "dummy.jar") .set(LivyConf.LIVY_SPARK_VERSION, "1.6.0") .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10.5") } diff --git a/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionServletSpec.scala b/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionServletSpec.scala index 0a3fb99fc..63d605d98 100644 --- a/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionServletSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionServletSpec.scala @@ -19,12 +19,11 @@ package com.cloudera.livy.server.interactive import java.util.concurrent.atomic.AtomicInteger -import javax.servlet.http.HttpServletRequest +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.collection.JavaConverters._ import scala.concurrent.Future -import org.json4s.JsonAST._ import org.json4s.jackson.Json4sScalaModule import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -79,6 +78,14 @@ class InteractiveSessionServletSpec extends BaseInteractiveServletSpec { statement } }) + when(session.cancelStatement(anyInt())).thenAnswer( + new Answer[Unit] { + override def answer(args: InvocationOnMock): Unit = { + statements = IndexedSeq( + new Statement(statementCounter.get(), StatementState.Cancelled, null)) + } + } + ) session } @@ -114,6 +121,7 @@ class InteractiveSessionServletSpec extends BaseInteractiveServletSpec { jpost[Map[String, Any]]("/0/statements", ExecuteRequest("foo")) { data => data("id") should be (0) + data("progress") should be (0.0) data("output") shouldBe 1 } @@ -122,6 +130,15 @@ class InteractiveSessionServletSpec extends BaseInteractiveServletSpec { data("statements").asInstanceOf[Seq[Map[String, Any]]](0)("id") should be (0) } + jpost[Map[String, Any]]("/0/statements/0/cancel", null, HttpServletResponse.SC_OK) { data => + data should equal(Map("msg" -> "canceled")) + } + + jget[Map[String, Any]]("/0/statements") { data => + data("total_statements") should be (1) + data("statements").asInstanceOf[Seq[Map[String, Any]]](0)("state") should be ("cancelled") + } + jdelete[Map[String, Any]]("/0") { data => data should equal (Map("msg" -> "deleted")) diff --git a/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionSpec.scala b/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionSpec.scala index c5fd5925d..28d715781 100644 --- a/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/server/interactive/InteractiveSessionSpec.scala @@ -45,7 +45,7 @@ class InteractiveSessionSpec extends FunSpec with Matchers with BeforeAndAfterAll with LivyBaseUnitTestSuite { private val livyConf = new LivyConf() - livyConf.set(InteractiveSession.LIVY_REPL_JARS, "") + livyConf.set(LivyConf.REPL_JARS, "dummy.jar") .set(LivyConf.LIVY_SPARK_VERSION, "1.6.0") .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10.5") @@ -76,7 +76,7 @@ class InteractiveSessionSpec extends FunSpec val id = session.executeStatement(ExecuteRequest(code)).id eventually(timeout(30 seconds), interval(100 millis)) { val s = session.getStatement(id).get - s.state shouldBe StatementState.Available + s.state.get() shouldBe StatementState.Available parse(s.output) } } @@ -100,6 +100,61 @@ class InteractiveSessionSpec extends FunSpec } describe("A spark session") { + + it("should get scala version matched jars with livy.repl.jars") { + val testedJars = Seq( + "test_2.10-0.1.jar", + "local://dummy-path/test/test1_2.10-1.0.jar", + "file:///dummy-path/test/test2_2.11-1.0-SNAPSHOT.jar", + "hdfs:///dummy-path/test/test3.jar", + "non-jar", + "dummy.jar" + ) + val livyConf = new LivyConf(false) + .set(LivyConf.REPL_JARS, testedJars.mkString(",")) + .set(LivyConf.LIVY_SPARK_VERSION, "1.6.2") + .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10") + val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf) + assert(properties(LivyConf.SPARK_JARS).split(",").toSet === Set("test_2.10-0.1.jar", + "local://dummy-path/test/test1_2.10-1.0.jar", + "hdfs:///dummy-path/test/test3.jar", + "dummy.jar")) + + livyConf.set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.11") + val properties1 = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf) + assert(properties1(LivyConf.SPARK_JARS).split(",").toSet === Set( + "file:///dummy-path/test/test2_2.11-1.0-SNAPSHOT.jar", + "hdfs:///dummy-path/test/test3.jar", + "dummy.jar")) + } + + + it("should set rsc jars through livy conf") { + val rscJars = Set( + "dummy.jar", + "local:///dummy-path/dummy1.jar", + "file:///dummy-path/dummy2.jar", + "hdfs:///dummy-path/dummy3.jar") + val livyConf = new LivyConf(false) + .set(LivyConf.REPL_JARS, "dummy.jar") + .set(LivyConf.RSC_JARS, rscJars.mkString(",")) + .set(LivyConf.LIVY_SPARK_VERSION, "1.6.2") + .set(LivyConf.LIVY_SPARK_SCALA_VERSION, "2.10") + val properties = InteractiveSession.prepareBuilderProp(Map.empty, Spark(), livyConf) + // if livy.rsc.jars is configured in LivyConf, it should be passed to RSCConf. + properties(RSCConf.Entry.LIVY_JARS.key()).split(",").toSet === rscJars + + val rscJars1 = Set( + "foo.jar", + "local:///dummy-path/foo1.jar", + "file:///dummy-path/foo2.jar", + "hdfs:///dummy-path/foo3.jar") + val properties1 = InteractiveSession.prepareBuilderProp( + Map(RSCConf.Entry.LIVY_JARS.key() -> rscJars1.mkString(",")), Spark(), livyConf) + // if rsc jars are configured both in LivyConf and RSCConf, RSCConf should take precedence. + properties1(RSCConf.Entry.LIVY_JARS.key()).split(",").toSet === rscJars1 + } + it("should start in the idle state") { session = createSession() session.state should (be(a[SessionState.Starting]) or be(a[SessionState.Idle])) @@ -149,15 +204,32 @@ class InteractiveSessionSpec extends FunSpec )) result should equal (expectedResult) - session.state shouldBe a[SessionState.Idle] + eventually(timeout(10 seconds), interval(30 millis)) { + session.state shouldBe a[SessionState.Idle] + } + } + + withSession("should get statement progress along with statement result") { session => + val code = + """ + |from time import sleep + |sleep(3) + """.stripMargin + val statement = session.executeStatement(ExecuteRequest(code)) + statement.progress should be (0.0) + + eventually(timeout(10 seconds), interval(100 millis)) { + val s = session.getStatement(statement.id).get + s.state.get() shouldBe StatementState.Available + s.progress should be (1.0) + } } withSession("should error out the session if the interpreter dies") { session => - executeStatement("import os; os._exit(666)") - (session.state match { - case SessionState.Error(_) => true - case _ => false - }) should equal(true) + session.executeStatement(ExecuteRequest("import os; os._exit(666)")) + eventually(timeout(30 seconds), interval(100 millis)) { + session.state shouldBe a[SessionState.Error] + } } } @@ -168,7 +240,8 @@ class InteractiveSessionSpec extends FunSpec val mockClient = mock[RSCClient] when(mockClient.submit(any(classOf[PingJob]))).thenReturn(mock[JobHandle[Void]]) val m = - InteractiveRecoveryMetadata(78, None, "appTag", Spark(), null, None, Some(URI.create(""))) + InteractiveRecoveryMetadata( + 78, None, "appTag", Spark(), 0, null, None, Some(URI.create(""))) val s = InteractiveSession.recover(m, conf, sessionStore, None, Some(mockClient)) s.state shouldBe a[SessionState.Recovering] @@ -181,7 +254,8 @@ class InteractiveSessionSpec extends FunSpec it("should recover session to dead state if rscDriverUri is unknown") { val conf = new LivyConf() val sessionStore = mock[SessionStore] - val m = InteractiveRecoveryMetadata(78, Some("appId"), "appTag", Spark(), null, None, None) + val m = InteractiveRecoveryMetadata( + 78, Some("appId"), "appTag", Spark(), 0, null, None, None) val s = InteractiveSession.recover(m, conf, sessionStore, None) s.state shouldBe a[SessionState.Dead] diff --git a/server/src/test/scala/com/cloudera/livy/server/interactive/SessionHeartbeatSpec.scala b/server/src/test/scala/com/cloudera/livy/server/interactive/SessionHeartbeatSpec.scala new file mode 100644 index 000000000..0526a2a9a --- /dev/null +++ b/server/src/test/scala/com/cloudera/livy/server/interactive/SessionHeartbeatSpec.scala @@ -0,0 +1,88 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.server.interactive + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.mockito.Mockito.{never, verify, when} +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.mock.MockitoSugar.mock + +import com.cloudera.livy.LivyConf +import com.cloudera.livy.server.recovery.SessionStore +import com.cloudera.livy.sessions.{Session, SessionManager} +import com.cloudera.livy.sessions.Session.RecoveryMetadata + +class SessionHeartbeatSpec extends FunSpec with Matchers { + describe("SessionHeartbeat") { + class TestHeartbeat(override val heartbeatTimeout: FiniteDuration) extends SessionHeartbeat {} + + it("should not expire if heartbeat was never called.") { + val t = new TestHeartbeat(Duration.Zero) + t.heartbeatExpired shouldBe false + } + + it("should expire if time has elapsed.") { + val t = new TestHeartbeat(Duration.fromNanos(1)) + t.heartbeat() + eventually(timeout(2 nano), interval(1 nano)) { + t.heartbeatExpired shouldBe true + } + } + + it("should not expire if time hasn't elapsed.") { + val t = new TestHeartbeat(Duration.create(1, DAYS)) + t.heartbeat() + t.heartbeatExpired shouldBe false + } + } + + describe("SessionHeartbeatWatchdog") { + abstract class TestSession extends Session(0, null, null) with SessionHeartbeat {} + class TestWatchdog(conf: LivyConf) + extends SessionManager[TestSession, RecoveryMetadata]( + conf, + { _ => assert(false).asInstanceOf[TestSession] }, + mock[SessionStore], + "test", + Some(Seq.empty)) + with SessionHeartbeatWatchdog[TestSession, RecoveryMetadata] {} + + it("should delete only expired sessions") { + val expiredSession: TestSession = mock[TestSession] + when(expiredSession.id).thenReturn(0) + when(expiredSession.heartbeatExpired).thenReturn(true) + + val nonExpiredSession: TestSession = mock[TestSession] + when(nonExpiredSession.id).thenReturn(1) + when(nonExpiredSession.heartbeatExpired).thenReturn(false) + + val n = new TestWatchdog(new LivyConf()) + + n.register(expiredSession) + n.register(nonExpiredSession) + n.deleteExpiredSessions() + + verify(expiredSession).stop() + verify(nonExpiredSession, never).stop() + } + } +} diff --git a/server/src/test/scala/com/cloudera/livy/server/recovery/FileSystemStateStoreSpec.scala b/server/src/test/scala/com/cloudera/livy/server/recovery/FileSystemStateStoreSpec.scala index 935911b02..2b90a52e4 100644 --- a/server/src/test/scala/com/cloudera/livy/server/recovery/FileSystemStateStoreSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/server/recovery/FileSystemStateStoreSpec.scala @@ -82,9 +82,9 @@ class FileSystemStateStoreSpec extends FunSpec with LivyBaseUnitTestSuite { } test("600") test("400") - test("777") - test("770") - test("707") + test("677") + test("670") + test("607") } it("set should write with an intermediate file") { diff --git a/server/src/test/scala/com/cloudera/livy/sessions/MockSession.scala b/server/src/test/scala/com/cloudera/livy/sessions/MockSession.scala index dba60bf06..c276a5eb4 100644 --- a/server/src/test/scala/com/cloudera/livy/sessions/MockSession.scala +++ b/server/src/test/scala/com/cloudera/livy/sessions/MockSession.scala @@ -31,7 +31,5 @@ class MockSession(id: Int, owner: String, conf: LivyConf) extends Session(id, ow override def state: SessionState = SessionState.Idle() - override val timeout: Long = 0L - override def recoveryMetadata: RecoveryMetadata = RecoveryMetadata(0) } diff --git a/server/src/test/scala/com/cloudera/livy/sessions/SessionManagerSpec.scala b/server/src/test/scala/com/cloudera/livy/sessions/SessionManagerSpec.scala index 8195bf8f6..79f248269 100644 --- a/server/src/test/scala/com/cloudera/livy/sessions/SessionManagerSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/sessions/SessionManagerSpec.scala @@ -23,21 +23,24 @@ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.{Failure, Try} -import org.mockito.Mockito.{never, verify, when} +import org.mockito.Mockito.{doReturn, never, verify, when} import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.Eventually._ import org.scalatest.mock.MockitoSugar.mock import com.cloudera.livy.{LivyBaseUnitTestSuite, LivyConf} import com.cloudera.livy.server.batch.{BatchRecoveryMetadata, BatchSession} +import com.cloudera.livy.server.interactive.InteractiveSession import com.cloudera.livy.server.recovery.SessionStore import com.cloudera.livy.sessions.Session.RecoveryMetadata class SessionManagerSpec extends FunSpec with Matchers with LivyBaseUnitTestSuite { + implicit def executor: ExecutionContext = ExecutionContext.global + describe("SessionManager") { it("should garbage collect old sessions") { val livyConf = new LivyConf() - livyConf.set(SessionManager.SESSION_TIMEOUT, "100ms") + livyConf.set(LivyConf.SESSION_TIMEOUT, "100ms") val manager = new SessionManager[MockSession, RecoveryMetadata]( livyConf, { _ => assert(false).asInstanceOf[MockSession] }, @@ -51,6 +54,59 @@ class SessionManagerSpec extends FunSpec with Matchers with LivyBaseUnitTestSuit manager.get(session.id) should be(None) } } + + it("batch session should not be gc-ed until application is finished") { + val sessionId = 24 + val session = mock[BatchSession] + when(session.id).thenReturn(sessionId) + when(session.stop()).thenReturn(Future {}) + when(session.lastActivity).thenReturn(System.nanoTime()) + + val conf = new LivyConf().set(LivyConf.SESSION_STATE_RETAIN_TIME, "1s") + val sm = new BatchSessionManager(conf, mock[SessionStore], Some(Seq(session))) + testSessionGC(session, sm) + } + + it("interactive session should not gc-ed if session timeout check is off") { + val sessionId = 24 + val session = mock[InteractiveSession] + when(session.id).thenReturn(sessionId) + when(session.stop()).thenReturn(Future {}) + when(session.lastActivity).thenReturn(System.nanoTime()) + + val conf = new LivyConf().set(LivyConf.SESSION_TIMEOUT_CHECK, false) + .set(LivyConf.SESSION_STATE_RETAIN_TIME, "1s") + val sm = new InteractiveSessionManager(conf, mock[SessionStore], Some(Seq(session))) + testSessionGC(session, sm) + } + + def testSessionGC(session: Session, sm: SessionManager[_, _]): Unit = { + + def changeStateAndCheck(s: SessionState)(fn: SessionManager[_, _] => Unit): Unit = { + doReturn(s).when(session).state + Await.result(sm.collectGarbage(), Duration.Inf) + fn(sm) + } + + // Batch session should not be gc-ed when alive + for (s <- Seq(SessionState.Running(), + SessionState.Idle(), + SessionState.Recovering(), + SessionState.NotStarted(), + SessionState.Busy(), + SessionState.ShuttingDown())) { + changeStateAndCheck(s) { sm => sm.get(session.id) should be (Some(session)) } + } + + // Stopped session should be gc-ed after retained timeout + for (s <- Seq(SessionState.Error(), + SessionState.Success(), + SessionState.Dead())) { + eventually(timeout(30 seconds), interval(100 millis)) { + changeStateAndCheck(s) { sm => sm.get(session.id) should be (None) } + } + } + } } describe("BatchSessionManager") { diff --git a/server/src/test/scala/com/cloudera/livy/utils/LivySparkUtilsSuite.scala b/server/src/test/scala/com/cloudera/livy/utils/LivySparkUtilsSuite.scala index 528ea6524..9981fa241 100644 --- a/server/src/test/scala/com/cloudera/livy/utils/LivySparkUtilsSuite.scala +++ b/server/src/test/scala/com/cloudera/livy/utils/LivySparkUtilsSuite.scala @@ -47,6 +47,7 @@ class LivySparkUtilsSuite extends FunSuite with Matchers with LivyBaseUnitTestSu test("should support Spark 1.6") { testSparkVersion("1.6.0") testSparkVersion("1.6.1") + testSparkVersion("1.6.1-SNAPSHOT") testSparkVersion("1.6.2") testSparkVersion("1.6") testSparkVersion("1.6.3.2.5.0-12") @@ -56,15 +57,19 @@ class LivySparkUtilsSuite extends FunSuite with Matchers with LivyBaseUnitTestSu testSparkVersion("2.0.0") testSparkVersion("2.0.1") testSparkVersion("2.0.2") + testSparkVersion("2.0.3-SNAPSHOT") testSparkVersion("2.0.0.2.5.1.0-56") // LIVY-229 testSparkVersion("2.0") + testSparkVersion("2.1.0") + testSparkVersion("2.1.1") } - test("should not support Spark older than 1.6 or newer than 2.0") { + test("should not support Spark older than 1.6") { intercept[IllegalArgumentException] { testSparkVersion("1.4.0") } intercept[IllegalArgumentException] { testSparkVersion("1.5.0") } intercept[IllegalArgumentException] { testSparkVersion("1.5.1") } intercept[IllegalArgumentException] { testSparkVersion("1.5.2") } + intercept[IllegalArgumentException] { testSparkVersion("1.5.0-cdh5.6.1") } } test("should fail on bad version") { @@ -131,5 +136,6 @@ class LivySparkUtilsSuite extends FunSuite with Matchers with LivyBaseUnitTestSu sparkScalaVersion(formatSparkVersion("1.6.2"), None, livyConf) shouldBe "2.10" sparkScalaVersion(formatSparkVersion("2.0.0"), None, livyConf) shouldBe "2.11" sparkScalaVersion(formatSparkVersion("2.0.1"), None, livyConf) shouldBe "2.11" + sparkScalaVersion(formatSparkVersion("2.1.0"), None, livyConf) shouldBe "2.11" } } diff --git a/server/src/test/scala/com/cloudera/livy/utils/SparkYarnAppSpec.scala b/server/src/test/scala/com/cloudera/livy/utils/SparkYarnAppSpec.scala index 2f786213d..1d903677f 100644 --- a/server/src/test/scala/com/cloudera/livy/utils/SparkYarnAppSpec.scala +++ b/server/src/test/scala/com/cloudera/livy/utils/SparkYarnAppSpec.scala @@ -136,8 +136,12 @@ class SparkYarnAppSpec extends FunSpec with LivyBaseUnitTestSuite { Clock.withSleepMethod(mockSleep) { val mockYarnClient = mock[YarnClient] val mockSparkSubmit = mock[LineBufferedProcess] - val sparkSubmitLog = IndexedSeq("SPARK-SUBMIT", "LOG") - when(mockSparkSubmit.inputLines).thenReturn(sparkSubmitLog) + val sparkSubmitInfoLog = IndexedSeq("SPARK-SUBMIT", "LOG") + val sparkSubmitErrorLog = IndexedSeq("SPARK-SUBMIT", "error log") + val sparkSubmitLog = ("stdout: " +: sparkSubmitInfoLog) ++ + ("\nstderr: " +: sparkSubmitErrorLog) :+ "\nYARN Diagnostics: " + when(mockSparkSubmit.inputLines).thenReturn(sparkSubmitInfoLog) + when(mockSparkSubmit.errorLines).thenReturn(sparkSubmitErrorLog) val waitForCalledLatch = new CountDownLatch(1) when(mockSparkSubmit.waitFor()).thenAnswer(new Answer[Int]() { override def answer(invocation: InvocationOnMock): Int = { @@ -168,15 +172,18 @@ class SparkYarnAppSpec extends FunSpec with LivyBaseUnitTestSuite { it("can kill spark-submit while it's running") { Clock.withSleepMethod(mockSleep) { + val livyConf = new LivyConf() + livyConf.set(LivyConf.YARN_APP_LOOKUP_TIMEOUT, "0") + val mockYarnClient = mock[YarnClient] val mockSparkSubmit = mock[LineBufferedProcess] - when(mockSparkSubmit.exitValue()).thenReturn(1) val sparkSubmitRunningLatch = new CountDownLatch(1) // Simulate a running spark-submit - when(mockSparkSubmit.inputLines).thenAnswer(new Answer[Unit]() { - override def answer(invocation: InvocationOnMock): Unit = { + when(mockSparkSubmit.waitFor()).thenAnswer(new Answer[Int]() { + override def answer(invocation: InvocationOnMock): Int = { sparkSubmitRunningLatch.await() + 0 } }) @@ -190,6 +197,7 @@ class SparkYarnAppSpec extends FunSpec with LivyBaseUnitTestSuite { cleanupThread(app.yarnAppMonitorThread) { app.kill() verify(mockSparkSubmit, times(1)).destroy() + sparkSubmitRunningLatch.countDown() } } } diff --git a/test-lib/pom.xml b/test-lib/pom.xml index 0fd9e9033..b9a943243 100644 --- a/test-lib/pom.xml +++ b/test-lib/pom.xml @@ -21,12 +21,12 @@ com.cloudera.livy livy-main - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT com.cloudera.livy livy-test-lib - 0.3.0-SNAPSHOT + 0.4.0-SNAPSHOT jar @@ -85,7 +85,37 @@ true + + + org.codehaus.mojo + build-helper-maven-plugin + + + parse-spark-version + process-sources + + parse-version + + + spark + ${spark.version} + + + + add-spark2-source-code + process-sources + + add-source + + + + ${project.basedir}/src/main/spark${spark.majorVersion}/scala + ${project.basedir}/src/main/spark${spark.majorVersion}/java + + + + + - diff --git a/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/DatasetTest.java b/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/DatasetTest.java new file mode 100644 index 000000000..086609874 --- /dev/null +++ b/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/DatasetTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.test.jobs.spark2; + +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import com.cloudera.livy.Job; +import com.cloudera.livy.JobContext; + +public class DatasetTest implements Job { + + @Override + public Long call(JobContext jc) throws Exception { + SparkSession spark = jc.sparkSession(); + + JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3)).map( + new Function() { + public Row call(Integer integer) throws Exception { + return RowFactory.create(integer); + } + }); + StructType schema = DataTypes.createStructType(new StructField[] { + DataTypes.createStructField("value", DataTypes.IntegerType, false) + }); + + Dataset ds = spark.createDataFrame(rdd, schema); + + return ds.filter(new FilterFunction() { + @Override + public boolean call(Row row) throws Exception { + return row.getInt(0) >= 2; + } + }).count(); + } +} diff --git a/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/SparkSessionTest.java b/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/SparkSessionTest.java new file mode 100644 index 000000000..f6d1f2645 --- /dev/null +++ b/test-lib/src/main/spark2/java/com/cloudera/livy/test/jobs/spark2/SparkSessionTest.java @@ -0,0 +1,39 @@ +/* + * Licensed to Cloudera, Inc. under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Cloudera, Inc. licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.cloudera.livy.test.jobs.spark2; + +import java.util.Arrays; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +import com.cloudera.livy.Job; +import com.cloudera.livy.JobContext; + +public class SparkSessionTest implements Job { + + @Override + public Long call(JobContext jc) throws Exception { + // Make sure SparkSession and SparkContext is callable + SparkSession session = jc.sparkSession(); + + JavaSparkContext sc = new JavaSparkContext(session.sparkContext()); + return sc.parallelize(Arrays.asList(1, 2, 3)).count(); + } +}