You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by rm...@apache.org on 2018/04/27 19:30:05 UTC

[25/50] [abbrv] metron git commit: METRON-1449 Set Zookeeper URL for Stellar Running in Zeppelin Notebook (nickwallen) closes apache/metron#931

METRON-1449 Set Zookeeper URL for Stellar Running in Zeppelin Notebook (nickwallen) closes apache/metron#931


Project: http://git-wip-us.apache.org/repos/asf/metron/repo
Commit: http://git-wip-us.apache.org/repos/asf/metron/commit/ab4f8e65
Tree: http://git-wip-us.apache.org/repos/asf/metron/tree/ab4f8e65
Diff: http://git-wip-us.apache.org/repos/asf/metron/diff/ab4f8e65

Branch: refs/heads/feature/METRON-1416-upgrade-solr
Commit: ab4f8e65e13aaea3a10491290a7411b3b6dc955e
Parents: 0ab39a3
Author: nickwallen <ni...@nickallen.org>
Authored: Fri Apr 6 16:40:01 2018 -0400
Committer: nickallen <ni...@apache.org>
Committed: Fri Apr 6 16:40:01 2018 -0400

----------------------------------------------------------------------
 metron-platform/metron-management/pom.xml       |   1 -
 .../shell/DefaultStellarShellExecutor.java      |  32 +++---
 .../shell/DefaultStellarShellExecutorTest.java  |  11 ++
 metron-stellar/stellar-zeppelin/README.md       |  80 ++++----------
 metron-stellar/stellar-zeppelin/pom.xml         |  12 +++
 .../stellar/zeppelin/StellarInterpreter.java    |  95 ++++++++++-------
 .../zeppelin/StellarInterpreterProperty.java    |  79 ++++++++++++++
 .../StellarInterpreterPropertyTest.java         |  62 +++++++++++
 .../zeppelin/StellarInterpreterTest.java        |  60 ++++++++++-
 .../integration/ConfigUploadComponent.java      |  82 +++++++++++++++
 .../StellarInterpreterIntegrationTest.java      | 104 +++++++++++++++++++
 11 files changed, 501 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-platform/metron-management/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/pom.xml b/metron-platform/metron-management/pom.xml
index c185662..962fd46 100644
--- a/metron-platform/metron-management/pom.xml
+++ b/metron-platform/metron-management/pom.xml
@@ -183,7 +183,6 @@
             </exclusions>
         </dependency>
     </dependencies>
-
     <build>
         <plugins>
             <plugin>

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutor.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutor.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutor.java
index 3f2c495..781a0cf 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutor.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutor.java
@@ -52,6 +52,7 @@ import java.io.ByteArrayInputStream;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -271,10 +272,7 @@ public class DefaultStellarShellExecutor implements StellarShellExecutor {
       globals = (Map<String, Object>) capability.get();
 
     } else {
-      // if it does not exist, create it.  this creates the global config for the current stellar executor
-      // session only.  this does not change the global config maintained externally in zookeeper
-      globals = new HashMap<>();
-      getContext().addCapability(GLOBAL_CONFIG, () -> globals);
+      throw new IllegalStateException("'GLOBAL_CONFIG' is missing");
     }
 
     return globals;
@@ -344,21 +342,25 @@ public class DefaultStellarShellExecutor implements StellarShellExecutor {
    * @param zkClient An optional Zookeeper client.
    */
   private Context createContext(Properties properties, Optional<CuratorFramework> zkClient) throws Exception {
+    Context.Builder contextBuilder = new Context.Builder();
+    Map<String, Object> globals;
+    if (zkClient.isPresent()) {
 
-    Context.Builder contextBuilder = new Context.Builder()
-            .with(SHELL_VARIABLES, () -> variables)
-            .with(STELLAR_CONFIG, () -> properties);
+      // fetch globals from zookeeper
+      globals = fetchGlobalConfig(zkClient.get());
+      contextBuilder.with(ZOOKEEPER_CLIENT, () -> zkClient.get());
 
-    // load global configuration from zookeeper
-    if (zkClient.isPresent()) {
-      Map<String, Object> global = fetchGlobalConfig(zkClient.get());
-      contextBuilder
-              .with(GLOBAL_CONFIG, () -> global)
-              .with(ZOOKEEPER_CLIENT, () -> zkClient.get())
-              .with(STELLAR_CONFIG, () -> getStellarConfig(global, properties));
+    } else {
+
+      // use empty globals to allow a user to '%define' their own
+      globals = new HashMap<>();
     }
 
-    return contextBuilder.build();
+    return contextBuilder
+            .with(SHELL_VARIABLES, () -> variables)
+            .with(GLOBAL_CONFIG, () -> globals)
+            .with(STELLAR_CONFIG, () -> getStellarConfig(globals, properties))
+            .build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutorTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutorTest.java b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutorTest.java
index ebba84f..23b0204 100644
--- a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutorTest.java
+++ b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/common/shell/DefaultStellarShellExecutorTest.java
@@ -295,4 +295,15 @@ public class DefaultStellarShellExecutorTest {
     assertTrue(result.getValue().isPresent());
     assertEquals("", result.getValue().get());
   }
+
+  /**
+   * If the executor is initialized without a connection to Zookeeper, the globals should be
+   * defined, but empty.  This allows a user to '%define' their own with magic commands even
+   * without Zookeeper.
+   */
+  @Test
+  public void testEmptyGlobalsWithNoZookeeper() {
+    assertNotNull(executor.getGlobalConfig());
+    assertEquals(0, executor.getGlobalConfig().size());
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/README.md
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/README.md b/metron-stellar/stellar-zeppelin/README.md
index 1190658..60dcb3a 100644
--- a/metron-stellar/stellar-zeppelin/README.md
+++ b/metron-stellar/stellar-zeppelin/README.md
@@ -48,77 +48,27 @@ To install the Stellar Interpreter in your Apache Zeppelin installation, follow
     mvn clean install -DskipTests
     ```
 
-1. If you do not already have Zeppelin installed, [download and unpack Apache Zeppelin](https://zeppelin.apache.org/download.html).  The directory in which you unpack Zeppelin will be referred to as `$ZEPPELIN_HOME`.
-
-1. If Zeppelin was already installed, make sure that it is not running.
-
-1. Create a settings directory for the Stellar interpreter.
+1. If you do not already have Zeppelin installed, [download and unpack Apache Zeppelin](https://zeppelin.apache.org/download.html).  Then change directories to the root of your Zeppelin download.
 
     ```
-    mkdir $ZEPPELIN_HOME/interpreter/stellar
-    cat <<EOF > $ZEPPELIN_HOME/interpreter/stellar/interpreter-setting.json
-    [
-      {
-        "group": "stellar",
-        "name": "stellar",
-        "className": "org.apache.metron.stellar.zeppelin.StellarInterpreter",
-        "properties": {
-        }
-      }
-    ]
-    EOF
+    cd $ZEPPELIN_HOME
     ```
 
-1. Create a Zeppelin Site file (`$ZEPPELIN_HOME/conf/zeppelin-site.xml`).
-
-    ```
-    cp $ZEPPELIN_HOME/conf/zeppelin-site.xml.template $ZEPPELIN_HOME/conf/zeppelin-site.xml
-    ```
+1. Use Zeppelin's installation utility to install the Stellar Interpreter.
 
-1. In the Zeppelin site file, add `org.apache.metron.stellar.zeppelin.StellarInterpreter` to the comma-separated list of Zeppelin interpreters under the `zeppelin.interpreters` property.
+    If Zeppelin was already installed, make sure that it is stopped before running this command.  Update the version, '0.4.3' in the example below, to whatever is appropriate for your environment.
 
-    The property will likely look-like the following.
     ```
-    <property>
-      <name>zeppelin.interpreters</name>
-      <value>org.apache.zeppelin.spark.SparkInterpreter,org.apache.zeppelin.spark.PySparkInterpreter,org.apache.zeppelin.rinterpreter.RRepl,org.apache.zeppelin.rinterpreter.KnitR,org.apache.zeppelin.spark.SparkRInterpreter,org.apache.zeppelin.spark.SparkSqlInterpreter,org.apache.zeppelin.spark.DepInterpreter,org.apache.zeppelin.markdown.Markdown,org.apache.zeppelin.angular.AngularInterpreter,org.apache.zeppelin.shell.ShellInterpreter,org.apache.zeppelin.file.HDFSFileInterpreter,org.apache.zeppelin.flink.FlinkInterpreter,,org.apache.zeppelin.python.PythonInterpreter,org.apache.zeppelin.python.PythonInterpreterPandasSql,org.apache.zeppelin.python.PythonCondaInterpreter,org.apache.zeppelin.python.PythonDockerInterpreter,org.apache.zeppelin.lens.LensInterpreter,org.apache.zeppelin.ignite.IgniteInterpreter,org.apache.zeppelin.ignite.IgniteSqlInterpreter,org.apache.zeppelin.cassandra.CassandraInterpreter,org.apache.zeppelin.geode.GeodeOqlInterpreter,org.apache.zeppelin.postgresql.PostgreS
 qlInterpreter,org.apache.zeppelin.jdbc.JDBCInterpreter,org.apache.zeppelin.kylin.KylinInterpreter,org.apache.zeppelin.elasticsearch.ElasticsearchInterpreter,org.apache.zeppelin.scalding.ScaldingInterpreter,org.apache.zeppelin.alluxio.AlluxioInterpreter,org.apache.zeppelin.hbase.HbaseInterpreter,org.apache.zeppelin.livy.LivySparkInterpreter,org.apache.zeppelin.livy.LivyPySparkInterpreter,org.apache.zeppelin.livy.LivyPySpark3Interpreter,org.apache.zeppelin.livy.LivySparkRInterpreter,org.apache.zeppelin.livy.LivySparkSQLInterpreter,org.apache.zeppelin.bigquery.BigQueryInterpreter,org.apache.zeppelin.beam.BeamInterpreter,org.apache.zeppelin.pig.PigInterpreter,org.apache.zeppelin.pig.PigQueryInterpreter,org.apache.zeppelin.scio.ScioInterpreter,org.apache.metron.stellar.zeppelin.StellarInterpreter</value>
-      <description>Comma separated interpreter configurations. First interpreter become a default</description>
-    </property>
+    bin/install-interpreter.sh --name stellar --artifact org.apache.metron:stellar-zeppelin:0.4.3
     ```
 
 1. Start Zeppelin.  
 
     ```
-    $ZEPPELIN_HOME/bin/zeppelin-daemon.sh start
+    bin/zeppelin-daemon.sh start
     ```
 
-1. Navigate to Zeppelin running at [http://localhost:8080/](http://localhost:8080/).
-
-1. Register the Stellar interpreter in Zeppelin.
-
-    1. Click on the top-right menu item labelled "Anonymous" then choose "Interpreter" in the drop-down that opens.    
-
-1. Configure the Stellar interpreter.
-
-    1. Click on '**+ Create**' near the top-right.
-
-    1. Define the following values.
-        * **Interpreter Name** = `stellar`
-        * **Interpreter Group** = `stellar`
-
-    1. Under **Options**, set the following values.
-        * The interpreter will be instantiated **Per Note**  in **isolated** process.
-
-    1. Under **Dependencies**, define the following fields, then click the "+" icon.  Replace the Metron version as required.
-        * **Artifact** = `org.apache.metron:stellar-zeppelin:0.4.3`
-
-    1. Click "Save"
-
-1. Wait for the intrepreter to start.
-
-    1. Near the title '**stellar**', will be a status icon.  This will indicate that it is downloading the dependencies.  
-
-    1. Once the icon is shown as green, the interpreter is ready to work.
+1. Navigate to Zeppelin running at [http://localhost:8080/](http://localhost:8080/).  The Stellar Interpreter should be ready for use with a basic set of functions.
 
 Usage
 -----
@@ -141,19 +91,25 @@ Usage
 
 1. In the next block, check which functions are available to you.
 
+    When executing Stellar's magic functions, you must explicitly define which interpreter should be used in the code block.  If you define 'stellar' as the default interpreter when creating a notebook, then this is only required when using Stellar's magic functions.
+
     ```
+    %stellar
+
     %functions
     ```
 
     You will **only** 'see' the functions defined within `stellar-common` since that is the only library that we added to the interpreter.  
 
-1. To see how additional functions can be added, go back to the Stellar interpreter configuration and add another dependency as follows.
+1. Add additional Stellar functions to your session.
 
-    ```
-    org.apache.metron:metron-statistics:0.4.3
-    ```
+    1. Go back to the Stellar interpreter configuration and add another dependency as follows.
+
+        ```
+        org.apache.metron:metron-statistics:0.4.3
+        ```
 
-    Reload the Stellar interpreter and run `%functions` again.  You will see the additional functions defined within the `metron-statistics` project.
+    1. Go back to your notebook and run `%functions` again.  You will now see the additional functions defined within the `metron-statistics` project.
 
 1. Auto-completion is also available for Stellar expressions.  
 

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/pom.xml
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/pom.xml b/metron-stellar/stellar-zeppelin/pom.xml
index 7809342..732c62b 100644
--- a/metron-stellar/stellar-zeppelin/pom.xml
+++ b/metron-stellar/stellar-zeppelin/pom.xml
@@ -35,6 +35,18 @@
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-integration-test</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-common</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <version>${global_mockito_version}</version>

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreter.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreter.java b/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreter.java
index 58287dc..5a7a175 100644
--- a/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreter.java
+++ b/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreter.java
@@ -18,16 +18,6 @@
 
 package org.apache.metron.stellar.zeppelin;
 
-import static org.apache.zeppelin.interpreter.InterpreterResult.Code.ERROR;
-import static org.apache.zeppelin.interpreter.InterpreterResult.Code.SUCCESS;
-import static org.apache.zeppelin.interpreter.InterpreterResult.Type.TEXT;
-
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-import java.util.Properties;
-
 import org.apache.commons.lang3.exception.ExceptionUtils;
 import org.apache.metron.stellar.common.shell.DefaultStellarAutoCompleter;
 import org.apache.metron.stellar.common.shell.DefaultStellarShellExecutor;
@@ -41,6 +31,16 @@ import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.ERROR;
+import static org.apache.zeppelin.interpreter.InterpreterResult.Code.SUCCESS;
+import static org.apache.zeppelin.interpreter.InterpreterResult.Type.TEXT;
+
 /**
  * A Zeppelin Interpreter for Stellar.
  */
@@ -65,16 +65,21 @@ public class StellarInterpreter extends Interpreter {
 
   public StellarInterpreter(Properties properties) {
     super(properties);
-    this.autoCompleter = new DefaultStellarAutoCompleter();
   }
 
   @Override
   public void open() {
     try {
-      executor = createExecutor();
+      // create the auto-completer
+      this.autoCompleter = new DefaultStellarAutoCompleter();
+
+      // create the stellar executor
+      Properties props = getProperty();
+      this.executor = createExecutor(props);
 
     } catch (Exception e) {
       LOG.error("Unable to create a StellarShellExecutor", e);
+      throw new RuntimeException(e);
     }
   }
 
@@ -85,36 +90,55 @@ public class StellarInterpreter extends Interpreter {
 
   @Override
   public InterpreterResult interpret(final String input, InterpreterContext context) {
-    InterpreterResult result;
+    InterpreterResult result = new InterpreterResult(SUCCESS, TEXT, "");
+
     try {
 
-      // execute the input
-      StellarResult stellarResult = executor.execute(input);
-      if(stellarResult.isSuccess()) {
+      // allow separate expressions on each line
+      String[] expressions = input.split(System.lineSeparator());
+      for (String expression : expressions) {
+        result = execute(expression);
+      }
+
+    } catch(Throwable t){
 
-        // on success - if no result, use a blank value
-        Object value = stellarResult.getValue().orElse("");
-        String text = value.toString();
-        result = new InterpreterResult(SUCCESS, TEXT, text);
+      // unexpected exception
+      String message = getErrorMessage(Optional.of(t), input);
+      result = new InterpreterResult(ERROR, TEXT, message);
+    }
 
-      } else if(stellarResult.isError()) {
+    // result is from the last expression that was executed
+    return result;
+  }
 
-        // an error occurred
-        Optional<Throwable> e = stellarResult.getException();
-        String message = getErrorMessage(e, input);
-        result = new InterpreterResult(ERROR, TEXT, message);
+  /**
+   * Execute a single Stellar expression.
+   * @param expression The Stellar expression to execute.
+   * @return The result of execution.
+   */
+  private InterpreterResult execute(final String expression) {
+    InterpreterResult result;
 
-      } else {
+    // execute the expression
+    StellarResult stellarResult = executor.execute(expression);
+    if (stellarResult.isSuccess()) {
 
-        // should never happen
-        throw new IllegalStateException("Unexpected error. result=" + stellarResult);
-      }
+      // on success - if no result, use a blank value
+      Object value = stellarResult.getValue().orElse("");
+      String text = value.toString();
+      result = new InterpreterResult(SUCCESS, TEXT, text);
 
-    } catch(Throwable t) {
+    } else if (stellarResult.isError()) {
 
-      // unexpected exception
-      String message = getErrorMessage(Optional.of(t), input);
+      // an error occurred
+      Optional<Throwable> e = stellarResult.getException();
+      String message = getErrorMessage(e, expression);
       result = new InterpreterResult(ERROR, TEXT, message);
+
+    } else {
+
+      // should never happen
+      throw new IllegalStateException("Unexpected error. result=" + stellarResult);
     }
 
     return result;
@@ -176,10 +200,11 @@ public class StellarInterpreter extends Interpreter {
    * Create an executor that will run the Stellar code for the Zeppelin Notebook.
    * @return The stellar executor.
    */
-  private StellarShellExecutor createExecutor() throws Exception {
+  private StellarShellExecutor createExecutor(Properties properties) throws Exception {
 
-    Properties props = getProperty();
-    StellarShellExecutor executor = new DefaultStellarShellExecutor(props, Optional.empty());
+    // a zookeeper URL may be defined
+    String zookeeperURL = StellarInterpreterProperty.ZOOKEEPER_URL.get(properties, String.class);
+    StellarShellExecutor executor = new DefaultStellarShellExecutor(properties, Optional.ofNullable(zookeeperURL));
 
     // register the auto-completer to be notified
     executor.addSpecialListener((magic) -> autoCompleter.addCandidateFunction(magic.getCommand()));

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreterProperty.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreterProperty.java b/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreterProperty.java
new file mode 100644
index 0000000..7392219
--- /dev/null
+++ b/metron-stellar/stellar-zeppelin/src/main/java/org/apache/metron/stellar/zeppelin/StellarInterpreterProperty.java
@@ -0,0 +1,79 @@
+/*
+ * 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 org.apache.metron.stellar.zeppelin;
+
+import org.apache.metron.stellar.common.utils.ConversionUtils;
+
+import java.util.Map;
+
+/**
+ * Defines the properties that a user can define when configuring
+ * the Stellar Interpreter in Zeppelin.
+ */
+public enum StellarInterpreterProperty {
+
+  /**
+   * A property that defines the URL for connecting to Zookeeper.  By default this is empty.
+   */
+  ZOOKEEPER_URL("zookeeper.url", null);
+
+  /**
+   * The key or property name.
+   */
+  String key;
+
+  /**
+   * The default value of the property.
+   */
+  Object defaultValue;
+
+  StellarInterpreterProperty(String key, Object defaultValue) {
+    this.key = key;
+    this.defaultValue = defaultValue;
+  }
+
+  /**
+   * @return The key or name of the property.
+   */
+  public String getKey() {
+    return key;
+  }
+
+  /**
+   * @return The default value of the property.
+   */
+  public <T> T getDefault(Class<T> clazz) {
+    return ConversionUtils.convert(defaultValue, clazz);
+  }
+
+  /**
+   * Retrieves the property value from a map of properties.
+   * @param properties A map of properties.
+   * @return The value of the property within the map.
+   */
+  public <T> T get(Map<Object, Object> properties, Class<T> clazz) {
+    Object o = properties.getOrDefault(key, defaultValue);
+    return o == null ? null : ConversionUtils.convert(o, clazz);
+  }
+
+  @Override
+  public String toString() {
+    return key;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterPropertyTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterPropertyTest.java b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterPropertyTest.java
new file mode 100644
index 0000000..d474eaf
--- /dev/null
+++ b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterPropertyTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.metron.stellar.zeppelin;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static org.apache.metron.stellar.zeppelin.StellarInterpreterProperty.ZOOKEEPER_URL;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the StellarInterpreterProperty class.
+ */
+public class StellarInterpreterPropertyTest {
+
+  /**
+   * By defining the 'zookeeper.url' property a user is able to 'set' the Zookeeper URL.
+   */
+  @Test
+  public void testGet() {
+
+    // define the zookeeper URL property
+    final String expected = "zookeeper:2181";
+    Map<Object, Object> props = Collections.singletonMap("zookeeper.url", expected);
+
+    // should be able to get the zookeeper URL property from the properties
+    String actual = ZOOKEEPER_URL.get(props, String.class);
+    assertEquals(expected, actual);
+  }
+
+  /**
+   * The default value should be returned when the user does not defined a 'zookeeper.url'.
+   */
+  @Test
+  public void testGetWhenPropertyNotDefined() {
+
+    // the property is not defined
+    Map<Object, Object> props = Collections.singletonMap("foo", "bar");
+    String actual = ZOOKEEPER_URL.get(props, String.class);
+
+    // expect to get the default value since its not defined
+    String expected = ZOOKEEPER_URL.getDefault(String.class);
+    assertEquals(expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterTest.java b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterTest.java
index 363938e..c817747 100644
--- a/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterTest.java
+++ b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/StellarInterpreterTest.java
@@ -17,22 +17,29 @@
  */
 package org.apache.metron.stellar.zeppelin;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
 import com.google.common.collect.Iterables;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.metron.stellar.common.shell.VariableResult;
+import org.apache.metron.stellar.dsl.Context;
 import org.apache.zeppelin.interpreter.InterpreterContext;
 import org.apache.zeppelin.interpreter.InterpreterResult;
 import org.apache.zeppelin.interpreter.InterpreterResultMessage;
 import org.apache.zeppelin.interpreter.thrift.InterpreterCompletion;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 
+import static org.apache.metron.stellar.zeppelin.StellarInterpreterProperty.ZOOKEEPER_URL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
 /**
  * Tests the StellarInterpreter.
  */
@@ -189,4 +196,49 @@ public class StellarInterpreterTest {
     // expect no completions
     assertEquals(0, completions.size());
   }
+
+  /**
+   * No Zookeeper client connection should be made if the Zookeeper URL is not defined
+   */
+  @Test
+  public void testOpenWithNoZookeeperURL() {
+
+    // no zookeeper URL defined
+    Properties props = new Properties();
+
+    // open the interpreter
+    interpreter = new StellarInterpreter(props);
+    interpreter.open();
+
+    // no zookeeper client should be defined
+    Optional<Object> zk = interpreter.getExecutor().getContext().getCapability(Context.Capabilities.ZOOKEEPER_CLIENT, false);
+    assertFalse(zk.isPresent());
+  }
+
+  /**
+   * Ensure that we can run Stellar code in the interpreter.
+   */
+  @Test
+  public void testExecuteStellarMultipleLines() {
+
+    // multi-line input
+    String input =
+            "x := 2 + 2" + System.lineSeparator() +
+            "y := 4 + 4";
+    InterpreterResult result = interpreter.interpret(input, context);
+
+    // expect x == 4 and y == 8
+    Map<String, VariableResult> vars = interpreter.getExecutor().getState();
+    assertEquals(4, vars.get("x").getResult());
+    assertEquals(8, vars.get("y").getResult());
+
+    // validate the result
+    assertEquals(InterpreterResult.Code.SUCCESS, result.code());
+    assertEquals(1, result.message().size());
+
+    // the output is the result of only the 'last' expression
+    InterpreterResultMessage message = result.message().get(0);
+    assertEquals("8", message.getData());
+    assertEquals(InterpreterResult.Type.TEXT, message.getType());
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/ConfigUploadComponent.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/ConfigUploadComponent.java b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/ConfigUploadComponent.java
new file mode 100644
index 0000000..9257e62
--- /dev/null
+++ b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/ConfigUploadComponent.java
@@ -0,0 +1,82 @@
+/*
+ *
+ *  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 org.apache.metron.stellar.zeppelin.integration;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.imps.CuratorFrameworkState;
+import org.apache.metron.integration.InMemoryComponent;
+import org.apache.metron.integration.UnableToStartException;
+
+import java.util.Map;
+
+import static org.apache.metron.common.configuration.ConfigurationsUtils.getClient;
+import static org.apache.metron.common.configuration.ConfigurationsUtils.writeGlobalConfigToZookeeper;
+
+/**
+ * Uploads configuration to Zookeeper.
+ */
+public class ConfigUploadComponent implements InMemoryComponent {
+
+  private String zookeeperURL;
+  private Map<String, Object> globals;
+
+  @Override
+  public void start() throws UnableToStartException {
+    try {
+      upload();
+
+    } catch (Exception e) {
+      throw new UnableToStartException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void stop() {
+    // nothing to do
+  }
+
+  /**
+   * Uploads configuration to Zookeeper.
+   * @throws Exception
+   */
+  private void upload() throws Exception {
+    assert zookeeperURL != null;
+    try(CuratorFramework client = getClient(zookeeperURL)) {
+      if(client.getState() != CuratorFrameworkState.STARTED) {
+        client.start();
+      }
+
+      if (globals != null) {
+        writeGlobalConfigToZookeeper(globals, client);
+      }
+    }
+  }
+
+
+  public ConfigUploadComponent withZookeeperURL(String zookeeperURL) {
+    this.zookeeperURL = zookeeperURL;
+    return this;
+  }
+
+  public ConfigUploadComponent withGlobals(Map<String, Object> globals) {
+    this.globals = globals;
+    return this;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/ab4f8e65/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/StellarInterpreterIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/StellarInterpreterIntegrationTest.java b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/StellarInterpreterIntegrationTest.java
new file mode 100644
index 0000000..b6395eb
--- /dev/null
+++ b/metron-stellar/stellar-zeppelin/src/test/java/org/apache/metron/stellar/zeppelin/integration/StellarInterpreterIntegrationTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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 org.apache.metron.stellar.zeppelin.integration;
+
+import org.apache.metron.integration.BaseIntegrationTest;
+import org.apache.metron.integration.ComponentRunner;
+import org.apache.metron.integration.components.ZKServerComponent;
+import org.apache.metron.stellar.dsl.Context;
+import org.apache.metron.stellar.zeppelin.StellarInterpreter;
+import org.apache.zeppelin.interpreter.InterpreterContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static org.apache.metron.stellar.zeppelin.StellarInterpreterProperty.ZOOKEEPER_URL;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+/**
+ * An integration test for the StellarInterpreter.
+ */
+public class StellarInterpreterIntegrationTest extends BaseIntegrationTest {
+
+  private StellarInterpreter interpreter;
+  private InterpreterContext context;
+  private Properties properties;
+  private String zookeeperURL;
+  private ZKServerComponent zkServer;
+  private ComponentRunner runner;
+
+  @Before
+  public void setup() throws Exception {
+
+    // a component that uploads the global configuration
+    Map<String, Object> globals = new HashMap<>();
+    ConfigUploadComponent configUploader = new ConfigUploadComponent()
+            .withGlobals(globals);
+
+    // create zookeeper component
+    properties = new Properties();
+    zkServer = getZKServerComponent(properties);
+
+    // can only get the zookeeperUrl AFTER it has started
+    zkServer.withPostStartCallback((zk) -> {
+      zookeeperURL = zk.getConnectionString();
+      configUploader.withZookeeperURL(zookeeperURL);
+    });
+
+    // start the integration test components
+    runner = new ComponentRunner.Builder()
+            .withComponent("zk", zkServer)
+            .withComponent("config", configUploader)
+            .build();
+    runner.start();
+
+    context = mock(InterpreterContext.class);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    runner.stop();
+  }
+
+  /**
+   * A user should be able to define a Zookeeper URL as a property.  When this property
+   * is defined, a connection to Zookeeper is created and available in the Stellar session.
+   */
+  @Test
+  public void testOpenWithZookeeperURL() {
+
+    // define a zookeeper URL
+    Properties props = new Properties();
+    props.put(ZOOKEEPER_URL.toString(), zookeeperURL);
+
+    // open the interpreter
+    interpreter = new StellarInterpreter(props);
+    interpreter.open();
+
+    // a zookeeper client should be defined
+    Optional<Object> zk = interpreter.getExecutor().getContext().getCapability(Context.Capabilities.ZOOKEEPER_CLIENT, false);
+    assertTrue(zk.isPresent());
+  }
+
+}