You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@predictionio.apache.org by do...@apache.org on 2017/02/11 22:00:08 UTC

[01/10] incubator-predictionio git commit: Add support for Elasticsearch 5.x

Repository: incubator-predictionio
Updated Branches:
  refs/heads/feature/es5 [created] c64941b6e


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
new file mode 100644
index 0000000..0e3eec8
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
@@ -0,0 +1,151 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.MapWritable
+import org.apache.hadoop.io.Text
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.PEvents
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.elasticsearch.client.RestClient
+import org.elasticsearch.hadoop.mr.EsInputFormat
+import org.elasticsearch.spark._
+import org.joda.time.DateTime
+import java.io.IOException
+import org.apache.http.util.EntityUtils
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.entity.ContentType
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.ext.JodaTimeSerializers
+
+
+class ESPEvents(client: RestClient, config: StorageClientConfig, index: String)
+    extends PEvents {
+  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
+
+  // client is not used.
+  try client.close() catch {
+    case e: Exception =>
+      logger.error("Failed to close client.", e)
+  }
+
+  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
+    channelId.map { ch =>
+      s"${appId}_${ch}"
+    }.getOrElse {
+      s"${appId}"
+    }
+  }
+
+  def getESNodes(): String = {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
+    val schemes = config.properties.get("SCHEMES").
+      map(_.split(",").toSeq).getOrElse(Seq("http"))
+    (hosts, ports, schemes).zipped.map(
+      (h, p, s) => s"$s://$h:$p").mkString(",")
+  }
+
+  override def find(
+    appId: Int,
+    channelId: Option[Int] = None,
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None)(sc: SparkContext): RDD[Event] = {
+
+    val query = ESUtils.createEventQuery(
+      startTime, untilTime, entityType, entityId,
+      eventNames, targetEntityType, targetEntityId, None)
+
+    val estype = getEsType(appId, channelId)
+    val conf = new Configuration()
+    conf.set("es.resource", s"$index/$estype")
+    conf.set("es.query", query)
+    conf.set("es.nodes", getESNodes())
+
+    val rdd = sc.newAPIHadoopRDD(conf, classOf[EsInputFormat[Text, MapWritable]],
+      classOf[Text], classOf[MapWritable]).map {
+        case (key, doc) => {
+          ESEventsUtil.resultToEvent(key, doc, appId)
+        }
+      }
+
+    rdd
+  }
+
+  override def write(
+    events: RDD[Event],
+    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
+    val estype = getEsType(appId, channelId)
+    events.map { event =>
+      ESEventsUtil.eventToPut(event, appId)
+    }.saveToEs(s"$index/$estype")
+  }
+
+  override def delete(
+    eventIds: RDD[String],
+    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
+    val estype = getEsType(appId, channelId)
+    val restClient = ESUtils.createRestClient(config)
+    try {
+      eventIds.foreachPartition { iter =>
+        iter.foreach { eventId =>
+          try {
+            val json =
+              ("query" ->
+                ("term" ->
+                  ("eventId" -> eventId)))
+            val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+            val response = client.performRequest(
+              "POST",
+              s"/$index/$estype/_delete_by_query",
+              Map.empty[String, String].asJava)
+            val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+            val result = (jsonResponse \ "result").extract[String]
+            result match {
+              case "deleted" => true
+              case _ =>
+                logger.error(s"[$result] Failed to update $index/$estype:$eventId")
+                false
+            }
+          } catch {
+            case e: IOException =>
+              logger.error(s"Failed to update $index/$estype:$eventId", e)
+              false
+          }
+        }
+      }
+    } finally {
+      restClient.close()
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
index 5c9e170..c067f3a 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
@@ -15,50 +15,57 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import org.apache.http.Header
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.json4s.JsonDSL._
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.RestClient
 import org.json4s._
+import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
 
-class ESSequences(client: Client, config: StorageClientConfig, index: String) extends Logging {
+class ESSequences(client: RestClient, config: StorageClientConfig, index: String) extends Logging {
   implicit val formats = DefaultFormats
   private val estype = "sequences"
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    // val settingsJson =
-    //   ("number_of_shards" -> 1) ~
-    //   ("auto_expand_replicas" -> "0-all")
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val mappingJson =
-      (estype ->
-        ("_source" -> ("enabled" -> 0)) ~
-        ("_all" -> ("enabled" -> 0)) ~
-        ("_type" -> ("index" -> "no")) ~
-        ("enabled" -> 0))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(mappingJson))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0)))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def genNext(name: String): Int = {
     try {
-      val response = client.prepareIndex(index, estype, name).
-        setSource(compact(render("n" -> name))).get
-      response.getVersion().toInt
+      val entity = new NStringEntity(write("n" -> name), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$name",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+          (jsonResponse \ "_version").extract[Int]
+        case "updated" =>
+          (jsonResponse \ "_version").extract[Int]
+        case _ =>
+          throw new IllegalStateException(s"[$result] Failed to update $index/$estype/$name")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        0
+      case e: IOException =>
+        throw new StorageClientException(s"Failed to update $index/$estype/$name", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
index f5c99bf..68e3f57 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
@@ -15,34 +15,151 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import org.elasticsearch.action.search.SearchRequestBuilder
-import org.elasticsearch.client.Client
-import org.elasticsearch.common.unit.TimeValue
-import org.json4s.Formats
-import org.json4s.native.Serialization.read
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
 
-import scala.collection.mutable.ArrayBuffer
+import org.apache.http.entity.ContentType
+import org.apache.http.entity.StringEntity
+import org.apache.http.nio.entity.NStringEntity
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.apache.http.util.EntityUtils
+import org.joda.time.DateTime
+import org.joda.time.format.DateTimeFormat
+import org.joda.time.DateTimeZone
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.http.HttpHost
 
 object ESUtils {
-  val scrollLife = new TimeValue(60000)
+  val scrollLife = "1m"
 
-  def getAll[T : Manifest](
-      client: Client,
-      builder: SearchRequestBuilder)(
+  def getAll[T: Manifest](
+    client: RestClient,
+    index: String,
+    estype: String,
+    query: String)(
       implicit formats: Formats): Seq[T] = {
-    val results = ArrayBuffer[T]()
-    var response = builder.setScroll(scrollLife).get
-    var hits = response.getHits().hits()
-    results ++= hits.map(h => read[T](h.getSourceAsString))
-    while (hits.size > 0) {
-      response = client.prepareSearchScroll(response.getScrollId).
-        setScroll(scrollLife).get
-      hits = response.getHits().hits()
-      results ++= hits.map(h => read[T](h.getSourceAsString))
+
+    @scala.annotation.tailrec
+    def scroll(scrollId: String, hits: Seq[JValue], results: Seq[T]): Seq[T] = {
+      if (hits.isEmpty) results
+      else {
+        val json = ("scroll" -> scrollLife) ~ ("scroll_id" -> scrollId)
+        val scrollBody = new StringEntity(compact(render(json)))
+        val response = client.performRequest(
+          "POST",
+          "/_search/scroll",
+          Map[String, String](),
+          scrollBody)
+        val responseJValue = parse(EntityUtils.toString(response.getEntity))
+        scroll((responseJValue \ "_scroll_id").extract[String],
+          (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
+          hits.map(h => (h \ "_source").extract[T]) ++ results)
+      }
     }
-    results
+
+    val response = client.performRequest(
+      "POST",
+      s"/$index/$estype/_search",
+      Map("scroll" -> scrollLife),
+      new StringEntity(query))
+    val responseJValue = parse(EntityUtils.toString(response.getEntity))
+    scroll((responseJValue \ "_scroll_id").extract[String],
+        (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
+        Nil)
+  }
+
+  def createIndex(
+    client: RestClient,
+    index: String): Unit = {
+    client.performRequest(
+      "HEAD",
+      s"/$index",
+      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+        case 404 =>
+          client.performRequest(
+            "PUT",
+            s"/$index",
+            Map.empty[String, String].asJava)
+        case 200 =>
+        case _ =>
+          throw new IllegalStateException(s"/$index is invalid.")
+      }
+  }
+
+  def createMapping(
+    client: RestClient,
+    index: String,
+    estype: String,
+    json: String): Unit = {
+    client.performRequest(
+      "HEAD",
+      s"/$index/_mapping/$estype",
+      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+        case 404 =>
+          val entity = new NStringEntity(json, ContentType.APPLICATION_JSON)
+          client.performRequest(
+            "PUT",
+            s"/$index/_mapping/$estype",
+            Map.empty[String, String].asJava,
+            entity)
+        case 200 =>
+        case _ =>
+          throw new IllegalStateException(s"/$index/$estype is invalid: $json")
+      }
+  }
+
+  def createEventQuery(
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None,
+    reversed: Option[Boolean] = None): String = {
+    val mustQueries = Seq(
+      startTime.map(x => {
+        val v = DateTimeFormat
+          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
+        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
+      }),
+      untilTime.map(x => {
+        val v = DateTimeFormat
+          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
+        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
+      }),
+      entityType.map(x => s"""{"term":{"entityType":"${x}"}}"""),
+      entityId.map(x => s"""{"term":{"entityId":"${x}"}}"""),
+      targetEntityType.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityType":"${x}"}}""")),
+      targetEntityId.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityId":"${x}"}}""")),
+      eventNames
+        .map { xx => xx.map(x => "\"%s\"".format(x)) }
+        .map(x => s"""{"terms":{"event":[${x.mkString(",")}]}}""")).flatten.mkString(",")
+    val sortOrder = reversed.map(x => x match {
+      case true => "desc"
+      case _ => "asc"
+    })
+    s"""{
+       |"query":{"bool":{"must":[${mustQueries}]}},
+       |"sort":[{"eventTime":{"order":"${sortOrder}"}}]
+       |}""".stripMargin
+  }
+
+  def createRestClient(config: StorageClientConfig): RestClient = {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
+    val schemes = config.properties.get("SCHEMES").
+      map(_.split(",").toSeq).getOrElse(Seq("http"))
+    val httpHosts = (hosts, ports, schemes).zipped.map(
+      (h, p, s) => new HttpHost(h, p, s))
+    RestClient.builder(httpHosts: _*).build()
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
index 75ac2b0..912d467 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
@@ -15,36 +15,24 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
 import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.BaseStorageClient
 import org.apache.predictionio.data.storage.StorageClientConfig
 import org.apache.predictionio.data.storage.StorageClientException
-import org.elasticsearch.client.transport.TransportClient
-import org.elasticsearch.common.settings.ImmutableSettings
-import org.elasticsearch.common.transport.InetSocketTransportAddress
-import org.elasticsearch.transport.ConnectTransportException
+import java.net.InetAddress
+import org.elasticsearch.client.RestClient
+import org.apache.http.HttpHost
 
 class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
     with Logging {
   override val prefix = "ES"
+
   val client = try {
-    val hosts = config.properties.get("HOSTS").
-      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
-    val ports = config.properties.get("PORTS").
-      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9300))
-    val settings = ImmutableSettings.settingsBuilder()
-      .put("cluster.name", config.properties.getOrElse("CLUSTERNAME", "elasticsearch"))
-    val transportClient = new TransportClient(settings)
-    (hosts zip ports) foreach { hp =>
-      transportClient.addTransportAddress(
-        new InetSocketTransportAddress(hp._1, hp._2))
-    }
-    transportClient
+    ESUtils.createRestClient(config)
   } catch {
-    case e: ConnectTransportException =>
+    case e: Throwable =>
       throw new StorageClientException(e.getMessage, e)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/tools/build.sbt
----------------------------------------------------------------------
diff --git a/tools/build.sbt b/tools/build.sbt
index fefdb45..4e2b266 100644
--- a/tools/build.sbt
+++ b/tools/build.sbt
@@ -42,6 +42,7 @@ dependencyOverrides +=   "org.slf4j" % "slf4j-log4j12" % "1.7.18"
 assemblyMergeStrategy in assembly := {
   case PathList("META-INF", "LICENSE.txt") => MergeStrategy.concat
   case PathList("META-INF", "NOTICE.txt")  => MergeStrategy.concat
+  case PathList("org", "joda", "time", "base", "BaseDateTime.class") => MergeStrategy.first
   case x =>
     val oldStrategy = (assemblyMergeStrategy in assembly).value
     oldStrategy(x)
@@ -62,7 +63,8 @@ excludedJars in assembly <<= (fullClasspath in assembly) map { cp =>
 assemblyShadeRules in assembly := Seq(
   ShadeRule.rename("org.objenesis.**" -> "shadeio.@1").inLibrary("com.esotericsoftware.kryo" % "kryo" % "2.21").inProject,
   ShadeRule.rename("com.esotericsoftware.reflectasm.**" -> "shadeio.@1").inLibrary("com.esotericsoftware.kryo" % "kryo" % "2.21").inProject,
-  ShadeRule.rename("com.esotericsoftware.minlog.**" -> "shadeio.@1").inLibrary("com.esotericsoftware.kryo" % "kryo" % "2.21").inProject
+  ShadeRule.rename("com.esotericsoftware.minlog.**" -> "shadeio.@1").inLibrary("com.esotericsoftware.kryo" % "kryo" % "2.21").inProject,
+  ShadeRule.rename("org.apache.http.**" -> "shadeio.http.@1").inAll
 )
 
 // skip test in assembly


[07/10] incubator-predictionio git commit: Merge branch 'elasticsearch5-support' of https://github.com/haginot/incubator-predictionio into es5

Posted by do...@apache.org.
Merge branch 'elasticsearch5-support' of https://github.com/haginot/incubator-predictionio into es5


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

Branch: refs/heads/feature/es5
Commit: d039dda52a9dd550015016f41ee33a33851e3edd
Parents: 9b62980 89c9d26
Author: Donald Szeto <do...@apache.org>
Authored: Sat Feb 11 08:56:44 2017 -0800
Committer: Donald Szeto <do...@apache.org>
Committed: Sat Feb 11 08:56:44 2017 -0800

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 bin/install.sh                                  |   8 +-
 bin/pio-start-all                               |  99 ++++---
 bin/pio-stop-all                                |  50 ++--
 build.sbt                                       |   6 +-
 conf/pio-env.sh.template                        |  14 +-
 core/build.sbt                                  |  41 +--
 data/build.sbt                                  |  50 ++--
 .../storage/elasticsearch/ESAccessKeys.scala    | 154 ++++++----
 .../data/storage/elasticsearch/ESApps.scala     | 176 ++++++++----
 .../data/storage/elasticsearch/ESChannels.scala | 144 ++++++----
 .../elasticsearch/ESEngineInstances.scala       | 246 +++++++++++-----
 .../elasticsearch/ESEvaluationInstances.scala   | 186 +++++++-----
 .../storage/elasticsearch/ESEventsUtil.scala    | 125 ++++++++
 .../data/storage/elasticsearch/ESLEvents.scala  | 286 +++++++++++++++++++
 .../data/storage/elasticsearch/ESPEvents.scala  | 145 ++++++++++
 .../storage/elasticsearch/ESSequences.scala     |  71 +++--
 .../data/storage/elasticsearch/ESUtils.scala    | 157 ++++++++--
 .../storage/elasticsearch/StorageClient.scala   |  40 ++-
 .../storage/elasticsearch1/ESAccessKeys.scala   | 119 ++++++++
 .../data/storage/elasticsearch1/ESApps.scala    | 130 +++++++++
 .../storage/elasticsearch1/ESChannels.scala     | 117 ++++++++
 .../elasticsearch1/ESEngineInstances.scala      | 158 ++++++++++
 .../elasticsearch1/ESEngineManifests.scala      |  84 ++++++
 .../elasticsearch1/ESEvaluationInstances.scala  | 136 +++++++++
 .../storage/elasticsearch1/ESSequences.scala    |  64 +++++
 .../data/storage/elasticsearch1/ESUtils.scala   |  48 ++++
 .../storage/elasticsearch1/StorageClient.scala  |  50 ++++
 .../data/storage/elasticsearch1/package.scala   |  25 ++
 project/Build.scala                             |   3 +
 tests/Dockerfile                                |   6 +-
 tests/docker-files/env-conf/pio-env.sh          |  16 +-
 tests/run_docker.sh                             |   6 +-
 tools/build.sbt                                 |   3 +-
 34 files changed, 2474 insertions(+), 493 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d039dda5/bin/install.sh
----------------------------------------------------------------------
diff --cc bin/install.sh
index e485df9,2f7c162..ae1a7eb
--- a/bin/install.sh
+++ b/bin/install.sh
@@@ -18,9 -18,11 +18,10 @@@
  #
  
  OS=`uname`
- SPARK_VERSION=1.6.2
 -PIO_VERSION=0.11.0-SNAPSHOT
+ SPARK_VERSION=1.6.3
  # Looks like support for Elasticsearch 2.0 will require 2.0 so deferring
- ELASTICSEARCH_VERSION=1.7.5
+ #ELASTICSEARCH_VERSION=1.7.6
+ ELASTICSEARCH_VERSION=5.1.2
  HBASE_VERSION=1.2.2
  POSTGRES_VERSION=9.4-1204.jdbc41
  MYSQL_VERSION=5.1.37

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d039dda5/build.sbt
----------------------------------------------------------------------
diff --cc build.sbt
index fc09ee5,776b2ad..ce626ce
--- a/build.sbt
+++ b/build.sbt
@@@ -38,9 -40,9 +40,9 @@@ elasticsearch1Version in ThisBuild := "
  
  json4sVersion in ThisBuild := "3.2.10"
  
- sparkVersion in ThisBuild := "1.4.0"
+ sparkVersion in ThisBuild := "1.6.3"
  
 -lazy val pioBuildInfoSettings = buildInfoSettings ++ Seq(
 +val pioBuildInfoSettings = buildInfoSettings ++ Seq(
    sourceGenerators in Compile <+= buildInfo,
    buildInfoKeys := Seq[BuildInfoKey](
      name,

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d039dda5/tests/Dockerfile
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d039dda5/tools/build.sbt
----------------------------------------------------------------------


[09/10] incubator-predictionio git commit: [PIO-49] Rename Elasticsearch packages

Posted by do...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
deleted file mode 100644
index 7f50488..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.AccessKey
-import org.apache.predictionio.data.storage.AccessKeys
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.json4s.JsonDSL._
-import org.json4s._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-import scala.util.Random
-
-/** Elasticsearch implementation of AccessKeys. */
-class ESAccessKeys(client: Client, config: StorageClientConfig, index: String)
-    extends AccessKeys with Logging {
-  implicit val formats = DefaultFormats.lossless
-  private val estype = "accesskeys"
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("key" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("events" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
-
-  def insert(accessKey: AccessKey): Option[String] = {
-    val key = if (accessKey.key.isEmpty) generateKey else accessKey.key
-    update(accessKey.copy(key = key))
-    Some(key)
-  }
-
-  def get(key: String): Option[AccessKey] = {
-    try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        key).get()
-      Some(read[AccessKey](response.getSourceAsString))
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-      case e: NullPointerException => None
-    }
-  }
-
-  def getAll(): Seq[AccessKey] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[AccessKey](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[AccessKey]()
-    }
-  }
-
-  def getByAppid(appid: Int): Seq[AccessKey] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("appid", appid))
-      ESUtils.getAll[AccessKey](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[AccessKey]()
-    }
-  }
-
-  def update(accessKey: AccessKey): Unit = {
-    try {
-      client.prepareIndex(index, estype, accessKey.key).setSource(write(accessKey)).get()
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-    }
-  }
-
-  def delete(key: String): Unit = {
-    try {
-      client.prepareDelete(index, estype, key).get
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
deleted file mode 100644
index af61e17..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.App
-import org.apache.predictionio.data.storage.Apps
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.json4s.JsonDSL._
-import org.json4s._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-/** Elasticsearch implementation of Items. */
-class ESApps(client: Client, config: StorageClientConfig, index: String)
-  extends Apps with Logging {
-  implicit val formats = DefaultFormats.lossless
-  private val estype = "apps"
-  private val seq = new ESSequences(client, config, index)
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
-
-  def insert(app: App): Option[Int] = {
-    val id =
-      if (app.id == 0) {
-        var roll = seq.genNext("apps")
-        while (!get(roll).isEmpty) roll = seq.genNext("apps")
-        roll
-      }
-      else app.id
-    val realapp = app.copy(id = id)
-    update(realapp)
-    Some(id)
-  }
-
-  def get(id: Int): Option[App] = {
-    try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        id.toString).get()
-      Some(read[App](response.getSourceAsString))
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-      case e: NullPointerException => None
-    }
-  }
-
-  def getByName(name: String): Option[App] = {
-    try {
-      val response = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("name", name)).get
-      val hits = response.getHits().hits()
-      if (hits.size > 0) {
-        Some(read[App](hits.head.getSourceAsString))
-      } else {
-        None
-      }
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-    }
-  }
-
-  def getAll(): Seq[App] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[App](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[App]()
-    }
-  }
-
-  def update(app: App): Unit = {
-    try {
-      val response = client.prepareIndex(index, estype, app.id.toString).
-        setSource(write(app)).get()
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-    }
-  }
-
-  def delete(id: Int): Unit = {
-    try {
-      client.prepareDelete(index, estype, id.toString).get
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
deleted file mode 100644
index f955bee..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.Channel
-import org.apache.predictionio.data.storage.Channels
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders.termFilter
-import org.json4s.DefaultFormats
-import org.json4s.JsonDSL._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-class ESChannels(client: Client, config: StorageClientConfig, index: String)
-    extends Channels with Logging {
-
-  implicit val formats = DefaultFormats.lossless
-  private val estype = "channels"
-  private val seq = new ESSequences(client, config, index)
-  private val seqName = "channels"
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
-
-  def insert(channel: Channel): Option[Int] = {
-    val id =
-      if (channel.id == 0) {
-        var roll = seq.genNext(seqName)
-        while (!get(roll).isEmpty) roll = seq.genNext(seqName)
-        roll
-      } else channel.id
-
-    val realChannel = channel.copy(id = id)
-    if (update(realChannel)) Some(id) else None
-  }
-
-  def get(id: Int): Option[Channel] = {
-    try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        id.toString).get()
-      Some(read[Channel](response.getSourceAsString))
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-      case e: NullPointerException => None
-    }
-  }
-
-  def getByAppid(appid: Int): Seq[Channel] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("appid", appid))
-      ESUtils.getAll[Channel](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[Channel]()
-    }
-  }
-
-  def update(channel: Channel): Boolean = {
-    try {
-      val response = client.prepareIndex(index, estype, channel.id.toString).
-        setSource(write(channel)).get()
-      true
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        false
-    }
-  }
-
-  def delete(id: Int): Unit = {
-    try {
-      client.prepareDelete(index, estype, id.toString).get
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
deleted file mode 100644
index cc10ff0..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.EngineInstance
-import org.apache.predictionio.data.storage.EngineInstanceSerializer
-import org.apache.predictionio.data.storage.EngineInstances
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.elasticsearch.search.sort.SortOrder
-import org.json4s.JsonDSL._
-import org.json4s._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-class ESEngineInstances(client: Client, config: StorageClientConfig, index: String)
-  extends EngineInstances with Logging {
-  implicit val formats = DefaultFormats + new EngineInstanceSerializer
-  private val estype = "engine_instances"
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("startTime" -> ("type" -> "date")) ~
-          ("endTime" -> ("type" -> "date")) ~
-          ("engineId" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineVersion" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineVariant" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineFactory" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("batch" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("dataSourceParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("preparatorParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("algorithmsParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("servingParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
-
-  def insert(i: EngineInstance): String = {
-    try {
-      val response = client.prepareIndex(index, estype).
-        setSource(write(i)).get
-      response.getId
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        ""
-    }
-  }
-
-  def get(id: String): Option[EngineInstance] = {
-    try {
-      val response = client.prepareGet(index, estype, id).get
-      if (response.isExists) {
-        Some(read[EngineInstance](response.getSourceAsString))
-      } else {
-        None
-      }
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-    }
-  }
-
-  def getAll(): Seq[EngineInstance] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[EngineInstance](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
-    }
-  }
-
-  def getCompleted(
-      engineId: String,
-      engineVersion: String,
-      engineVariant: String): Seq[EngineInstance] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
-        andFilter(
-          termFilter("status", "COMPLETED"),
-          termFilter("engineId", engineId),
-          termFilter("engineVersion", engineVersion),
-          termFilter("engineVariant", engineVariant))).
-        addSort("startTime", SortOrder.DESC)
-      ESUtils.getAll[EngineInstance](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
-    }
-  }
-
-  def getLatestCompleted(
-      engineId: String,
-      engineVersion: String,
-      engineVariant: String): Option[EngineInstance] =
-    getCompleted(
-      engineId,
-      engineVersion,
-      engineVariant).headOption
-
-  def update(i: EngineInstance): Unit = {
-    try {
-      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
-    } catch {
-      case e: ElasticsearchException => error(e.getMessage)
-    }
-  }
-
-  def delete(id: String): Unit = {
-    try {
-      val response = client.prepareDelete(index, estype, id).get
-    } catch {
-      case e: ElasticsearchException => error(e.getMessage)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
deleted file mode 100644
index 307b582..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.EngineManifestSerializer
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.EngineManifest
-import org.apache.predictionio.data.storage.EngineManifests
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.json4s._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-class ESEngineManifests(client: Client, config: StorageClientConfig, index: String)
-  extends EngineManifests with Logging {
-  implicit val formats = DefaultFormats + new EngineManifestSerializer
-  private val estype = "engine_manifests"
-  private def esid(id: String, version: String) = s"$id $version"
-
-  def insert(engineManifest: EngineManifest): Unit = {
-    val json = write(engineManifest)
-    val response = client.prepareIndex(
-      index,
-      estype,
-      esid(engineManifest.id, engineManifest.version)).
-      setSource(json).execute().actionGet()
-  }
-
-  def get(id: String, version: String): Option[EngineManifest] = {
-    try {
-      val response = client.prepareGet(index, estype, esid(id, version)).
-        execute().actionGet()
-      if (response.isExists) {
-        Some(read[EngineManifest](response.getSourceAsString))
-      } else {
-        None
-      }
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-    }
-  }
-
-  def getAll(): Seq[EngineManifest] = {
-    try {
-      val builder = client.prepareSearch()
-      ESUtils.getAll[EngineManifest](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
-    }
-  }
-
-  def update(engineManifest: EngineManifest, upsert: Boolean = false): Unit =
-    insert(engineManifest)
-
-  def delete(id: String, version: String): Unit = {
-    try {
-      client.prepareDelete(index, estype, esid(id, version)).execute().actionGet()
-    } catch {
-      case e: ElasticsearchException => error(e.getMessage)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
deleted file mode 100644
index b8d7056..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.EvaluationInstance
-import org.apache.predictionio.data.storage.EvaluationInstanceSerializer
-import org.apache.predictionio.data.storage.EvaluationInstances
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.elasticsearch.search.sort.SortOrder
-import org.json4s.JsonDSL._
-import org.json4s._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
-import org.json4s.native.Serialization.write
-
-class ESEvaluationInstances(client: Client, config: StorageClientConfig, index: String)
-  extends EvaluationInstances with Logging {
-  implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
-  private val estype = "evaluation_instances"
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("startTime" -> ("type" -> "date")) ~
-          ("endTime" -> ("type" -> "date")) ~
-          ("evaluationClass" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineParamsGeneratorClass" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("batch" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("evaluatorResults" ->
-            ("type" -> "string") ~ ("index" -> "no")) ~
-          ("evaluatorResultsHTML" ->
-            ("type" -> "string") ~ ("index" -> "no")) ~
-          ("evaluatorResultsJSON" ->
-            ("type" -> "string") ~ ("index" -> "no"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
-
-  def insert(i: EvaluationInstance): String = {
-    try {
-      val response = client.prepareIndex(index, estype).
-        setSource(write(i)).get
-      response.getId
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        ""
-    }
-  }
-
-  def get(id: String): Option[EvaluationInstance] = {
-    try {
-      val response = client.prepareGet(index, estype, id).get
-      if (response.isExists) {
-        Some(read[EvaluationInstance](response.getSourceAsString))
-      } else {
-        None
-      }
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        None
-    }
-  }
-
-  def getAll(): Seq[EvaluationInstance] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[EvaluationInstance](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
-    }
-  }
-
-  def getCompleted(): Seq[EvaluationInstance] = {
-    try {
-      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
-        termFilter("status", "EVALCOMPLETED")).
-        addSort("startTime", SortOrder.DESC)
-      ESUtils.getAll[EvaluationInstance](client, builder)
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
-    }
-  }
-
-  def update(i: EvaluationInstance): Unit = {
-    try {
-      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
-    } catch {
-      case e: ElasticsearchException => error(e.getMessage)
-    }
-  }
-
-  def delete(id: String): Unit = {
-    try {
-      client.prepareDelete(index, estype, id).get
-    } catch {
-      case e: ElasticsearchException => error(e.getMessage)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
deleted file mode 100644
index 80247ec..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.json4s.JsonDSL._
-import org.json4s._
-import org.json4s.native.JsonMethods._
-
-class ESSequences(client: Client, config: StorageClientConfig, index: String) extends Logging {
-  implicit val formats = DefaultFormats
-  private val estype = "sequences"
-
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    // val settingsJson =
-    //   ("number_of_shards" -> 1) ~
-    //   ("auto_expand_replicas" -> "0-all")
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val mappingJson =
-      (estype ->
-        ("_source" -> ("enabled" -> 0)) ~
-        ("_all" -> ("enabled" -> 0)) ~
-        ("_type" -> ("index" -> "no")) ~
-        ("enabled" -> 0))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(mappingJson))).get
-  }
-
-  def genNext(name: String): Int = {
-    try {
-      val response = client.prepareIndex(index, estype, name).
-        setSource(compact(render("n" -> name))).get
-      response.getVersion().toInt
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        0
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
deleted file mode 100644
index 5de2999..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import org.elasticsearch.action.search.SearchRequestBuilder
-import org.elasticsearch.client.Client
-import org.elasticsearch.common.unit.TimeValue
-import org.json4s.Formats
-import org.json4s.native.Serialization.read
-
-import scala.collection.mutable.ArrayBuffer
-
-object ESUtils {
-  val scrollLife = new TimeValue(60000)
-
-  def getAll[T : Manifest](
-      client: Client,
-      builder: SearchRequestBuilder)(
-      implicit formats: Formats): Seq[T] = {
-    val results = ArrayBuffer[T]()
-    var response = builder.setScroll(scrollLife).get
-    var hits = response.getHits().hits()
-    results ++= hits.map(h => read[T](h.getSourceAsString))
-    while (hits.size > 0) {
-      response = client.prepareSearchScroll(response.getScrollId).
-        setScroll(scrollLife).get
-      hits = response.getHits().hits()
-      results ++= hits.map(h => read[T](h.getSourceAsString))
-    }
-    results
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
deleted file mode 100644
index 6f6b1c9..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch1
-
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.BaseStorageClient
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.StorageClientException
-import org.elasticsearch.client.transport.TransportClient
-import org.elasticsearch.common.settings.ImmutableSettings
-import org.elasticsearch.common.transport.InetSocketTransportAddress
-import org.elasticsearch.transport.ConnectTransportException
-
-class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
-    with Logging {
-  override val prefix = "ES"
-  val client = try {
-    val hosts = config.properties.get("HOSTS").
-      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
-    val ports = config.properties.get("PORTS").
-      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9300))
-    val settings = ImmutableSettings.settingsBuilder()
-      .put("cluster.name", config.properties.getOrElse("CLUSTERNAME", "elasticsearch"))
-    val transportClient = new TransportClient(settings)
-    (hosts zip ports) foreach { hp =>
-      transportClient.addTransportAddress(
-        new InetSocketTransportAddress(hp._1, hp._2))
-    }
-    transportClient
-  } catch {
-    case e: ConnectTransportException =>
-      throw new StorageClientException(e.getMessage, e)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
deleted file mode 100644
index d6aa24a..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.predictionio.data.storage
-
-/** Elasticsearch implementation of storage traits, supporting meta data only
-  *
-  * @group Implementation
-  */
-package object elasticsearch1 {}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESAccessKeys.scala
new file mode 100644
index 0000000..3cdfad9
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESAccessKeys.scala
@@ -0,0 +1,175 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.AccessKey
+import org.apache.predictionio.data.storage.AccessKeys
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+/** Elasticsearch implementation of AccessKeys. */
+class ESAccessKeys(client: ESClient, config: StorageClientConfig, index: String)
+    extends AccessKeys with Logging {
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "accesskeys"
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("key" -> ("type" -> "keyword")) ~
+          ("events" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def insert(accessKey: AccessKey): Option[String] = {
+    val key = if (accessKey.key.isEmpty) generateKey else accessKey.key
+    update(accessKey.copy(key = key))
+    Some(key)
+  }
+
+  def get(id: String): Option[AccessKey] = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[AccessKey])
+        case _ =>
+          None
+      }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/$key", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getAll(): Seq[AccessKey] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getByAppid(appid: Int): Seq[AccessKey] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("term" ->
+            ("appid" -> appid)))
+      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def update(accessKey: AccessKey): Unit = {
+    val id = accessKey.key
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write(accessKey), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def delete(id: String): Unit = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESApps.scala
new file mode 100644
index 0000000..99fa2f0
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESApps.scala
@@ -0,0 +1,194 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.App
+import org.apache.predictionio.data.storage.Apps
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+/** Elasticsearch implementation of Items. */
+class ESApps(client: ESClient, config: StorageClientConfig, index: String)
+    extends Apps with Logging {
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "apps"
+  private val seq = new ESSequences(client, config, index)
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("id" -> ("type" -> "keyword")) ~
+          ("name" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def insert(app: App): Option[Int] = {
+    val id =
+      if (app.id == 0) {
+        var roll = seq.genNext(estype)
+        while (!get(roll).isEmpty) roll = seq.genNext(estype)
+        roll
+      } else app.id
+    update(app.copy(id = id))
+    Some(id)
+  }
+
+  def get(id: Int): Option[App] = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[App])
+        case _ =>
+          None
+      }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getByName(name: String): Option[App] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("term" ->
+            ("name" -> name)))
+      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/_search",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "hits" \ "total").extract[Long] match {
+        case 0 => None
+        case _ =>
+          val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
+          val result = (results.head \ "_source").extract[App]
+          Some(result)
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getAll(): Seq[App] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[App](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def update(app: App): Unit = {
+    val id = app.id.toString
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write(app), ContentType.APPLICATION_JSON);
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def delete(id: Int): Unit = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESChannels.scala
new file mode 100644
index 0000000..b8ae5b4
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESChannels.scala
@@ -0,0 +1,165 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.Channel
+import org.apache.predictionio.data.storage.Channels
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESChannels(client: ESClient, config: StorageClientConfig, index: String)
+    extends Channels with Logging {
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "channels"
+  private val seq = new ESSequences(client, config, index)
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("name" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def insert(channel: Channel): Option[Int] = {
+    val id =
+      if (channel.id == 0) {
+        var roll = seq.genNext(estype)
+        while (!get(roll).isEmpty) roll = seq.genNext(estype)
+        roll
+      } else channel.id
+
+    if (update(channel.copy(id = id))) Some(id) else None
+  }
+
+  def get(id: Int): Option[Channel] = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[Channel])
+        case _ =>
+          None
+      }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getByAppid(appid: Int): Seq[Channel] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("term" ->
+            ("appid" -> appid)))
+      ESUtils.getAll[Channel](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def update(channel: Channel): Boolean = {
+    val id = channel.id.toString
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write(channel), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "created" => true
+        case "updated" => true
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+          false
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+        false
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def delete(id: Int): Unit = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEngineInstances.scala
new file mode 100644
index 0000000..d8d3c75
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEngineInstances.scala
@@ -0,0 +1,248 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.EngineInstance
+import org.apache.predictionio.data.storage.EngineInstanceSerializer
+import org.apache.predictionio.data.storage.EngineInstances
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESEngineInstances(client: ESClient, config: StorageClientConfig, index: String)
+    extends EngineInstances with Logging {
+  implicit val formats = DefaultFormats + new EngineInstanceSerializer
+  private val estype = "engine_instances"
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("status" -> ("type" -> "keyword")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("engineId" -> ("type" -> "keyword")) ~
+          ("engineVersion" -> ("type" -> "keyword")) ~
+          ("engineVariant" -> ("type" -> "keyword")) ~
+          ("engineFactory" -> ("type" -> "keyword")) ~
+          ("batch" -> ("type" -> "keyword")) ~
+          ("dataSourceParams" -> ("type" -> "keyword")) ~
+          ("preparatorParams" -> ("type" -> "keyword")) ~
+          ("algorithmsParams" -> ("type" -> "keyword")) ~
+          ("servingParams" -> ("type" -> "keyword")) ~
+          ("status" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def insert(i: EngineInstance): String = {
+    val id = i.id match {
+      case x if x.isEmpty =>
+        @scala.annotation.tailrec
+        def generateId(newId: Option[String]): String = {
+          newId match {
+            case Some(x) => x
+            case _ => generateId(preInsert())
+          }
+        }
+        generateId(preInsert())
+      case x => x
+    }
+
+    update(i.copy(id = id))
+    id
+  }
+
+  def preInsert(): Option[String] = {
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity("{}", ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+          Some((jsonResponse \ "_id").extract[String])
+        case _ =>
+          error(s"[$result] Failed to create $index/$estype")
+          None
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to create $index/$estype", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def get(id: String): Option[EngineInstance] = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[EngineInstance])
+        case _ =>
+          None
+      }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getAll(): Seq[EngineInstance] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getCompleted(
+    engineId: String,
+    engineVersion: String,
+    engineVariant: String): Seq[EngineInstance] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("bool" ->
+            ("must" -> List(
+              ("term" ->
+                ("status" -> "COMPLETED")),
+              ("term" ->
+                ("engineId" -> engineId)),
+              ("term" ->
+                ("engineVersion" -> engineVersion)),
+              ("term" ->
+                ("engineVariant" -> engineVariant)))))) ~
+              ("sort" -> List(
+                ("startTime" ->
+                  ("order" -> "desc"))))
+      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getLatestCompleted(
+    engineId: String,
+    engineVersion: String,
+    engineVariant: String): Option[EngineInstance] =
+    getCompleted(
+      engineId,
+      engineVersion,
+      engineVariant).headOption
+
+  def update(i: EngineInstance): Unit = {
+    val id = i.id
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def delete(id: String): Unit = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEvaluationInstances.scala
new file mode 100644
index 0000000..d2f8623
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEvaluationInstances.scala
@@ -0,0 +1,194 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.EvaluationInstance
+import org.apache.predictionio.data.storage.EvaluationInstanceSerializer
+import org.apache.predictionio.data.storage.EvaluationInstances
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESEvaluationInstances(client: ESClient, config: StorageClientConfig, index: String)
+    extends EvaluationInstances with Logging {
+  implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
+  private val estype = "evaluation_instances"
+  private val seq = new ESSequences(client, config, index)
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("status" -> ("type" -> "keyword")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("evaluationClass" -> ("type" -> "keyword")) ~
+          ("engineParamsGeneratorClass" -> ("type" -> "keyword")) ~
+          ("batch" -> ("type" -> "keyword")) ~
+          ("evaluatorResults" -> ("type" -> "text") ~ ("index" -> "no")) ~
+          ("evaluatorResultsHTML" -> ("type" -> "text") ~ ("index" -> "no")) ~
+          ("evaluatorResultsJSON" -> ("type" -> "text") ~ ("index" -> "no"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def insert(i: EvaluationInstance): String = {
+    val id = i.id match {
+      case x if x.isEmpty =>
+        var roll = seq.genNext(estype).toString
+        while (!get(roll).isEmpty) roll = seq.genNext(estype).toString
+        roll
+      case x => x
+    }
+
+    update(i.copy(id = id))
+    id
+  }
+
+  def get(id: String): Option[EvaluationInstance] = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[EvaluationInstance])
+        case _ =>
+          None
+      }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
+        None
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getAll(): Seq[EvaluationInstance] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def getCompleted(): Seq[EvaluationInstance] = {
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("term" ->
+            ("status" -> "EVALCOMPLETED"))) ~
+            ("sort" ->
+              ("startTime" ->
+                ("order" -> "desc")))
+      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
+    } catch {
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def update(i: EvaluationInstance): Unit = {
+    val id = i.id
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+
+  def delete(id: String): Unit = {
+    val restClient = client.open()
+    try {
+      val response = restClient.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEventsUtil.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEventsUtil.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEventsUtil.scala
new file mode 100644
index 0000000..3f70266
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESEventsUtil.scala
@@ -0,0 +1,125 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import org.apache.hadoop.io.DoubleWritable
+import org.apache.hadoop.io.LongWritable
+import org.apache.hadoop.io.MapWritable
+import org.apache.hadoop.io.Text
+import org.apache.predictionio.data.storage.DataMap
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.EventValidation
+import org.joda.time.DateTime
+import org.joda.time.DateTimeZone
+import org.json4s._
+
+object ESEventsUtil {
+
+  implicit val formats = DefaultFormats
+
+  def resultToEvent(id: Text, result: MapWritable, appId: Int): Event = {
+
+    def getStringCol(col: String): String = {
+      val r = result.get(new Text(col)).asInstanceOf[Text]
+      require(r != null,
+        s"Failed to get value for column ${col}. " +
+          s"StringBinary: ${r.getBytes()}.")
+
+      r.toString()
+    }
+
+    def getOptStringCol(col: String): Option[String] = {
+      val r = result.get(new Text(col))
+      if (r == null) {
+        None
+      } else {
+        Some(r.asInstanceOf[Text].toString())
+      }
+    }
+
+    val tmp = result
+      .get(new Text("properties")).asInstanceOf[MapWritable]
+      .get(new Text("fields")).asInstanceOf[MapWritable]
+      .get(new Text("rating"))
+
+    val rating =
+      if (tmp.isInstanceOf[DoubleWritable]) tmp.asInstanceOf[DoubleWritable]
+      else if (tmp.isInstanceOf[LongWritable]) {
+        new DoubleWritable(tmp.asInstanceOf[LongWritable].get().toDouble)
+      }
+      else null
+
+    val properties: DataMap =
+      if (rating != null) DataMap(s"""{"rating":${rating.get().toString}}""")
+      else DataMap()
+
+
+    val eventId = Some(getStringCol("eventId"))
+    val event = getStringCol("event")
+    val entityType = getStringCol("entityType")
+    val entityId = getStringCol("entityId")
+    val targetEntityType = getOptStringCol("targetEntityType")
+    val targetEntityId = getOptStringCol("targetEntityId")
+    val prId = getOptStringCol("prId")
+    val eventTimeZone = getOptStringCol("eventTimeZone")
+      .map(DateTimeZone.forID(_))
+      .getOrElse(EventValidation.defaultTimeZone)
+    val eventTime = new DateTime(
+      getStringCol("eventTime"), eventTimeZone)
+    val creationTimeZone = getOptStringCol("creationTimeZone")
+      .map(DateTimeZone.forID(_))
+      .getOrElse(EventValidation.defaultTimeZone)
+    val creationTime: DateTime = new DateTime(
+      getStringCol("creationTime"), creationTimeZone)
+
+
+    Event(
+      eventId = eventId,
+      event = event,
+      entityType = entityType,
+      entityId = entityId,
+      targetEntityType = targetEntityType,
+      targetEntityId = targetEntityId,
+      properties = properties,
+      eventTime = eventTime,
+      tags = Seq(),
+      prId = prId,
+      creationTime = creationTime
+    )
+  }
+
+  def eventToPut(event: Event, appId: Int): Seq[Map[String, Any]] = {
+    Seq(
+      Map(
+        "eventId" -> event.eventId,
+        "event" -> event.event,
+        "entityType" -> event.entityType,
+        "entityId" -> event.entityId,
+        "targetEntityType" -> event.targetEntityType,
+        "targetEntityId" -> event.targetEntityId,
+        "properties" -> event.properties,
+        "eventTime" -> event.eventTime,
+        "tags" -> event.tags,
+        "prId" -> event.prId,
+        "creationTime" -> event.creationTime
+      )
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESLEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESLEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESLEvents.scala
new file mode 100644
index 0000000..74aa4b5
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESLEvents.scala
@@ -0,0 +1,286 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.LEvents
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.joda.time.DateTime
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+import org.json4s.ext.JodaTimeSerializers
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESLEvents(val client: ESClient, config: StorageClientConfig, val index: String)
+    extends LEvents with Logging {
+  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
+  private val seq = new ESSequences(client, config, index)
+  private val seqName = "events"
+
+  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
+    channelId.map { ch =>
+      s"${appId}_${ch}"
+    }.getOrElse {
+      s"${appId}"
+    }
+  }
+
+  override def init(appId: Int, channelId: Option[Int] = None): Boolean = {
+    val estype = getEsType(appId, channelId)
+    val restClient = client.open()
+    try {
+      ESUtils.createIndex(restClient, index)
+      val json =
+        (estype ->
+          ("_all" -> ("enabled" -> 0)) ~
+          ("properties" ->
+            ("name" -> ("type" -> "keyword")) ~
+            ("eventId" -> ("type" -> "keyword")) ~
+            ("event" -> ("type" -> "keyword")) ~
+            ("entityType" -> ("type" -> "keyword")) ~
+            ("entityId" -> ("type" -> "keyword")) ~
+            ("targetEntityType" -> ("type" -> "keyword")) ~
+            ("targetEntityId" -> ("type" -> "keyword")) ~
+            ("properties" ->
+              ("type" -> "nested") ~
+              ("properties" ->
+                ("fields" -> ("type" -> "nested") ~
+                  ("properties" ->
+                    ("user" -> ("type" -> "long")) ~
+                    ("num" -> ("type" -> "long")))))) ~
+                    ("eventTime" -> ("type" -> "date")) ~
+                    ("tags" -> ("type" -> "keyword")) ~
+                    ("prId" -> ("type" -> "keyword")) ~
+                    ("creationTime" -> ("type" -> "date"))))
+      ESUtils.createMapping(restClient, index, estype, compact(render(json)))
+    } finally {
+      restClient.close()
+    }
+    true
+  }
+
+  override def remove(appId: Int, channelId: Option[Int] = None): Boolean = {
+    val estype = getEsType(appId, channelId)
+    val restClient = client.open()
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+      restClient.performRequest(
+        "POST",
+        s"/$index/$estype/_delete_by_query",
+        Map.empty[String, String].asJava,
+        entity).getStatusLine.getStatusCode match {
+          case 200 => true
+          case _ =>
+            error(s"Failed to remove $index/$estype")
+            false
+        }
+    } catch {
+      case e: Exception =>
+        error(s"Failed to remove $index/$estype", e)
+        false
+    } finally {
+      restClient.close()
+    }
+  }
+
+  override def close(): Unit = {
+    // nothing
+  }
+
+  override def futureInsert(
+    event: Event,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[String] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      val restClient = client.open()
+      try {
+        val id = event.eventId.getOrElse {
+          var roll = seq.genNext(seqName)
+          while (exists(restClient, estype, roll)) roll = seq.genNext(seqName)
+          roll.toString
+        }
+        val json = write(event.copy(eventId = Some(id)))
+        val entity = new NStringEntity(json, ContentType.APPLICATION_JSON);
+        val response = restClient.performRequest(
+          "POST",
+          s"/$index/$estype/$id",
+          Map.empty[String, String].asJava,
+          entity)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        val result = (jsonResponse \ "result").extract[String]
+        result match {
+          case "created" => id
+          case "updated" => id
+          case _ =>
+            error(s"[$result] Failed to update $index/$estype/$id")
+            ""
+        }
+      } catch {
+        case e: IOException =>
+          error(s"Failed to update $index/$estype/<id>", e)
+          ""
+      } finally {
+        restClient.close()
+      }
+    }
+  }
+
+  private def exists(restClient: RestClient, estype: String, id: Int): Boolean = {
+    try {
+      restClient.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+          case 200 => true
+          case _ => false
+        }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => false
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            false
+        }
+      case e: IOException =>
+        error(s"Failed to access to $index/$estype/$id", e)
+        false
+    }
+  }
+
+  override def futureGet(
+    eventId: String,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Option[Event]] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      val restClient = client.open()
+      try {
+        val json =
+          ("query" ->
+            ("term" ->
+              ("eventId" -> eventId)))
+        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+        val response = restClient.performRequest(
+          "POST",
+          s"/$index/$estype/_search",
+          Map.empty[String, String].asJava,
+          entity)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        (jsonResponse \ "hits" \ "total").extract[Long] match {
+          case 0 => None
+          case _ =>
+            val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
+            val result = (results.head \ "_source").extract[Event]
+            Some(result)
+        }
+      } catch {
+        case e: IOException =>
+          error("Failed to access to /$index/$estype/_search", e)
+          None
+      } finally {
+        restClient.close()
+      }
+    }
+  }
+
+  override def futureDelete(
+    eventId: String,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Boolean] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      val restClient = client.open()
+      try {
+        val json =
+          ("query" ->
+            ("term" ->
+              ("eventId" -> eventId)))
+        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+        val response = restClient.performRequest(
+          "POST",
+          s"/$index/$estype/_delete_by_query",
+          Map.empty[String, String].asJava)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        val result = (jsonResponse \ "result").extract[String]
+        result match {
+          case "deleted" => true
+          case _ =>
+            error(s"[$result] Failed to update $index/$estype:$eventId")
+            false
+        }
+      } catch {
+        case e: IOException =>
+          error(s"Failed to update $index/$estype:$eventId", e)
+          false
+      } finally {
+        restClient.close()
+      }
+    }
+  }
+
+  override def futureFind(
+    appId: Int,
+    channelId: Option[Int] = None,
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None,
+    limit: Option[Int] = None,
+    reversed: Option[Boolean] = None)
+    (implicit ec: ExecutionContext): Future[Iterator[Event]] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      val restClient = client.open()
+      try {
+        val query = ESUtils.createEventQuery(
+          startTime, untilTime, entityType, entityId,
+          eventNames, targetEntityType, targetEntityId, None)
+        ESUtils.getAll[Event](restClient, index, estype, query).toIterator
+      } catch {
+        case e: IOException =>
+          error(e.getMessage)
+          Iterator[Event]()
+      } finally {
+        restClient.close()
+      }
+    }
+  }
+
+}



[05/10] incubator-predictionio git commit: add elasticsearch 1.x support

Posted by do...@apache.org.
add elasticsearch 1.x support


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

Branch: refs/heads/feature/es5
Commit: bad2f0384d6d892eb576a6c2267bf88612a8eb66
Parents: 48e18b5
Author: Shinsuke Sugaya <sh...@yahoo.co.jp>
Authored: Wed Jan 18 16:29:15 2017 +0900
Committer: Shinsuke Sugaya <sh...@yahoo.co.jp>
Committed: Wed Jan 18 16:29:15 2017 +0900

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 bin/install.sh                                  |   4 +-
 bin/pio-start-all                               |   9 +-
 build.sbt                                       |   2 +
 conf/pio-env.sh.template                        |   6 +
 core/build.sbt                                  |   1 +
 data/build.sbt                                  |   1 +
 .../storage/elasticsearch1/ESAccessKeys.scala   | 119 ++++++++++++++
 .../data/storage/elasticsearch1/ESApps.scala    | 130 +++++++++++++++
 .../storage/elasticsearch1/ESChannels.scala     | 117 ++++++++++++++
 .../elasticsearch1/ESEngineInstances.scala      | 158 +++++++++++++++++++
 .../elasticsearch1/ESEngineManifests.scala      |  84 ++++++++++
 .../elasticsearch1/ESEvaluationInstances.scala  | 136 ++++++++++++++++
 .../storage/elasticsearch1/ESSequences.scala    |  64 ++++++++
 .../data/storage/elasticsearch1/ESUtils.scala   |  48 ++++++
 .../storage/elasticsearch1/StorageClient.scala  |  50 ++++++
 .../data/storage/elasticsearch1/package.scala   |  25 +++
 project/Build.scala                             |   3 +
 tests/Dockerfile                                |   2 +
 tests/docker-files/env-conf/pio-env.sh          |  16 +-
 tools/build.sbt                                 |   1 -
 21 files changed, 969 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 68dee42..4d62999 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -63,8 +63,8 @@ env:
   matrix:
     - BUILD_TYPE=Unit
     - BUILD_TYPE=Integration METADATA_REP=PGSQL EVENTDATA_REP=PGSQL MODELDATA_REP=PGSQL
-    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH EVENTDATA_REP=HBASE MODELDATA_REP=LOCALFS
-    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH EVENTDATA_REP=PGSQL MODELDATA_REP=HDFS
+    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH1 EVENTDATA_REP=HBASE MODELDATA_REP=LOCALFS
+    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH1 EVENTDATA_REP=PGSQL MODELDATA_REP=HDFS
 
 before_install:
   - unset SBT_OPTS JVM_OPTS

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/bin/install.sh
----------------------------------------------------------------------
diff --git a/bin/install.sh b/bin/install.sh
index e4fe220..2f7c162 100755
--- a/bin/install.sh
+++ b/bin/install.sh
@@ -21,6 +21,7 @@ OS=`uname`
 PIO_VERSION=0.11.0-SNAPSHOT
 SPARK_VERSION=1.6.3
 # Looks like support for Elasticsearch 2.0 will require 2.0 so deferring
+#ELASTICSEARCH_VERSION=1.7.6
 ELASTICSEARCH_VERSION=5.1.2
 HBASE_VERSION=1.2.2
 POSTGRES_VERSION=9.4-1204.jdbc41
@@ -352,7 +353,8 @@ installES() {
     fi
     if [[ ! -e elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz ]]; then
       echo "Downloading Elasticsearch..."
-      curl -O https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
+      #curl -O https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
+      curl -O https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
     fi
     tar zxf elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
     rm -rf ${elasticsearch_dir}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/bin/pio-start-all
----------------------------------------------------------------------
diff --git a/bin/pio-start-all b/bin/pio-start-all
index 03e10ae..87130fa 100755
--- a/bin/pio-start-all
+++ b/bin/pio-start-all
@@ -33,6 +33,11 @@ SOURCE_TYPE=$SOURCE_TYPE$PIO_STORAGE_REPOSITORIES_MODELDATA_SOURCE
 if [ `echo $SOURCE_TYPE | grep -i elasticsearch | wc -l` != 0 ] ; then
   echo "Starting Elasticsearch..."
   if [ -n "$PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME" ]; then
+    ELASTICSEARCH_HOME=$PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME
+  elif [ -n "$PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOME" ]; then
+    ELASTICSEARCH_HOME=$PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOME
+  fi
+  if [ -n "$ELASTICSEARCH_HOME" ]; then
     if [ -n "$JAVA_HOME" ]; then
       JPS=`$JAVA_HOME/bin/jps`
     else
@@ -44,7 +49,7 @@ if [ `echo $SOURCE_TYPE | grep -i elasticsearch | wc -l` != 0 ] ; then
       echo -e "\033[0;31mAborting...\033[0m"
       exit 1
     else
-      $PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME/bin/elasticsearch -d -p $PIO_HOME/es.pid
+      $ELASTICSEARCH_HOME/bin/elasticsearch -d -p $PIO_HOME/es.pid
     fi
   else
     echo -e "\033[0;31mPlease set PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME in conf/pio-env.sh, or in your environment.\033[0m"
@@ -91,7 +96,7 @@ if [ `echo $SOURCE_TYPE | grep -i pgsql | wc -l` != 0 ] ; then
 fi
 
 # PredictionIO Event Server
-echo "Waiting 10 seconds for HBase/Elasticsearch to fully initialize..."
+echo "Waiting 10 seconds for Storage Repositories to fully initialize..."
 sleep 10
 echo "Starting PredictionIO Event Server..."
 ${PIO_HOME}/bin/pio-daemon ${PIO_HOME}/eventserver.pid eventserver --ip 0.0.0.0

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/build.sbt
----------------------------------------------------------------------
diff --git a/build.sbt b/build.sbt
index 4d6a5b6..776b2ad 100644
--- a/build.sbt
+++ b/build.sbt
@@ -36,6 +36,8 @@ javacOptions in (ThisBuild, compile) ++= Seq("-source", "1.7", "-target", "1.7",
 
 elasticsearchVersion in ThisBuild := "5.1.2"
 
+elasticsearch1Version in ThisBuild := "1.7.6"
+
 json4sVersion in ThisBuild := "3.2.10"
 
 sparkVersion in ThisBuild := "1.6.3"

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/conf/pio-env.sh.template
----------------------------------------------------------------------
diff --git a/conf/pio-env.sh.template b/conf/pio-env.sh.template
index a2841e3..8f5d7b1 100644
--- a/conf/pio-env.sh.template
+++ b/conf/pio-env.sh.template
@@ -89,6 +89,12 @@ PIO_STORAGE_SOURCES_PGSQL_PASSWORD=pio
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-5.1.2
+# Elasticsearch 1.x Example
+# PIO_STORAGE_SOURCES_ELASTICSEARCH1_TYPE=elasticsearch1
+# PIO_STORAGE_SOURCES_ELASTICSEARCH1_CLUSTERNAME=<elasticsearch_cluster_name>
+# PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOSTS=localhost
+# PIO_STORAGE_SOURCES_ELASTICSEARCH1_PORTS=9300
+# PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOME=$PIO_HOME/vendors/elasticsearch-1.7.6
 
 # Local File System Example
 # PIO_STORAGE_SOURCES_LOCALFS_TYPE=localfs

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/core/build.sbt
----------------------------------------------------------------------
diff --git a/core/build.sbt b/core/build.sbt
index abd8e07..305075e 100644
--- a/core/build.sbt
+++ b/core/build.sbt
@@ -33,6 +33,7 @@ libraryDependencies ++= Seq(
   "org.apache.spark"        %% "spark-sql"        % sparkVersion.value % "provided",
   "org.clapper"             %% "grizzled-slf4j"   % "1.0.2",
   "org.elasticsearch.client" % "rest"             % elasticsearchVersion.value,
+  "org.elasticsearch"        % "elasticsearch"    % elasticsearch1Version.value,
   "org.json4s"              %% "json4s-native"    % json4sVersion.value,
   "org.json4s"              %% "json4s-ext"       % json4sVersion.value,
   "org.scalaj"              %% "scalaj-http"      % "1.1.6",

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/build.sbt
----------------------------------------------------------------------
diff --git a/data/build.sbt b/data/build.sbt
index 4ae9b42..75d3c09 100644
--- a/data/build.sbt
+++ b/data/build.sbt
@@ -44,6 +44,7 @@ libraryDependencies ++= Seq(
   "org.apache.spark"        %% "spark-sql"      % sparkVersion.value % "provided",
   "org.clapper"             %% "grizzled-slf4j" % "1.0.2",
   "org.elasticsearch.client" % "rest"           % elasticsearchVersion.value,
+  "org.elasticsearch"        % "elasticsearch"  % elasticsearch1Version.value,
   "org.elasticsearch"        % "elasticsearch-spark-13_2.10" % elasticsearchVersion.value % "provided",
   "org.elasticsearch"        % "elasticsearch-hadoop-mr" % elasticsearchVersion.value,
   "org.json4s"              %% "json4s-native"  % json4sVersion.value,

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
new file mode 100644
index 0000000..7f50488
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESAccessKeys.scala
@@ -0,0 +1,119 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.AccessKey
+import org.apache.predictionio.data.storage.AccessKeys
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.json4s.JsonDSL._
+import org.json4s._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+import scala.util.Random
+
+/** Elasticsearch implementation of AccessKeys. */
+class ESAccessKeys(client: Client, config: StorageClientConfig, index: String)
+    extends AccessKeys with Logging {
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "accesskeys"
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
+      (estype ->
+        ("properties" ->
+          ("key" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("events" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
+  }
+
+  def insert(accessKey: AccessKey): Option[String] = {
+    val key = if (accessKey.key.isEmpty) generateKey else accessKey.key
+    update(accessKey.copy(key = key))
+    Some(key)
+  }
+
+  def get(key: String): Option[AccessKey] = {
+    try {
+      val response = client.prepareGet(
+        index,
+        estype,
+        key).get()
+      Some(read[AccessKey](response.getSourceAsString))
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+      case e: NullPointerException => None
+    }
+  }
+
+  def getAll(): Seq[AccessKey] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[AccessKey](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[AccessKey]()
+    }
+  }
+
+  def getByAppid(appid: Int): Seq[AccessKey] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("appid", appid))
+      ESUtils.getAll[AccessKey](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[AccessKey]()
+    }
+  }
+
+  def update(accessKey: AccessKey): Unit = {
+    try {
+      client.prepareIndex(index, estype, accessKey.key).setSource(write(accessKey)).get()
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+    }
+  }
+
+  def delete(key: String): Unit = {
+    try {
+      client.prepareDelete(index, estype, key).get
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
new file mode 100644
index 0000000..af61e17
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESApps.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.App
+import org.apache.predictionio.data.storage.Apps
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.json4s.JsonDSL._
+import org.json4s._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+/** Elasticsearch implementation of Items. */
+class ESApps(client: Client, config: StorageClientConfig, index: String)
+  extends Apps with Logging {
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "apps"
+  private val seq = new ESSequences(client, config, index)
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
+      (estype ->
+        ("properties" ->
+          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
+  }
+
+  def insert(app: App): Option[Int] = {
+    val id =
+      if (app.id == 0) {
+        var roll = seq.genNext("apps")
+        while (!get(roll).isEmpty) roll = seq.genNext("apps")
+        roll
+      }
+      else app.id
+    val realapp = app.copy(id = id)
+    update(realapp)
+    Some(id)
+  }
+
+  def get(id: Int): Option[App] = {
+    try {
+      val response = client.prepareGet(
+        index,
+        estype,
+        id.toString).get()
+      Some(read[App](response.getSourceAsString))
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+      case e: NullPointerException => None
+    }
+  }
+
+  def getByName(name: String): Option[App] = {
+    try {
+      val response = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("name", name)).get
+      val hits = response.getHits().hits()
+      if (hits.size > 0) {
+        Some(read[App](hits.head.getSourceAsString))
+      } else {
+        None
+      }
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+    }
+  }
+
+  def getAll(): Seq[App] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[App](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[App]()
+    }
+  }
+
+  def update(app: App): Unit = {
+    try {
+      val response = client.prepareIndex(index, estype, app.id.toString).
+        setSource(write(app)).get()
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+    }
+  }
+
+  def delete(id: Int): Unit = {
+    try {
+      client.prepareDelete(index, estype, id.toString).get
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
new file mode 100644
index 0000000..f955bee
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESChannels.scala
@@ -0,0 +1,117 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.Channel
+import org.apache.predictionio.data.storage.Channels
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders.termFilter
+import org.json4s.DefaultFormats
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+class ESChannels(client: Client, config: StorageClientConfig, index: String)
+    extends Channels with Logging {
+
+  implicit val formats = DefaultFormats.lossless
+  private val estype = "channels"
+  private val seq = new ESSequences(client, config, index)
+  private val seqName = "channels"
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
+      (estype ->
+        ("properties" ->
+          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
+  }
+
+  def insert(channel: Channel): Option[Int] = {
+    val id =
+      if (channel.id == 0) {
+        var roll = seq.genNext(seqName)
+        while (!get(roll).isEmpty) roll = seq.genNext(seqName)
+        roll
+      } else channel.id
+
+    val realChannel = channel.copy(id = id)
+    if (update(realChannel)) Some(id) else None
+  }
+
+  def get(id: Int): Option[Channel] = {
+    try {
+      val response = client.prepareGet(
+        index,
+        estype,
+        id.toString).get()
+      Some(read[Channel](response.getSourceAsString))
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+      case e: NullPointerException => None
+    }
+  }
+
+  def getByAppid(appid: Int): Seq[Channel] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("appid", appid))
+      ESUtils.getAll[Channel](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[Channel]()
+    }
+  }
+
+  def update(channel: Channel): Boolean = {
+    try {
+      val response = client.prepareIndex(index, estype, channel.id.toString).
+        setSource(write(channel)).get()
+      true
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        false
+    }
+  }
+
+  def delete(id: Int): Unit = {
+    try {
+      client.prepareDelete(index, estype, id.toString).get
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
new file mode 100644
index 0000000..cc10ff0
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineInstances.scala
@@ -0,0 +1,158 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.EngineInstance
+import org.apache.predictionio.data.storage.EngineInstanceSerializer
+import org.apache.predictionio.data.storage.EngineInstances
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.elasticsearch.search.sort.SortOrder
+import org.json4s.JsonDSL._
+import org.json4s._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+class ESEngineInstances(client: Client, config: StorageClientConfig, index: String)
+  extends EngineInstances with Logging {
+  implicit val formats = DefaultFormats + new EngineInstanceSerializer
+  private val estype = "engine_instances"
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
+      (estype ->
+        ("properties" ->
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("engineId" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineVersion" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineVariant" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineFactory" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("batch" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("dataSourceParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("preparatorParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("algorithmsParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("servingParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
+  }
+
+  def insert(i: EngineInstance): String = {
+    try {
+      val response = client.prepareIndex(index, estype).
+        setSource(write(i)).get
+      response.getId
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        ""
+    }
+  }
+
+  def get(id: String): Option[EngineInstance] = {
+    try {
+      val response = client.prepareGet(index, estype, id).get
+      if (response.isExists) {
+        Some(read[EngineInstance](response.getSourceAsString))
+      } else {
+        None
+      }
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+    }
+  }
+
+  def getAll(): Seq[EngineInstance] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[EngineInstance](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
+    }
+  }
+
+  def getCompleted(
+      engineId: String,
+      engineVersion: String,
+      engineVariant: String): Seq[EngineInstance] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
+        andFilter(
+          termFilter("status", "COMPLETED"),
+          termFilter("engineId", engineId),
+          termFilter("engineVersion", engineVersion),
+          termFilter("engineVariant", engineVariant))).
+        addSort("startTime", SortOrder.DESC)
+      ESUtils.getAll[EngineInstance](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
+    }
+  }
+
+  def getLatestCompleted(
+      engineId: String,
+      engineVersion: String,
+      engineVariant: String): Option[EngineInstance] =
+    getCompleted(
+      engineId,
+      engineVersion,
+      engineVariant).headOption
+
+  def update(i: EngineInstance): Unit = {
+    try {
+      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
+    } catch {
+      case e: ElasticsearchException => error(e.getMessage)
+    }
+  }
+
+  def delete(id: String): Unit = {
+    try {
+      val response = client.prepareDelete(index, estype, id).get
+    } catch {
+      case e: ElasticsearchException => error(e.getMessage)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
new file mode 100644
index 0000000..307b582
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEngineManifests.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.EngineManifestSerializer
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.EngineManifest
+import org.apache.predictionio.data.storage.EngineManifests
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.json4s._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+class ESEngineManifests(client: Client, config: StorageClientConfig, index: String)
+  extends EngineManifests with Logging {
+  implicit val formats = DefaultFormats + new EngineManifestSerializer
+  private val estype = "engine_manifests"
+  private def esid(id: String, version: String) = s"$id $version"
+
+  def insert(engineManifest: EngineManifest): Unit = {
+    val json = write(engineManifest)
+    val response = client.prepareIndex(
+      index,
+      estype,
+      esid(engineManifest.id, engineManifest.version)).
+      setSource(json).execute().actionGet()
+  }
+
+  def get(id: String, version: String): Option[EngineManifest] = {
+    try {
+      val response = client.prepareGet(index, estype, esid(id, version)).
+        execute().actionGet()
+      if (response.isExists) {
+        Some(read[EngineManifest](response.getSourceAsString))
+      } else {
+        None
+      }
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+    }
+  }
+
+  def getAll(): Seq[EngineManifest] = {
+    try {
+      val builder = client.prepareSearch()
+      ESUtils.getAll[EngineManifest](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
+    }
+  }
+
+  def update(engineManifest: EngineManifest, upsert: Boolean = false): Unit =
+    insert(engineManifest)
+
+  def delete(id: String, version: String): Unit = {
+    try {
+      client.prepareDelete(index, estype, esid(id, version)).execute().actionGet()
+    } catch {
+      case e: ElasticsearchException => error(e.getMessage)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
new file mode 100644
index 0000000..b8d7056
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESEvaluationInstances.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.EvaluationInstance
+import org.apache.predictionio.data.storage.EvaluationInstanceSerializer
+import org.apache.predictionio.data.storage.EvaluationInstances
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.elasticsearch.search.sort.SortOrder
+import org.json4s.JsonDSL._
+import org.json4s._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.json4s.native.Serialization.write
+
+class ESEvaluationInstances(client: Client, config: StorageClientConfig, index: String)
+  extends EvaluationInstances with Logging {
+  implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
+  private val estype = "evaluation_instances"
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
+      (estype ->
+        ("properties" ->
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("evaluationClass" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineParamsGeneratorClass" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("batch" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("evaluatorResults" ->
+            ("type" -> "string") ~ ("index" -> "no")) ~
+          ("evaluatorResultsHTML" ->
+            ("type" -> "string") ~ ("index" -> "no")) ~
+          ("evaluatorResultsJSON" ->
+            ("type" -> "string") ~ ("index" -> "no"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
+  }
+
+  def insert(i: EvaluationInstance): String = {
+    try {
+      val response = client.prepareIndex(index, estype).
+        setSource(write(i)).get
+      response.getId
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        ""
+    }
+  }
+
+  def get(id: String): Option[EvaluationInstance] = {
+    try {
+      val response = client.prepareGet(index, estype, id).get
+      if (response.isExists) {
+        Some(read[EvaluationInstance](response.getSourceAsString))
+      } else {
+        None
+      }
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        None
+    }
+  }
+
+  def getAll(): Seq[EvaluationInstance] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[EvaluationInstance](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
+    }
+  }
+
+  def getCompleted(): Seq[EvaluationInstance] = {
+    try {
+      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
+        termFilter("status", "EVALCOMPLETED")).
+        addSort("startTime", SortOrder.DESC)
+      ESUtils.getAll[EvaluationInstance](client, builder)
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
+    }
+  }
+
+  def update(i: EvaluationInstance): Unit = {
+    try {
+      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
+    } catch {
+      case e: ElasticsearchException => error(e.getMessage)
+    }
+  }
+
+  def delete(id: String): Unit = {
+    try {
+      client.prepareDelete(index, estype, id).get
+    } catch {
+      case e: ElasticsearchException => error(e.getMessage)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
new file mode 100644
index 0000000..80247ec
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESSequences.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.json4s.JsonDSL._
+import org.json4s._
+import org.json4s.native.JsonMethods._
+
+class ESSequences(client: Client, config: StorageClientConfig, index: String) extends Logging {
+  implicit val formats = DefaultFormats
+  private val estype = "sequences"
+
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    // val settingsJson =
+    //   ("number_of_shards" -> 1) ~
+    //   ("auto_expand_replicas" -> "0-all")
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val mappingJson =
+      (estype ->
+        ("_source" -> ("enabled" -> 0)) ~
+        ("_all" -> ("enabled" -> 0)) ~
+        ("_type" -> ("index" -> "no")) ~
+        ("enabled" -> 0))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(mappingJson))).get
+  }
+
+  def genNext(name: String): Int = {
+    try {
+      val response = client.prepareIndex(index, estype, name).
+        setSource(compact(render("n" -> name))).get
+      response.getVersion().toInt
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        0
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
new file mode 100644
index 0000000..5de2999
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/ESUtils.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import org.elasticsearch.action.search.SearchRequestBuilder
+import org.elasticsearch.client.Client
+import org.elasticsearch.common.unit.TimeValue
+import org.json4s.Formats
+import org.json4s.native.Serialization.read
+
+import scala.collection.mutable.ArrayBuffer
+
+object ESUtils {
+  val scrollLife = new TimeValue(60000)
+
+  def getAll[T : Manifest](
+      client: Client,
+      builder: SearchRequestBuilder)(
+      implicit formats: Formats): Seq[T] = {
+    val results = ArrayBuffer[T]()
+    var response = builder.setScroll(scrollLife).get
+    var hits = response.getHits().hits()
+    results ++= hits.map(h => read[T](h.getSourceAsString))
+    while (hits.size > 0) {
+      response = client.prepareSearchScroll(response.getScrollId).
+        setScroll(scrollLife).get
+      hits = response.getHits().hits()
+      results ++= hits.map(h => read[T](h.getSourceAsString))
+    }
+    results
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
new file mode 100644
index 0000000..6f6b1c9
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/StorageClient.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch1
+
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.BaseStorageClient
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.transport.TransportClient
+import org.elasticsearch.common.settings.ImmutableSettings
+import org.elasticsearch.common.transport.InetSocketTransportAddress
+import org.elasticsearch.transport.ConnectTransportException
+
+class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
+    with Logging {
+  override val prefix = "ES"
+  val client = try {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9300))
+    val settings = ImmutableSettings.settingsBuilder()
+      .put("cluster.name", config.properties.getOrElse("CLUSTERNAME", "elasticsearch"))
+    val transportClient = new TransportClient(settings)
+    (hosts zip ports) foreach { hp =>
+      transportClient.addTransportAddress(
+        new InetSocketTransportAddress(hp._1, hp._2))
+    }
+    transportClient
+  } catch {
+    case e: ConnectTransportException =>
+      throw new StorageClientException(e.getMessage, e)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
new file mode 100644
index 0000000..d6aa24a
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch1/package.scala
@@ -0,0 +1,25 @@
+/*
+ * 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.predictionio.data.storage
+
+/** Elasticsearch implementation of storage traits, supporting meta data only
+  *
+  * @group Implementation
+  */
+package object elasticsearch1 {}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/project/Build.scala
----------------------------------------------------------------------
diff --git a/project/Build.scala b/project/Build.scala
index 885073a..a8d730b 100644
--- a/project/Build.scala
+++ b/project/Build.scala
@@ -22,6 +22,9 @@ object PIOBuild extends Build {
   val elasticsearchVersion = SettingKey[String](
     "elasticsearch-version",
     "The version of Elasticsearch used for building.")
+  val elasticsearch1Version = SettingKey[String](
+    "elasticsearch1-version",
+    "The version of Elasticsearch 1.x used for building.")
   val json4sVersion = SettingKey[String](
     "json4s-version",
     "The version of JSON4S used for building.")

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/tests/Dockerfile
----------------------------------------------------------------------
diff --git a/tests/Dockerfile b/tests/Dockerfile
index 1f87554..0fe7187 100644
--- a/tests/Dockerfile
+++ b/tests/Dockerfile
@@ -18,6 +18,7 @@
 from ubuntu
 
 ENV SPARK_VERSION 1.4.0
+#ENV ELASTICSEARCH_VERSION 1.4.4
 ENV ELASTICSEARCH_VERSION 5.1.2
 ENV HBASE_VERSION 1.0.0
 
@@ -48,6 +49,7 @@ RUN rm spark-${SPARK_VERSION}-bin-hadoop2.6.tgz
 ENV SPARK_HOME /vendors/spark-${SPARK_VERSION}-bin-hadoop2.6
 
 RUN echo "== Installing Elasticsearch =="
+#RUN wget https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
 RUN wget https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
 RUN tar zxvfC elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz /vendors
 RUN rm elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/tests/docker-files/env-conf/pio-env.sh
----------------------------------------------------------------------
diff --git a/tests/docker-files/env-conf/pio-env.sh b/tests/docker-files/env-conf/pio-env.sh
index 7ea2164..aa18ff8 100644
--- a/tests/docker-files/env-conf/pio-env.sh
+++ b/tests/docker-files/env-conf/pio-env.sh
@@ -84,11 +84,17 @@ PIO_STORAGE_SOURCES_PGSQL_PASSWORD=pio
 # PIO_STORAGE_SOURCES_MYSQL_PASSWORD=pio
 
 # Elasticsearch Example
-PIO_STORAGE_SOURCES_ELASTICSEARCH_TYPE=elasticsearch
-PIO_STORAGE_SOURCES_ELASTICSEARCH_HOSTS=localhost
-PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
-PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
-PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$ELASTICSEARCH_HOME
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_TYPE=elasticsearch
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOSTS=localhost
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-5.1.2
+# Elasticsearch 1.x Example
+PIO_STORAGE_SOURCES_ELASTICSEARCH1_TYPE=elasticsearch1
+#PIO_STORAGE_SOURCES_ELASTICSEARCH1_CLUSTERNAME=pio
+PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOSTS=localhost
+PIO_STORAGE_SOURCES_ELASTICSEARCH1_PORTS=9300
+PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOME=$ELASTICSEARCH_HOME
 
 # Local File System Example
 PIO_STORAGE_SOURCES_LOCALFS_TYPE=localfs

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/bad2f038/tools/build.sbt
----------------------------------------------------------------------
diff --git a/tools/build.sbt b/tools/build.sbt
index 4e2b266..108e2fb 100644
--- a/tools/build.sbt
+++ b/tools/build.sbt
@@ -42,7 +42,6 @@ dependencyOverrides +=   "org.slf4j" % "slf4j-log4j12" % "1.7.18"
 assemblyMergeStrategy in assembly := {
   case PathList("META-INF", "LICENSE.txt") => MergeStrategy.concat
   case PathList("META-INF", "NOTICE.txt")  => MergeStrategy.concat
-  case PathList("org", "joda", "time", "base", "BaseDateTime.class") => MergeStrategy.first
   case x =>
     val oldStrategy = (assemblyMergeStrategy in assembly).value
     oldStrategy(x)


[02/10] incubator-predictionio git commit: Add support for Elasticsearch 5.x

Posted by do...@apache.org.
Add support for Elasticsearch 5.x


Project: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/commit/36b79d7d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/tree/36b79d7d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/diff/36b79d7d

Branch: refs/heads/feature/es5
Commit: 36b79d7d029d086d14b74c675308fca948c6aa85
Parents: 09d04ed
Author: takahiro-hagino <ta...@bizreach.co.jp>
Authored: Mon Jan 16 11:50:48 2017 +0900
Committer: takahiro-hagino <ta...@bizreach.co.jp>
Committed: Mon Jan 16 13:41:44 2017 +0900

----------------------------------------------------------------------
 bin/install.sh                                  |   4 +-
 bin/pio-start-all                               |  94 ++++---
 bin/pio-stop-all                                |  50 ++--
 build.sbt                                       |   4 +-
 conf/pio-env.sh.template                        |   8 +-
 core/build.sbt                                  |  40 +--
 .../predictionio/workflow/CreateWorkflow.scala  |  74 ++---
 data/build.sbt                                  |  49 ++--
 .../storage/elasticsearch/ESAccessKeys.scala    | 140 ++++++----
 .../data/storage/elasticsearch/ESApps.scala     | 162 +++++++----
 .../data/storage/elasticsearch/ESChannels.scala | 132 +++++----
 .../elasticsearch/ESEngineInstances.scala       | 233 ++++++++++------
 .../elasticsearch/ESEngineManifests.scala       | 111 +++++---
 .../elasticsearch/ESEvaluationInstances.scala   | 176 +++++++-----
 .../storage/elasticsearch/ESEventsUtil.scala    | 125 +++++++++
 .../data/storage/elasticsearch/ESLEvents.scala  | 269 +++++++++++++++++++
 .../data/storage/elasticsearch/ESPEvents.scala  | 151 +++++++++++
 .../storage/elasticsearch/ESSequences.scala     |  69 ++---
 .../data/storage/elasticsearch/ESUtils.scala    | 159 +++++++++--
 .../storage/elasticsearch/StorageClient.scala   |  24 +-
 tools/build.sbt                                 |   4 +-
 21 files changed, 1518 insertions(+), 560 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/bin/install.sh
----------------------------------------------------------------------
diff --git a/bin/install.sh b/bin/install.sh
index f17cde5..e4fe220 100755
--- a/bin/install.sh
+++ b/bin/install.sh
@@ -19,9 +19,9 @@
 
 OS=`uname`
 PIO_VERSION=0.11.0-SNAPSHOT
-SPARK_VERSION=1.6.2
+SPARK_VERSION=1.6.3
 # Looks like support for Elasticsearch 2.0 will require 2.0 so deferring
-ELASTICSEARCH_VERSION=1.7.5
+ELASTICSEARCH_VERSION=5.1.2
 HBASE_VERSION=1.2.2
 POSTGRES_VERSION=9.4-1204.jdbc41
 MYSQL_VERSION=5.1.37

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/bin/pio-start-all
----------------------------------------------------------------------
diff --git a/bin/pio-start-all b/bin/pio-start-all
index a78b0d2..03e10ae 100755
--- a/bin/pio-start-all
+++ b/bin/pio-start-all
@@ -25,63 +25,73 @@ export PIO_HOME="$(cd `dirname $0`/..; pwd)"
 
 . ${PIO_HOME}/bin/load-pio-env.sh
 
+SOURCE_TYPE=$PIO_STORAGE_REPOSITORIES_METADATA_SOURCE
+SOURCE_TYPE=$SOURCE_TYPE$PIO_STORAGE_REPOSITORIES_EVENTDATA_SOURCE
+SOURCE_TYPE=$SOURCE_TYPE$PIO_STORAGE_REPOSITORIES_MODELDATA_SOURCE
+
 # Elasticsearch
-echo "Starting Elasticsearch..."
-if [ -n "$PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME" ]; then
-  if [ -n "$JAVA_HOME" ]; then
-    JPS=`$JAVA_HOME/bin/jps`
+if [ `echo $SOURCE_TYPE | grep -i elasticsearch | wc -l` != 0 ] ; then
+  echo "Starting Elasticsearch..."
+  if [ -n "$PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME" ]; then
+    if [ -n "$JAVA_HOME" ]; then
+      JPS=`$JAVA_HOME/bin/jps`
+    else
+      JPS=`jps`
+    fi
+    if [[ ${JPS} =~ "Elasticsearch" ]]; then
+      echo -e "\033[0;31mElasticsearch is already running. Please use pio-stop-all to try stopping it first.\033[0m"
+      echo -e "\033[0;31mNote: If you started Elasticsearch manually, you will need to kill it manually.\033[0m"
+      echo -e "\033[0;31mAborting...\033[0m"
+      exit 1
+    else
+      $PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME/bin/elasticsearch -d -p $PIO_HOME/es.pid
+    fi
   else
-    JPS=`jps`
-  fi
-  if [[ ${JPS} =~ "Elasticsearch" ]]; then
-    echo -e "\033[0;31mElasticsearch is already running. Please use pio-stop-all to try stopping it first.\033[0m"
-    echo -e "\033[0;31mNote: If you started Elasticsearch manually, you will need to kill it manually.\033[0m"
-    echo -e "\033[0;31mAborting...\033[0m"
+    echo -e "\033[0;31mPlease set PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME in conf/pio-env.sh, or in your environment.\033[0m"
+    echo -e "\033[0;31mCannot start Elasticsearch. Aborting...\033[0m"
     exit 1
-  else
-    $PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME/bin/elasticsearch -d -p $PIO_HOME/es.pid
   fi
-else
-  echo -e "\033[0;31mPlease set PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME in conf/pio-env.sh, or in your environment.\033[0m"
-  echo -e "\033[0;31mCannot start Elasticsearch. Aborting...\033[0m"
-  exit 1
 fi
 
 # HBase
-echo "Starting HBase..."
-if [ -n "$PIO_STORAGE_SOURCES_HBASE_HOME" ]; then
-  $PIO_STORAGE_SOURCES_HBASE_HOME/bin/start-hbase.sh
-else
-  echo -e "\033[0;31mPlease set PIO_STORAGE_SOURCES_HBASE_HOME in conf/pio-env.sh, or in your environment.\033[0m"
-  # Kill everything for cleanliness
-  echo -e "\033[0;31mCannot start HBase. Aborting...\033[0m"
-  sleep 3
-  ${PIO_HOME}/bin/pio-stop-all
-  exit 1
+if [ `echo $SOURCE_TYPE | grep -i hbase | wc -l` != 0 ] ; then
+  echo "Starting HBase..."
+  if [ -n "$PIO_STORAGE_SOURCES_HBASE_HOME" ]; then
+    $PIO_STORAGE_SOURCES_HBASE_HOME/bin/start-hbase.sh
+  else
+    echo -e "\033[0;31mPlease set PIO_STORAGE_SOURCES_HBASE_HOME in conf/pio-env.sh, or in your environment.\033[0m"
+    # Kill everything for cleanliness
+    echo -e "\033[0;31mCannot start HBase. Aborting...\033[0m"
+    sleep 3
+    ${PIO_HOME}/bin/pio-stop-all
+    exit 1
+  fi
 fi
 
 #PGSQL
-pgsqlStatus="$(ps auxwww | grep postgres | wc -l)"
-if [[ "$pgsqlStatus" < 5 ]]; then
-  # Detect OS
-  OS=`uname`
-  if [[ "$OS" = "Darwin" ]]; then
-    pg_cmd=`which pg_ctl`
-    if [[ "$pg_cmd" != "" ]]; then
-      pg_ctl -D /usr/local/var/postgres -l /usr/local/var/postgres/server.log start
+if [ `echo $SOURCE_TYPE | grep -i pgsql | wc -l` != 0 ] ; then
+  pgsqlStatus="$(ps auxwww | grep postgres | wc -l)"
+  if [[ "$pgsqlStatus" < 5 ]]; then
+    # Detect OS
+    OS=`uname`
+    if [[ "$OS" = "Darwin" ]]; then
+      pg_cmd=`which pg_ctl`
+      if [[ "$pg_cmd" != "" ]]; then
+        pg_ctl -D /usr/local/var/postgres -l /usr/local/var/postgres/server.log start
+      fi
+    elif [[ "$OS" = "Linux" ]]; then
+      sudo service postgresql start
+    else
+      echo -e "\033[1;31mYour OS $OS is not yet supported for automatic postgresql startup:(\033[0m"
+      echo -e "\033[1;31mPlease do a manual startup!\033[0m"
+      ${PIO_HOME}/bin/pio-stop-all
+      exit 1
     fi
-  elif [[ "$OS" = "Linux" ]]; then
-    sudo service postgresql start
-  else
-    echo -e "\033[1;31mYour OS $OS is not yet supported for automatic postgresql startup:(\033[0m"
-    echo -e "\033[1;31mPlease do a manual startup!\033[0m"
-    ${PIO_HOME}/bin/pio-stop-all
-    exit 1
   fi
 fi
 
 # PredictionIO Event Server
-echo "Waiting 10 seconds for HBase to fully initialize..."
+echo "Waiting 10 seconds for HBase/Elasticsearch to fully initialize..."
 sleep 10
 echo "Starting PredictionIO Event Server..."
 ${PIO_HOME}/bin/pio-daemon ${PIO_HOME}/eventserver.pid eventserver --ip 0.0.0.0

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/bin/pio-stop-all
----------------------------------------------------------------------
diff --git a/bin/pio-stop-all b/bin/pio-stop-all
index 4aab5a3..dabad5d 100755
--- a/bin/pio-stop-all
+++ b/bin/pio-stop-all
@@ -25,6 +25,10 @@ export PIO_HOME="$(cd `dirname $0`/..; pwd)"
 
 . ${PIO_HOME}/bin/load-pio-env.sh
 
+SOURCE_TYPE=$PIO_STORAGE_REPOSITORIES_METADATA_SOURCE
+SOURCE_TYPE=$SOURCE_TYPE$PIO_STORAGE_REPOSITORIES_EVENTDATA_SOURCE
+SOURCE_TYPE=$SOURCE_TYPE$PIO_STORAGE_REPOSITORIES_MODELDATA_SOURCE
+
 # PredictionIO Event Server
 echo "Stopping PredictionIO Event Server..."
 PIDFILE=${PIO_HOME}/eventserver.pid
@@ -34,30 +38,38 @@ if [ -e ${PIDFILE} ]; then
 fi
 
 # HBase
-echo "Stopping HBase..."
-if [ -n "$PIO_STORAGE_SOURCES_HBASE_HOME" ]; then
-  $PIO_STORAGE_SOURCES_HBASE_HOME/bin/stop-hbase.sh
+if [ `echo $SOURCE_TYPE | grep -i hbase | wc -l` != 0 ] ; then
+  echo "Stopping HBase..."
+  if [ -n "$PIO_STORAGE_SOURCES_HBASE_HOME" ]; then
+    $PIO_STORAGE_SOURCES_HBASE_HOME/bin/stop-hbase.sh
+  fi
 fi
 
 # Elasticsearch
-echo "Stopping Elasticsearch..."
-PIDFILE=${PIO_HOME}/es.pid
-if [ -e ${PIDFILE} ]; then
-  cat ${PIDFILE} | xargs kill
-  rm ${PIDFILE}
+if [ `echo $SOURCE_TYPE | grep -i elasticsearch | wc -l` != 0 ] ; then
+  echo "Stopping Elasticsearch..."
+  PIDFILE=${PIO_HOME}/es.pid
+  if [ -e ${PIDFILE} ]; then
+    cat ${PIDFILE} | xargs kill
+    rm ${PIDFILE}
+  fi
 fi
 
 #PGSQL
-OS=`uname`
-if [[ "$OS" = "Darwin" ]]; then
-  pg_cmd=`which pg_ctl`
-  if [[ "$pg_cmd" != "" ]]; then
-    pg_ctl -D /usr/local/var/postgres stop -s -m fast
+if [ `echo $SOURCE_TYPE | grep -i pgsql | wc -l` != 0 ] ; then
+  if [ -n "$PIO_STORAGE_SOURCES_PGSQL_TYPE" ]; then
+    OS=`uname`
+    if [[ "$OS" = "Darwin" ]]; then
+      pg_cmd=`which pg_ctl`
+      if [[ "$pg_cmd" != "" ]]; then
+        pg_ctl -D /usr/local/var/postgres stop -s -m fast
+      fi
+    elif [[ "$OS" = "Linux" ]]; then
+      sudo service postgresql stop
+    else
+      echo -e "\033[1;31mYour OS $OS is not yet supported for automatic postgresql startup:(\033[0m"
+      echo -e "\033[1;31mPlease do a manual shutdown!\033[0m"
+      exit 1
+    fi
   fi
-elif [[ "$OS" = "Linux" ]]; then
-  sudo service postgresql stop
-else
-  echo -e "\033[1;31mYour OS $OS is not yet supported for automatic postgresql startup:(\033[0m"
-  echo -e "\033[1;31mPlease do a manual shutdown!\033[0m"
-  exit 1
 fi

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/build.sbt
----------------------------------------------------------------------
diff --git a/build.sbt b/build.sbt
index 21c31c6..4d6a5b6 100644
--- a/build.sbt
+++ b/build.sbt
@@ -34,11 +34,11 @@ fork in (ThisBuild, run) := true
 javacOptions in (ThisBuild, compile) ++= Seq("-source", "1.7", "-target", "1.7",
   "-Xlint:deprecation", "-Xlint:unchecked")
 
-elasticsearchVersion in ThisBuild := "1.4.4"
+elasticsearchVersion in ThisBuild := "5.1.2"
 
 json4sVersion in ThisBuild := "3.2.10"
 
-sparkVersion in ThisBuild := "1.4.0"
+sparkVersion in ThisBuild := "1.6.3"
 
 lazy val pioBuildInfoSettings = buildInfoSettings ++ Seq(
   sourceGenerators in Compile <+= buildInfo,

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/conf/pio-env.sh.template
----------------------------------------------------------------------
diff --git a/conf/pio-env.sh.template b/conf/pio-env.sh.template
index a06cd8e..a2841e3 100644
--- a/conf/pio-env.sh.template
+++ b/conf/pio-env.sh.template
@@ -24,7 +24,7 @@
 # you need to change these to fit your site.
 
 # SPARK_HOME: Apache Spark is a hard dependency and must be configured.
-SPARK_HOME=$PIO_HOME/vendors/spark-1.5.1-bin-hadoop2.6
+SPARK_HOME=$PIO_HOME/vendors/spark-1.6.3-bin-hadoop2.6
 
 POSTGRES_JDBC_DRIVER=$PIO_HOME/lib/postgresql-9.4-1204.jdbc41.jar
 MYSQL_JDBC_DRIVER=$PIO_HOME/lib/mysql-connector-java-5.1.37.jar
@@ -85,10 +85,10 @@ PIO_STORAGE_SOURCES_PGSQL_PASSWORD=pio
 
 # Elasticsearch Example
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_TYPE=elasticsearch
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_CLUSTERNAME=<elasticsearch_cluster_name>
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_HOSTS=localhost
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9300
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-1.4.4
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-5.1.2
 
 # Local File System Example
 # PIO_STORAGE_SOURCES_LOCALFS_TYPE=localfs

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/core/build.sbt
----------------------------------------------------------------------
diff --git a/core/build.sbt b/core/build.sbt
index 637d4ea..abd8e07 100644
--- a/core/build.sbt
+++ b/core/build.sbt
@@ -18,27 +18,27 @@
 name := "apache-predictionio-core"
 
 libraryDependencies ++= Seq(
-  "com.github.scopt"       %% "scopt"            % "3.3.0",
-  "com.google.code.gson"    % "gson"             % "2.5",
-  "com.google.guava"        % "guava"            % "18.0",
-  "com.twitter"            %% "chill"            % "0.7.2"
+  "com.github.scopt"        %% "scopt"            % "3.3.0",
+  "com.google.code.gson"     % "gson"             % "2.5",
+  "com.google.guava"         % "guava"            % "18.0",
+  "com.twitter"             %% "chill"            % "0.7.2"
     exclude("com.esotericsoftware.minlog", "minlog"),
-  "com.twitter"            %% "chill-bijection"  % "0.7.2",
-  "de.javakaffee"           % "kryo-serializers" % "0.37",
-  "commons-io"              % "commons-io"       % "2.4",
-  "io.spray"               %% "spray-can"        % "1.3.3",
-  "io.spray"               %% "spray-routing"    % "1.3.3",
-  "net.jodah"               % "typetools"        % "0.3.1",
-  "org.apache.spark"       %% "spark-core"       % sparkVersion.value % "provided",
-  "org.apache.spark"       %% "spark-sql"        % sparkVersion.value % "provided",
-  "org.clapper"            %% "grizzled-slf4j"   % "1.0.2",
-  "org.elasticsearch"       % "elasticsearch"    % elasticsearchVersion.value,
-  "org.json4s"             %% "json4s-native"    % json4sVersion.value,
-  "org.json4s"             %% "json4s-ext"       % json4sVersion.value,
-  "org.scalaj"             %% "scalaj-http"      % "1.1.6",
-  "org.scalatest"          %% "scalatest"        % "2.1.7" % "test",
-  "org.slf4j"               % "slf4j-log4j12"    % "1.7.18",
-  "org.specs2"             %% "specs2"           % "2.3.13" % "test")
+  "com.twitter"             %% "chill-bijection"  % "0.7.2",
+  "de.javakaffee"            % "kryo-serializers" % "0.37",
+  "commons-io"               % "commons-io"       % "2.4",
+  "io.spray"                %% "spray-can"        % "1.3.3",
+  "io.spray"                %% "spray-routing"    % "1.3.3",
+  "net.jodah"                % "typetools"        % "0.3.1",
+  "org.apache.spark"        %% "spark-core"       % sparkVersion.value % "provided",
+  "org.apache.spark"        %% "spark-sql"        % sparkVersion.value % "provided",
+  "org.clapper"             %% "grizzled-slf4j"   % "1.0.2",
+  "org.elasticsearch.client" % "rest"             % elasticsearchVersion.value,
+  "org.json4s"              %% "json4s-native"    % json4sVersion.value,
+  "org.json4s"              %% "json4s-ext"       % json4sVersion.value,
+  "org.scalaj"              %% "scalaj-http"      % "1.1.6",
+  "org.scalatest"           %% "scalatest"        % "2.1.7" % "test",
+  "org.slf4j"                % "slf4j-log4j12"    % "1.7.18",
+  "org.specs2"              %% "specs2"           % "2.3.13" % "test")
 
 //testOptions := Seq(Tests.Filter(s => Seq("Dev").exists(s.contains(_))))
 

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala b/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
index 899ace2..edfc1b6 100644
--- a/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
+++ b/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
@@ -223,36 +223,40 @@ object CreateWorkflow extends Logging {
         engineFactoryObj.engineParams(wfc.engineParamsKey)
       }
 
-      val engineInstance = EngineInstance(
-        id = "",
-        status = "INIT",
-        startTime = DateTime.now,
-        endTime = DateTime.now,
-        engineId = wfc.engineId,
-        engineVersion = wfc.engineVersion,
-        engineVariant = variantId,
-        engineFactory = engineFactory,
-        batch = wfc.batch,
-        env = pioEnvVars,
-        sparkConf = workflowParams.sparkEnv,
-        dataSourceParams =
-          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.dataSourceParams),
-        preparatorParams =
-          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.preparatorParams),
-        algorithmsParams =
-          JsonExtractor.paramsToJson(wfc.jsonExtractor, engineParams.algorithmParamsList),
-        servingParams =
-          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.servingParams))
+      try {
+        val engineInstance = EngineInstance(
+          id = "",
+          status = "INIT",
+          startTime = DateTime.now,
+          endTime = DateTime.now,
+          engineId = wfc.engineId,
+          engineVersion = wfc.engineVersion,
+          engineVariant = variantId,
+          engineFactory = engineFactory,
+          batch = wfc.batch,
+          env = pioEnvVars,
+          sparkConf = workflowParams.sparkEnv,
+          dataSourceParams =
+            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.dataSourceParams),
+          preparatorParams =
+            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.preparatorParams),
+          algorithmsParams =
+            JsonExtractor.paramsToJson(wfc.jsonExtractor, engineParams.algorithmParamsList),
+          servingParams =
+            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.servingParams))
 
-      val engineInstanceId = Storage.getMetaDataEngineInstances.insert(
-        engineInstance)
+        val engineInstanceId = Storage.getMetaDataEngineInstances.insert(
+          engineInstance)
 
-      CoreWorkflow.runTrain(
-        env = pioEnvVars,
-        params = workflowParams,
-        engine = trainableEngine,
-        engineParams = engineParams,
-        engineInstance = engineInstance.copy(id = engineInstanceId))
+        CoreWorkflow.runTrain(
+          env = pioEnvVars,
+          params = workflowParams,
+          engine = trainableEngine,
+          engineParams = engineParams,
+          engineInstance = engineInstance.copy(id = engineInstanceId))
+      } finally {
+        Storage.getLEvents().close()
+      }
     } else {
       val workflowParams = WorkflowParams(
         verbose = wfc.verbosity,
@@ -267,11 +271,15 @@ object CreateWorkflow extends Logging {
         env = pioEnvVars,
         sparkConf = workflowParams.sparkEnv
       )
-      Workflow.runEvaluation(
-        evaluation = evaluation.get,
-        engineParamsGenerator = engineParamsGenerator.get,
-        evaluationInstance = evaluationInstance,
-        params = workflowParams)
+      try {
+        Workflow.runEvaluation(
+          evaluation = evaluation.get,
+          engineParamsGenerator = engineParamsGenerator.get,
+          evaluationInstance = evaluationInstance,
+          params = workflowParams)
+      } finally {
+        Storage.getLEvents().close()
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/build.sbt
----------------------------------------------------------------------
diff --git a/data/build.sbt b/data/build.sbt
index 4526c39..4ae9b42 100644
--- a/data/build.sbt
+++ b/data/build.sbt
@@ -18,41 +18,44 @@
 name := "apache-predictionio-data"
 
 libraryDependencies ++= Seq(
-  "com.github.nscala-time" %% "nscala-time"    % "2.6.0",
-  "commons-codec"           % "commons-codec"  % "1.9",
-  "io.spray"               %% "spray-can"      % "1.3.3",
-  "io.spray"               %% "spray-routing"  % "1.3.3",
-  "io.spray"               %% "spray-testkit"  % "1.3.3" % "test",
-  "mysql"                   % "mysql-connector-java" % "5.1.37" % "optional",
-  "org.apache.hadoop"       % "hadoop-common"  % "2.6.2"
+  "com.github.nscala-time"  %% "nscala-time"    % "2.6.0",
+  "commons-codec"            % "commons-codec"  % "1.9",
+  "io.spray"                %% "spray-can"      % "1.3.3",
+  "io.spray"                %% "spray-routing"  % "1.3.3",
+  "io.spray"                %% "spray-testkit"  % "1.3.3" % "test",
+  "mysql"                    % "mysql-connector-java" % "5.1.37" % "optional",
+  "org.apache.hadoop"        % "hadoop-common"  % "2.6.2"
     exclude("javax.servlet", "servlet-api"),
-  "org.apache.hbase"        % "hbase-common"   % "0.98.5-hadoop2",
-  "org.apache.hbase"        % "hbase-client"   % "0.98.5-hadoop2"
+  "org.apache.hbase"         % "hbase-common"   % "0.98.5-hadoop2",
+  "org.apache.hbase"         % "hbase-client"   % "0.98.5-hadoop2"
     exclude("org.apache.zookeeper", "zookeeper"),
   // added for Parallel storage interface
-  "org.apache.hbase"        % "hbase-server"   % "0.98.5-hadoop2"
+  "org.apache.hbase"         % "hbase-server"   % "0.98.5-hadoop2"
     exclude("org.apache.hbase", "hbase-client")
     exclude("org.apache.zookeeper", "zookeeper")
     exclude("javax.servlet", "servlet-api")
     exclude("org.mortbay.jetty", "servlet-api-2.5")
     exclude("org.mortbay.jetty", "jsp-api-2.1")
     exclude("org.mortbay.jetty", "jsp-2.1"),
-  "org.apache.zookeeper"    % "zookeeper"      % "3.4.7"
+  "org.apache.zookeeper"     % "zookeeper"      % "3.4.7"
     exclude("org.slf4j", "slf4j-api")
     exclude("org.slf4j", "slf4j-log4j12"),
-  "org.apache.spark"       %% "spark-core"     % sparkVersion.value % "provided",
-  "org.apache.spark"       %% "spark-sql"      % sparkVersion.value % "provided",
-  "org.clapper"            %% "grizzled-slf4j" % "1.0.2",
-  "org.elasticsearch"       % "elasticsearch"  % elasticsearchVersion.value,
-  "org.json4s"             %% "json4s-native"  % json4sVersion.value,
-  "org.json4s"             %% "json4s-ext"     % json4sVersion.value,
-  "org.postgresql"          % "postgresql"     % "9.4-1204-jdbc41",
-  "org.scalatest"          %% "scalatest"      % "2.1.7" % "test",
-  "org.scalikejdbc"        %% "scalikejdbc"    % "2.3.5",
-  "org.slf4j"               % "slf4j-log4j12"  % "1.7.18",
-  "org.spark-project.akka" %% "akka-actor"     % "2.3.4-spark",
-  "org.specs2"             %% "specs2"         % "2.3.13" % "test")
+  "org.apache.spark"        %% "spark-core"     % sparkVersion.value % "provided",
+  "org.apache.spark"        %% "spark-sql"      % sparkVersion.value % "provided",
+  "org.clapper"             %% "grizzled-slf4j" % "1.0.2",
+  "org.elasticsearch.client" % "rest"           % elasticsearchVersion.value,
+  "org.elasticsearch"        % "elasticsearch-spark-13_2.10" % elasticsearchVersion.value % "provided",
+  "org.elasticsearch"        % "elasticsearch-hadoop-mr" % elasticsearchVersion.value,
+  "org.json4s"              %% "json4s-native"  % json4sVersion.value,
+  "org.json4s"              %% "json4s-ext"     % json4sVersion.value,
+  "org.postgresql"           % "postgresql"     % "9.4-1204-jdbc41",
+  "org.scalatest"           %% "scalatest"      % "2.1.7" % "test",
+  "org.scalikejdbc"         %% "scalikejdbc"    % "2.3.5",
+  "org.slf4j"                % "slf4j-log4j12"  % "1.7.18",
+  "org.spark-project.akka"  %% "akka-actor"     % "2.3.4-spark",
+  "org.specs2"              %% "specs2"         % "2.3.13" % "test")
 
 parallelExecution in Test := false
 
 pomExtra := childrenPomExtra.value
+

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
index 077168a..2c69cf4 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
@@ -15,45 +15,41 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.StorageClientConfig
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.AccessKey
 import org.apache.predictionio.data.storage.AccessKeys
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.json4s.JsonDSL._
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
 import org.json4s._
+import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import scala.util.Random
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
 
 /** Elasticsearch implementation of AccessKeys. */
-class ESAccessKeys(client: Client, config: StorageClientConfig, index: String)
+class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: String)
     extends AccessKeys with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "accesskeys"
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("key" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("events" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0)) ~
+      ("properties" ->
+        ("key" -> ("type" -> "keyword")) ~
+        ("events" -> ("type" -> "keyword"))))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def insert(accessKey: AccessKey): Option[String] = {
     val key = if (accessKey.key.isEmpty) generateKey else accessKey.key
@@ -61,59 +57,99 @@ class ESAccessKeys(client: Client, config: StorageClientConfig, index: String)
     Some(key)
   }
 
-  def get(key: String): Option[AccessKey] = {
+  def get(id: String): Option[AccessKey] = {
     try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        key).get()
-      Some(read[AccessKey](response.getSourceAsString))
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[AccessKey])
+        case _ =>
+          None
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/$key", e)
         None
-      case e: NullPointerException => None
     }
   }
 
   def getAll(): Seq[AccessKey] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[AccessKey](client, builder)
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[AccessKey](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[AccessKey]()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def getByAppid(appid: Int): Seq[AccessKey] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("appid", appid))
-      ESUtils.getAll[AccessKey](client, builder)
+      val json =
+        ("query" ->
+          ("term" ->
+            ("appid" -> appid)))
+      ESUtils.getAll[AccessKey](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[AccessKey]()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def update(accessKey: AccessKey): Unit = {
+    val id = accessKey.key
     try {
-      client.prepareIndex(index, estype, accessKey.key).setSource(write(accessKey)).get()
+      val entity = new NStringEntity(write(accessKey), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 
-  def delete(key: String): Unit = {
+  def delete(id: String): Unit = {
     try {
-      client.prepareDelete(index, estype, key).get
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/id")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/id", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
index 3781a4b..7a65379 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
@@ -15,116 +15,160 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.StorageClientConfig
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.App
 import org.apache.predictionio.data.storage.Apps
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.json4s.JsonDSL._
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
 import org.json4s._
+import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
 /** Elasticsearch implementation of Items. */
-class ESApps(client: Client, config: StorageClientConfig, index: String)
-  extends Apps with Logging {
+class ESApps(client: RestClient, config: StorageClientConfig, index: String)
+    extends Apps with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "apps"
   private val seq = new ESSequences(client, config, index)
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0))~
+      ("properties" ->
+        ("id" -> ("type" -> "keyword")) ~
+        ("name" -> ("type" -> "keyword"))))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def insert(app: App): Option[Int] = {
     val id =
       if (app.id == 0) {
-        var roll = seq.genNext("apps")
-        while (!get(roll).isEmpty) roll = seq.genNext("apps")
+        var roll = seq.genNext(estype)
+        while (!get(roll).isEmpty) roll = seq.genNext(estype)
         roll
-      }
-      else app.id
-    val realapp = app.copy(id = id)
-    update(realapp)
+      } else app.id
+    update(app.copy(id = id))
     Some(id)
   }
 
   def get(id: Int): Option[App] = {
     try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        id.toString).get()
-      Some(read[App](response.getSourceAsString))
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[App])
+        case _ =>
+          None
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
         None
-      case e: NullPointerException => None
     }
   }
 
   def getByName(name: String): Option[App] = {
     try {
-      val response = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("name", name)).get
-      val hits = response.getHits().hits()
-      if (hits.size > 0) {
-        Some(read[App](hits.head.getSourceAsString))
-      } else {
-        None
+      val json =
+        ("query" ->
+          ("term" ->
+            ("name" -> name)))
+      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/_search",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "hits" \ "total").extract[Long] match {
+        case 0 => None
+        case _ =>
+          val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
+          val result = (results.head \ "_source").extract[App]
+          Some(result)
       }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
         None
     }
   }
 
   def getAll(): Seq[App] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[App](client, builder)
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[App](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[App]()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def update(app: App): Unit = {
+    val id = app.id.toString
     try {
-      val response = client.prepareIndex(index, estype, app.id.toString).
-        setSource(write(app)).get()
+      val entity = new NStringEntity(write(app), ContentType.APPLICATION_JSON);
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 
   def delete(id: Int): Unit = {
     try {
-      client.prepareDelete(index, estype, id.toString).get
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/id", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
index 52697fd..c90d668 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
@@ -15,102 +15,134 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.Channel
 import org.apache.predictionio.data.storage.Channels
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders.termFilter
-import org.json4s.DefaultFormats
+import org.elasticsearch.client.RestClient
+import org.json4s._
 import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-class ESChannels(client: Client, config: StorageClientConfig, index: String)
-    extends Channels with Logging {
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
 
+class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
+    extends Channels with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "channels"
   private val seq = new ESSequences(client, config, index)
-  private val seqName = "channels"
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0))~
+      ("properties" ->
+        ("name" -> ("type" -> "keyword"))))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def insert(channel: Channel): Option[Int] = {
     val id =
       if (channel.id == 0) {
-        var roll = seq.genNext(seqName)
-        while (!get(roll).isEmpty) roll = seq.genNext(seqName)
+        var roll = seq.genNext(estype)
+        while (!get(roll).isEmpty) roll = seq.genNext(estype)
         roll
       } else channel.id
 
-    val realChannel = channel.copy(id = id)
-    if (update(realChannel)) Some(id) else None
+    if (update(channel.copy(id = id))) Some(id) else None
   }
 
   def get(id: Int): Option[Channel] = {
     try {
-      val response = client.prepareGet(
-        index,
-        estype,
-        id.toString).get()
-      Some(read[Channel](response.getSourceAsString))
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[Channel])
+        case _ =>
+          None
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
         None
-      case e: NullPointerException => None
     }
   }
 
   def getByAppid(appid: Int): Seq[Channel] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype).
-        setPostFilter(termFilter("appid", appid))
-      ESUtils.getAll[Channel](client, builder)
+      val json =
+        ("query" ->
+          ("term" ->
+            ("appid" -> appid)))
+      ESUtils.getAll[Channel](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq[Channel]()
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def update(channel: Channel): Boolean = {
+    val id = channel.id.toString
     try {
-      val response = client.prepareIndex(index, estype, channel.id.toString).
-        setSource(write(channel)).get()
-      true
+      val entity = new NStringEntity(write(channel), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "created" => true
+        case "updated" => true
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+          false
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
         false
     }
   }
 
   def delete(id: Int): Unit = {
     try {
-      client.prepareDelete(index, estype, id.toString).get
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
index 21690bf..08f87f3 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
@@ -15,144 +15,211 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
+import java.io.IOException
+
+import scala.collection.JavaConverters.mapAsJavaMapConverter
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.EngineInstance
 import org.apache.predictionio.data.storage.EngineInstanceSerializer
 import org.apache.predictionio.data.storage.EngineInstances
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.elasticsearch.search.sort.SortOrder
-import org.json4s.JsonDSL._
+import org.elasticsearch.client.RestClient
 import org.json4s._
+import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-class ESEngineInstances(client: Client, config: StorageClientConfig, index: String)
-  extends EngineInstances with Logging {
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESEngineInstances(client: RestClient, config: StorageClientConfig, index: String)
+    extends EngineInstances with Logging {
   implicit val formats = DefaultFormats + new EngineInstanceSerializer
   private val estype = "engine_instances"
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("startTime" -> ("type" -> "date")) ~
-          ("endTime" -> ("type" -> "date")) ~
-          ("engineId" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineVersion" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineVariant" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineFactory" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("batch" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("dataSourceParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("preparatorParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("algorithmsParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("servingParams" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0))~
+      ("properties" ->
+        ("status" -> ("type" -> "keyword")) ~
+        ("startTime" -> ("type" -> "date")) ~
+        ("endTime" -> ("type" -> "date")) ~
+        ("engineId" -> ("type" -> "keyword")) ~
+        ("engineVersion" -> ("type" -> "keyword")) ~
+        ("engineVariant" -> ("type" -> "keyword")) ~
+        ("engineFactory" -> ("type" -> "keyword")) ~
+        ("batch" -> ("type" -> "keyword")) ~
+        ("dataSourceParams" -> ("type" -> "keyword")) ~
+        ("preparatorParams" -> ("type" -> "keyword")) ~
+        ("algorithmsParams" -> ("type" -> "keyword")) ~
+        ("servingParams" -> ("type" -> "keyword")) ~
+        ("status" -> ("type" -> "keyword"))))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def insert(i: EngineInstance): String = {
+    val id = i.id match {
+      case x if x.isEmpty =>
+        @scala.annotation.tailrec
+        def generateId(newId: Option[String]): String = {
+          newId match {
+            case Some(x) => x
+            case _ => generateId(preInsert())
+          }
+        }
+        generateId(preInsert())
+      case x => x
+    }
+
+    update(i.copy(id = id))
+    id
+  }
+
+  def preInsert(): Option[String] = {
     try {
-      val response = client.prepareIndex(index, estype).
-        setSource(write(i)).get
-      response.getId
+      val entity = new NStringEntity("{}", ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+          Some((jsonResponse \ "_id").extract[String])
+        case _ =>
+          error(s"[$result] Failed to create $index/$estype")
+          None
+      }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        ""
+      case e: IOException =>
+        error(s"Failed to create $index/$estype", e)
+        None
     }
   }
 
   def get(id: String): Option[EngineInstance] = {
     try {
-      val response = client.prepareGet(index, estype, id).get
-      if (response.isExists) {
-        Some(read[EngineInstance](response.getSourceAsString))
-      } else {
-        None
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[EngineInstance])
+        case _ =>
+          None
       }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
         None
     }
   }
 
   def getAll(): Seq[EngineInstance] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[EngineInstance](client, builder)
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[EngineInstance](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def getCompleted(
-      engineId: String,
-      engineVersion: String,
-      engineVariant: String): Seq[EngineInstance] = {
+    engineId: String,
+    engineVersion: String,
+    engineVariant: String): Seq[EngineInstance] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
-        andFilter(
-          termFilter("status", "COMPLETED"),
-          termFilter("engineId", engineId),
-          termFilter("engineVersion", engineVersion),
-          termFilter("engineVariant", engineVariant))).
-        addSort("startTime", SortOrder.DESC)
-      ESUtils.getAll[EngineInstance](client, builder)
+      val json =
+        ("query" ->
+          ("bool" ->
+            ("must" -> List(
+              ("term" ->
+                ("status" -> "COMPLETED")),
+              ("term" ->
+                ("engineId" -> engineId)),
+              ("term" ->
+                ("engineVersion" -> engineVersion)),
+              ("term" ->
+                ("engineVariant" -> engineVariant)))))) ~
+              ("sort" -> List(
+                ("startTime" ->
+                  ("order" -> "desc"))))
+      ESUtils.getAll[EngineInstance](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def getLatestCompleted(
-      engineId: String,
-      engineVersion: String,
-      engineVariant: String): Option[EngineInstance] =
+    engineId: String,
+    engineVersion: String,
+    engineVariant: String): Option[EngineInstance] =
     getCompleted(
       engineId,
       engineVersion,
       engineVariant).headOption
 
   def update(i: EngineInstance): Unit = {
+    val id = i.id
     try {
-      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
+      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException => error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 
   def delete(id: String): Unit = {
     try {
-      val response = client.prepareDelete(index, estype, id).get
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException => error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
index 65b6691..a965c71 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
@@ -15,59 +15,96 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
-import org.apache.predictionio.data.storage.EngineManifestSerializer
-import org.apache.predictionio.data.storage.StorageClientConfig
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.EngineManifest
+import org.apache.predictionio.data.storage.EngineManifestSerializer
 import org.apache.predictionio.data.storage.EngineManifests
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
 import org.json4s._
-import org.json4s.native.Serialization.read
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
 import org.json4s.native.Serialization.write
 
-class ESEngineManifests(client: Client, config: StorageClientConfig, index: String)
-  extends EngineManifests with Logging {
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESEngineManifests(client: RestClient, config: StorageClientConfig, index: String)
+    extends EngineManifests with Logging {
   implicit val formats = DefaultFormats + new EngineManifestSerializer
   private val estype = "engine_manifests"
-  private def esid(id: String, version: String) = s"$id $version"
+  private def esid(id: String, version: String) = s"$id-$version"
 
   def insert(engineManifest: EngineManifest): Unit = {
-    val json = write(engineManifest)
-    val response = client.prepareIndex(
-      index,
-      estype,
-      esid(engineManifest.id, engineManifest.version)).
-      setSource(json).execute().actionGet()
+    val id = esid(engineManifest.id, engineManifest.version)
+    try {
+      val entity = new NStringEntity(write(engineManifest), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
+    } catch {
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
+    }
   }
 
   def get(id: String, version: String): Option[EngineManifest] = {
+    val esId = esid(id, version)
     try {
-      val response = client.prepareGet(index, estype, esid(id, version)).
-        execute().actionGet()
-      if (response.isExists) {
-        Some(read[EngineManifest](response.getSourceAsString))
-      } else {
-        None
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$esId",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[EngineManifest])
+        case _ =>
+          None
       }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$esId", e)
         None
     }
+
   }
 
   def getAll(): Seq[EngineManifest] = {
     try {
-      val builder = client.prepareSearch()
-      ESUtils.getAll[EngineManifest](client, builder)
+      val json =
+        ("query" ->
+          ("match_all" ->  List.empty))
+      ESUtils.getAll[EngineManifest](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
@@ -75,10 +112,22 @@ class ESEngineManifests(client: Client, config: StorageClientConfig, index: Stri
     insert(engineManifest)
 
   def delete(id: String, version: String): Unit = {
+    val esId = esid(id, version)
     try {
-      client.prepareDelete(index, estype, esid(id, version)).execute().actionGet()
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$esId",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$esId")
+      }
     } catch {
-      case e: ElasticsearchException => error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$esId", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
index 85bf820..0e71f79 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
@@ -15,122 +15,160 @@
  * limitations under the License.
  */
 
-
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
 import org.apache.predictionio.data.storage.EvaluationInstance
 import org.apache.predictionio.data.storage.EvaluationInstanceSerializer
 import org.apache.predictionio.data.storage.EvaluationInstances
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.ElasticsearchException
-import org.elasticsearch.client.Client
-import org.elasticsearch.index.query.FilterBuilders._
-import org.elasticsearch.search.sort.SortOrder
-import org.json4s.JsonDSL._
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.RestClient
 import org.json4s._
+import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-class ESEvaluationInstances(client: Client, config: StorageClientConfig, index: String)
-  extends EvaluationInstances with Logging {
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, index: String)
+    extends EvaluationInstances with Logging {
   implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
   private val estype = "evaluation_instances"
+  private val seq = new ESSequences(client, config, index)
 
-  val indices = client.admin.indices
-  val indexExistResponse = indices.prepareExists(index).get
-  if (!indexExistResponse.isExists) {
-    indices.prepareCreate(index).get
-  }
-  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
-  if (!typeExistResponse.isExists) {
-    val json =
-      (estype ->
-        ("properties" ->
-          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("startTime" -> ("type" -> "date")) ~
-          ("endTime" -> ("type" -> "date")) ~
-          ("evaluationClass" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("engineParamsGeneratorClass" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("batch" ->
-            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
-          ("evaluatorResults" ->
-            ("type" -> "string") ~ ("index" -> "no")) ~
-          ("evaluatorResultsHTML" ->
-            ("type" -> "string") ~ ("index" -> "no")) ~
-          ("evaluatorResultsJSON" ->
-            ("type" -> "string") ~ ("index" -> "no"))))
-    indices.preparePutMapping(index).setType(estype).
-      setSource(compact(render(json))).get
-  }
+  ESUtils.createIndex(client, index)
+  val mappingJson =
+    (estype ->
+      ("_all" -> ("enabled" -> 0))~
+      ("properties" ->
+        ("status" -> ("type" -> "keyword")) ~
+        ("startTime" -> ("type" -> "date")) ~
+        ("endTime" -> ("type" -> "date")) ~
+        ("evaluationClass" -> ("type" -> "keyword")) ~
+        ("engineParamsGeneratorClass" -> ("type" -> "keyword")) ~
+        ("batch" -> ("type" -> "keyword")) ~
+        ("evaluatorResults" -> ("type" -> "text") ~ ("index" -> "no")) ~
+        ("evaluatorResultsHTML" -> ("type" -> "text") ~ ("index" -> "no")) ~
+        ("evaluatorResultsJSON" -> ("type" -> "text") ~ ("index" -> "no"))))
+  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
 
   def insert(i: EvaluationInstance): String = {
-    try {
-      val response = client.prepareIndex(index, estype).
-        setSource(write(i)).get
-      response.getId
-    } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        ""
+    val id = i.id match {
+      case x if x.isEmpty =>
+        var roll = seq.genNext(estype).toString
+        while (!get(roll).isEmpty) roll = seq.genNext(estype).toString
+        roll
+      case x => x
     }
+
+    update(i.copy(id = id))
+    id
   }
 
   def get(id: String): Option[EvaluationInstance] = {
     try {
-      val response = client.prepareGet(index, estype, id).get
-      if (response.isExists) {
-        Some(read[EvaluationInstance](response.getSourceAsString))
-      } else {
-        None
+      val response = client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      (jsonResponse \ "found").extract[Boolean] match {
+        case true =>
+          Some((jsonResponse \ "_source").extract[EvaluationInstance])
+        case _ =>
+          None
       }
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => None
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            None
+        }
+      case e: IOException =>
+        error(s"Failed to access to /$index/$estype/$id", e)
         None
     }
   }
 
   def getAll(): Seq[EvaluationInstance] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype)
-      ESUtils.getAll[EvaluationInstance](client, builder)
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      ESUtils.getAll[EvaluationInstance](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def getCompleted(): Seq[EvaluationInstance] = {
     try {
-      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
-        termFilter("status", "EVALCOMPLETED")).
-        addSort("startTime", SortOrder.DESC)
-      ESUtils.getAll[EvaluationInstance](client, builder)
+      val json =
+        ("query" ->
+          ("term" ->
+            ("status" -> "EVALCOMPLETED"))) ~
+            ("sort" ->
+              ("startTime" ->
+                ("order" -> "desc")))
+      ESUtils.getAll[EvaluationInstance](client, index, estype, compact(render(json)))
     } catch {
-      case e: ElasticsearchException =>
-        error(e.getMessage)
-        Seq()
+      case e: IOException =>
+        error("Failed to access to /$index/$estype/_search", e)
+        Nil
     }
   }
 
   def update(i: EvaluationInstance): Unit = {
+    val id = i.id
     try {
-      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
+      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
+      val response = client.performRequest(
+        "POST",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava,
+        entity)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "created" =>
+        case "updated" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException => error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 
   def delete(id: String): Unit = {
     try {
-      client.prepareDelete(index, estype, id).get
+      val response = client.performRequest(
+        "DELETE",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava)
+      val json = parse(EntityUtils.toString(response.getEntity))
+      val result = (json \ "result").extract[String]
+      result match {
+        case "deleted" =>
+        case _ =>
+          error(s"[$result] Failed to update $index/$estype/$id")
+      }
     } catch {
-      case e: ElasticsearchException => error(e.getMessage)
+      case e: IOException =>
+        error(s"Failed to update $index/$estype/$id", e)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
new file mode 100644
index 0000000..f2ab7c2
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
@@ -0,0 +1,125 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch
+
+import org.apache.hadoop.io.DoubleWritable
+import org.apache.hadoop.io.LongWritable
+import org.apache.hadoop.io.MapWritable
+import org.apache.hadoop.io.Text
+import org.apache.predictionio.data.storage.DataMap
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.EventValidation
+import org.joda.time.DateTime
+import org.joda.time.DateTimeZone
+import org.json4s._
+
+object ESEventsUtil {
+
+  implicit val formats = DefaultFormats
+
+  def resultToEvent(id: Text, result: MapWritable, appId: Int): Event = {
+
+    def getStringCol(col: String): String = {
+      val r = result.get(new Text(col)).asInstanceOf[Text]
+      require(r != null,
+        s"Failed to get value for column ${col}. " +
+          s"StringBinary: ${r.getBytes()}.")
+
+      r.toString()
+    }
+
+    def getOptStringCol(col: String): Option[String] = {
+      val r = result.get(new Text(col))
+      if (r == null) {
+        None
+      } else {
+        Some(r.asInstanceOf[Text].toString())
+      }
+    }
+
+    val tmp = result
+      .get(new Text("properties")).asInstanceOf[MapWritable]
+      .get(new Text("fields")).asInstanceOf[MapWritable]
+      .get(new Text("rating"))
+
+    val rating =
+      if (tmp.isInstanceOf[DoubleWritable]) tmp.asInstanceOf[DoubleWritable]
+      else if (tmp.isInstanceOf[LongWritable]) {
+        new DoubleWritable(tmp.asInstanceOf[LongWritable].get().toDouble)
+      }
+      else null
+
+    val properties: DataMap =
+      if (rating != null) DataMap(s"""{"rating":${rating.get().toString}}""")
+      else DataMap()
+
+
+    val eventId = Some(getStringCol("eventId"))
+    val event = getStringCol("event")
+    val entityType = getStringCol("entityType")
+    val entityId = getStringCol("entityId")
+    val targetEntityType = getOptStringCol("targetEntityType")
+    val targetEntityId = getOptStringCol("targetEntityId")
+    val prId = getOptStringCol("prId")
+    val eventTimeZone = getOptStringCol("eventTimeZone")
+      .map(DateTimeZone.forID(_))
+      .getOrElse(EventValidation.defaultTimeZone)
+    val eventTime = new DateTime(
+      getStringCol("eventTime"), eventTimeZone)
+    val creationTimeZone = getOptStringCol("creationTimeZone")
+      .map(DateTimeZone.forID(_))
+      .getOrElse(EventValidation.defaultTimeZone)
+    val creationTime: DateTime = new DateTime(
+      getStringCol("creationTime"), creationTimeZone)
+
+
+    Event(
+      eventId = eventId,
+      event = event,
+      entityType = entityType,
+      entityId = entityId,
+      targetEntityType = targetEntityType,
+      targetEntityId = targetEntityId,
+      properties = properties,
+      eventTime = eventTime,
+      tags = Seq(),
+      prId = prId,
+      creationTime = creationTime
+    )
+  }
+
+  def eventToPut(event: Event, appId: Int): Seq[Map[String, Any]] = {
+    Seq(
+      Map(
+        "eventId" -> event.eventId,
+        "event" -> event.event,
+        "entityType" -> event.entityType,
+        "entityId" -> event.entityId,
+        "targetEntityType" -> event.targetEntityType,
+        "targetEntityId" -> event.targetEntityId,
+        "properties" -> event.properties,
+        "eventTime" -> event.eventTime,
+        "tags" -> event.tags,
+        "prId" -> event.prId,
+        "creationTime" -> event.creationTime
+      )
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/36b79d7d/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
new file mode 100644
index 0000000..ef25204
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
@@ -0,0 +1,269 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch
+
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
+
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.LEvents
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.elasticsearch.client.RestClient
+import org.joda.time.DateTime
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+import org.json4s.ext.JodaTimeSerializers
+
+import grizzled.slf4j.Logging
+import org.elasticsearch.client.ResponseException
+
+class ESLEvents(val client: RestClient, config: StorageClientConfig, val index: String)
+    extends LEvents with Logging {
+  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
+  private val seq = new ESSequences(client, config, index)
+  private val seqName = "events"
+
+  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
+    channelId.map { ch =>
+      s"${appId}_${ch}"
+    }.getOrElse {
+      s"${appId}"
+    }
+  }
+
+  override def init(appId: Int, channelId: Option[Int] = None): Boolean = {
+    val estype = getEsType(appId, channelId)
+    ESUtils.createIndex(client, index)
+    val json =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("name" -> ("type" -> "keyword")) ~
+          ("eventId" -> ("type" -> "keyword")) ~
+          ("event" -> ("type" -> "keyword")) ~
+          ("entityType" -> ("type" -> "keyword")) ~
+          ("entityId" -> ("type" -> "keyword")) ~
+          ("targetEntityType" -> ("type" -> "keyword")) ~
+          ("targetEntityId" -> ("type" -> "keyword")) ~
+          ("properties" ->
+            ("type" -> "nested") ~
+            ("properties" ->
+              ("fields" -> ("type" -> "nested") ~
+                ("properties" ->
+                  ("user" -> ("type" -> "long")) ~
+                  ("num" -> ("type" -> "long")))))) ~
+                  ("eventTime" -> ("type" -> "date")) ~
+                  ("tags" -> ("type" -> "keyword")) ~
+                  ("prId" -> ("type" -> "keyword")) ~
+                  ("creationTime" -> ("type" -> "date"))))
+    ESUtils.createMapping(client, index, estype, compact(render(json)))
+    true
+  }
+
+  override def remove(appId: Int, channelId: Option[Int] = None): Boolean = {
+    val estype = getEsType(appId, channelId)
+    try {
+      val json =
+        ("query" ->
+          ("match_all" -> List.empty))
+      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+      client.performRequest(
+        "POST",
+        s"/$index/$estype/_delete_by_query",
+        Map.empty[String, String].asJava,
+        entity).getStatusLine.getStatusCode match {
+          case 200 => true
+          case _ =>
+            error(s"Failed to remove $index/$estype")
+            false
+        }
+    } catch {
+      case e: Exception =>
+        error(s"Failed to remove $index/$estype", e)
+        false
+    }
+  }
+
+  override def close(): Unit = {
+    try client.close() catch {
+      case e: Exception =>
+        error("Failed to close client.", e)
+    }
+  }
+
+  override def futureInsert(
+    event: Event,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[String] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      val id = event.eventId.getOrElse {
+        var roll = seq.genNext(seqName)
+        while (exists(estype, roll)) roll = seq.genNext(seqName)
+        roll.toString
+      }
+      val json = write(event.copy(eventId = Some(id)))
+      try {
+        val entity = new NStringEntity(json, ContentType.APPLICATION_JSON);
+        val response = client.performRequest(
+          "POST",
+          s"/$index/$estype/$id",
+          Map.empty[String, String].asJava,
+          entity)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        val result = (jsonResponse \ "result").extract[String]
+        result match {
+          case "created" => id
+          case "updated" => id
+          case _ =>
+            error(s"[$result] Failed to update $index/$estype/$id")
+            ""
+        }
+      } catch {
+        case e: IOException =>
+          error(s"Failed to update $index/$estype/$id: $json", e)
+          ""
+      }
+    }
+  }
+
+  private def exists(estype: String, id: Int): Boolean = {
+    try {
+      client.performRequest(
+        "GET",
+        s"/$index/$estype/$id",
+        Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+          case 200 => true
+          case _ => false
+        }
+    } catch {
+      case e: ResponseException =>
+        e.getResponse.getStatusLine.getStatusCode match {
+          case 404 => false
+          case _ =>
+            error(s"Failed to access to /$index/$estype/$id", e)
+            false
+        }
+      case e: IOException =>
+        error(s"Failed to access to $index/$estype/$id", e)
+        false
+    }
+  }
+
+  override def futureGet(
+    eventId: String,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Option[Event]] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      try {
+        val json =
+          ("query" ->
+            ("term" ->
+              ("eventId" -> eventId)))
+        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+        val response = client.performRequest(
+          "POST",
+          s"/$index/$estype/_search",
+          Map.empty[String, String].asJava,
+          entity)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        (jsonResponse \ "hits" \ "total").extract[Long] match {
+          case 0 => None
+          case _ =>
+            val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
+            val result = (results.head \ "_source").extract[Event]
+            Some(result)
+        }
+      } catch {
+        case e: IOException =>
+          error("Failed to access to /$index/$estype/_search", e)
+          None
+      }
+    }
+  }
+
+  override def futureDelete(
+    eventId: String,
+    appId: Int,
+    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Boolean] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      try {
+        val json =
+          ("query" ->
+            ("term" ->
+              ("eventId" -> eventId)))
+        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+        val response = client.performRequest(
+          "POST",
+          s"/$index/$estype/_delete_by_query",
+          Map.empty[String, String].asJava)
+        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+        val result = (jsonResponse \ "result").extract[String]
+        result match {
+          case "deleted" => true
+          case _ =>
+            error(s"[$result] Failed to update $index/$estype:$eventId")
+            false
+        }
+      } catch {
+        case e: IOException =>
+          error(s"Failed to update $index/$estype:$eventId", e)
+          false
+      }
+    }
+  }
+
+  override def futureFind(
+    appId: Int,
+    channelId: Option[Int] = None,
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None,
+    limit: Option[Int] = None,
+    reversed: Option[Boolean] = None)
+    (implicit ec: ExecutionContext): Future[Iterator[Event]] = {
+    Future {
+      val estype = getEsType(appId, channelId)
+      try {
+        val query = ESUtils.createEventQuery(
+          startTime, untilTime, entityType, entityId,
+          eventNames, targetEntityType, targetEntityId, None)
+        ESUtils.getAll[Event](client, index, estype, query).toIterator
+      } catch {
+        case e: IOException =>
+          error(e.getMessage)
+          Iterator[Event]()
+      }
+    }
+  }
+
+}


[06/10] incubator-predictionio git commit: Fix run_docker.sh for using Elasticsearch1.x as METADATA_SOURCE

Posted by do...@apache.org.
Fix run_docker.sh for using Elasticsearch1.x as METADATA_SOURCE


Project: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/commit/89c9d266
Tree: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/tree/89c9d266
Diff: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/diff/89c9d266

Branch: refs/heads/feature/es5
Commit: 89c9d2663c82fb3071fa6d2cd02c30c9e646ecbe
Parents: bad2f03
Author: takahiro-hagino <ta...@bizreach.co.jp>
Authored: Wed Jan 18 17:22:59 2017 +0900
Committer: takahiro-hagino <ta...@bizreach.co.jp>
Committed: Wed Jan 18 17:24:40 2017 +0900

----------------------------------------------------------------------
 tests/run_docker.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/89c9d266/tests/run_docker.sh
----------------------------------------------------------------------
diff --git a/tests/run_docker.sh b/tests/run_docker.sh
index a8a79f3..9f87ae9 100755
--- a/tests/run_docker.sh
+++ b/tests/run_docker.sh
@@ -18,18 +18,18 @@
 
 USAGE=$"Usage: run_docer <meta> <event> <model> <pio> <command>
   Where:
-    meta         = [PGSQL,ELASTICSEARCH]
+    meta         = [PGSQL,ELASTICSEARCH,ELASTICSEARCH1]
     event        = [PGSQL,HBASE]
     model        = [PGSQL,LOCALFS,HDFS]
     pio          = path to PredictionIO directory
     command      = command to run in the container"
 
-if ! [[ "$1" =~ ^(PGSQL|ELASTICSEARCH)$ ]]; then
+if ! [[ "$1" =~ ^(PGSQL|ELASTICSEARCH|ELASTICSEARCH1)$ ]]; then
   echo "$USAGE"
   exit 1
 fi
 
-if ! [[ "$2" =~ ^(PGSQL|HBASE)$ ]]; then
+if ! [[ "$2" =~ ^(PGSQL|HBASE|ELASTICSEARCH)$ ]]; then
   echo "$USAGE"
   exit 1
 fi


[08/10] incubator-predictionio git commit: [PIO-49] Rename Elasticsearch packages

Posted by do...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESPEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESPEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESPEvents.scala
new file mode 100644
index 0000000..7f5dd9a
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESPEvents.scala
@@ -0,0 +1,145 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.MapWritable
+import org.apache.hadoop.io.Text
+import org.apache.predictionio.data.storage.Event
+import org.apache.predictionio.data.storage.PEvents
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.elasticsearch.client.RestClient
+import org.elasticsearch.hadoop.mr.EsInputFormat
+import org.elasticsearch.spark._
+import org.joda.time.DateTime
+import java.io.IOException
+import org.apache.http.util.EntityUtils
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.entity.ContentType
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.ext.JodaTimeSerializers
+
+
+class ESPEvents(client: ESClient, config: StorageClientConfig, index: String)
+    extends PEvents {
+  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
+
+  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
+    channelId.map { ch =>
+      s"${appId}_${ch}"
+    }.getOrElse {
+      s"${appId}"
+    }
+  }
+
+  def getESNodes(): String = {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
+    val schemes = config.properties.get("SCHEMES").
+      map(_.split(",").toSeq).getOrElse(Seq("http"))
+    (hosts, ports, schemes).zipped.map(
+      (h, p, s) => s"$s://$h:$p").mkString(",")
+  }
+
+  override def find(
+    appId: Int,
+    channelId: Option[Int] = None,
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None)(sc: SparkContext): RDD[Event] = {
+
+    val query = ESUtils.createEventQuery(
+      startTime, untilTime, entityType, entityId,
+      eventNames, targetEntityType, targetEntityId, None)
+
+    val estype = getEsType(appId, channelId)
+    val conf = new Configuration()
+    conf.set("es.resource", s"$index/$estype")
+    conf.set("es.query", query)
+    conf.set("es.nodes", getESNodes())
+
+    val rdd = sc.newAPIHadoopRDD(conf, classOf[EsInputFormat[Text, MapWritable]],
+      classOf[Text], classOf[MapWritable]).map {
+        case (key, doc) => {
+          ESEventsUtil.resultToEvent(key, doc, appId)
+        }
+      }
+
+    rdd
+  }
+
+  override def write(
+    events: RDD[Event],
+    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
+    val estype = getEsType(appId, channelId)
+    events.map { event =>
+      ESEventsUtil.eventToPut(event, appId)
+    }.saveToEs(s"$index/$estype")
+  }
+
+  override def delete(
+    eventIds: RDD[String],
+    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
+    val estype = getEsType(appId, channelId)
+    val restClient = client.open()
+    try {
+      eventIds.foreachPartition { iter =>
+        iter.foreach { eventId =>
+          try {
+            val json =
+              ("query" ->
+                ("term" ->
+                  ("eventId" -> eventId)))
+            val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
+            val response = restClient.performRequest(
+              "POST",
+              s"/$index/$estype/_delete_by_query",
+              Map.empty[String, String].asJava)
+            val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+            val result = (jsonResponse \ "result").extract[String]
+            result match {
+              case "deleted" => true
+              case _ =>
+                logger.error(s"[$result] Failed to update $index/$estype:$eventId")
+                false
+            }
+          } catch {
+            case e: IOException =>
+              logger.error(s"Failed to update $index/$estype:$eventId", e)
+              false
+          }
+        }
+      }
+    } finally {
+      restClient.close()
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESSequences.scala
new file mode 100644
index 0000000..ae83e6a
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESSequences.scala
@@ -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.predictionio.data.storage.elasticsearch5
+
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import org.apache.http.Header
+import org.apache.http.entity.ContentType
+import org.apache.http.nio.entity.NStringEntity
+import org.apache.http.util.EntityUtils
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.write
+
+import grizzled.slf4j.Logging
+
+class ESSequences(client: ESClient, config: StorageClientConfig, index: String) extends Logging {
+  implicit val formats = DefaultFormats
+  private val estype = "sequences"
+
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
+
+  def genNext(name: String): Int = {
+    val restClient = client.open()
+    try {
+      val entity = new NStringEntity(write("n" -> name), ContentType.APPLICATION_JSON)
+      val response = restClient.performRequest(
+        "POST",
+        s"/$index/$estype/$name",
+        Map.empty[String, String].asJava,
+        entity)
+      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
+      val result = (jsonResponse \ "result").extract[String]
+      result match {
+        case "created" =>
+          (jsonResponse \ "_version").extract[Int]
+        case "updated" =>
+          (jsonResponse \ "_version").extract[Int]
+        case _ =>
+          throw new IllegalStateException(s"[$result] Failed to update $index/$estype/$name")
+      }
+    } catch {
+      case e: IOException =>
+        throw new StorageClientException(s"Failed to update $index/$estype/$name", e)
+    } finally {
+      restClient.close()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESUtils.scala
new file mode 100644
index 0000000..ed46822
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/ESUtils.scala
@@ -0,0 +1,163 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+import org.apache.http.entity.ContentType
+import org.apache.http.entity.StringEntity
+import org.apache.http.nio.entity.NStringEntity
+import org.elasticsearch.client.RestClient
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
+import org.apache.http.util.EntityUtils
+import org.joda.time.DateTime
+import org.joda.time.format.DateTimeFormat
+import org.joda.time.DateTimeZone
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.http.HttpHost
+
+object ESUtils {
+  val scrollLife = "1m"
+
+  def getAll[T: Manifest](
+    client: RestClient,
+    index: String,
+    estype: String,
+    query: String)(
+      implicit formats: Formats): Seq[T] = {
+
+    @scala.annotation.tailrec
+    def scroll(scrollId: String, hits: Seq[JValue], results: Seq[T]): Seq[T] = {
+      if (hits.isEmpty) results
+      else {
+        val json = ("scroll" -> scrollLife) ~ ("scroll_id" -> scrollId)
+        val scrollBody = new StringEntity(compact(render(json)))
+        val response = client.performRequest(
+          "POST",
+          "/_search/scroll",
+          Map[String, String](),
+          scrollBody)
+        val responseJValue = parse(EntityUtils.toString(response.getEntity))
+        scroll((responseJValue \ "_scroll_id").extract[String],
+          (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
+          hits.map(h => (h \ "_source").extract[T]) ++ results)
+      }
+    }
+
+    val response = client.performRequest(
+      "POST",
+      s"/$index/$estype/_search",
+      Map("scroll" -> scrollLife),
+      new StringEntity(query))
+    val responseJValue = parse(EntityUtils.toString(response.getEntity))
+    scroll((responseJValue \ "_scroll_id").extract[String],
+        (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
+        Nil)
+  }
+
+  def createIndex(
+    client: RestClient,
+    index: String): Unit = {
+    client.performRequest(
+      "HEAD",
+      s"/$index",
+      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+        case 404 =>
+          client.performRequest(
+            "PUT",
+            s"/$index",
+            Map.empty[String, String].asJava)
+        case 200 =>
+        case _ =>
+          throw new IllegalStateException(s"/$index is invalid.")
+      }
+  }
+
+  def createMapping(
+    client: RestClient,
+    index: String,
+    estype: String,
+    json: String): Unit = {
+    client.performRequest(
+      "HEAD",
+      s"/$index/_mapping/$estype",
+      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
+        case 404 =>
+          val entity = new NStringEntity(json, ContentType.APPLICATION_JSON)
+          client.performRequest(
+            "PUT",
+            s"/$index/_mapping/$estype",
+            Map.empty[String, String].asJava,
+            entity)
+        case 200 =>
+        case _ =>
+          throw new IllegalStateException(s"/$index/$estype is invalid: $json")
+      }
+  }
+
+  def createEventQuery(
+    startTime: Option[DateTime] = None,
+    untilTime: Option[DateTime] = None,
+    entityType: Option[String] = None,
+    entityId: Option[String] = None,
+    eventNames: Option[Seq[String]] = None,
+    targetEntityType: Option[Option[String]] = None,
+    targetEntityId: Option[Option[String]] = None,
+    reversed: Option[Boolean] = None): String = {
+    val mustQueries = Seq(
+      startTime.map(x => {
+        val v = DateTimeFormat
+          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
+        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
+      }),
+      untilTime.map(x => {
+        val v = DateTimeFormat
+          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
+        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
+      }),
+      entityType.map(x => s"""{"term":{"entityType":"${x}"}}"""),
+      entityId.map(x => s"""{"term":{"entityId":"${x}"}}"""),
+      targetEntityType.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityType":"${x}"}}""")),
+      targetEntityId.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityId":"${x}"}}""")),
+      eventNames
+        .map { xx => xx.map(x => "\"%s\"".format(x)) }
+        .map(x => s"""{"terms":{"event":[${x.mkString(",")}]}}""")).flatten.mkString(",")
+    val sortOrder = reversed.map(x => x match {
+      case true => "desc"
+      case _ => "asc"
+    })
+    s"""{
+       |"query":{"bool":{"must":[${mustQueries}]}},
+       |"sort":[{"eventTime":{"order":"${sortOrder}"}}]
+       |}""".stripMargin
+  }
+
+  def getHttpHosts(config: StorageClientConfig): Seq[HttpHost] = {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
+    val schemes = config.properties.get("SCHEMES").
+      map(_.split(",").toSeq).getOrElse(Seq("http"))
+    (hosts, ports, schemes).zipped.map((h, p, s) => new HttpHost(h, p, s))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/StorageClient.scala
new file mode 100644
index 0000000..540892b
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/StorageClient.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.predictionio.data.storage.elasticsearch5
+
+import org.apache.http.HttpHost
+import org.apache.predictionio.data.storage.BaseStorageClient
+import org.apache.predictionio.data.storage.StorageClientConfig
+import org.apache.predictionio.data.storage.StorageClientException
+import org.elasticsearch.client.RestClient
+
+import grizzled.slf4j.Logging
+
+case class ESClient(hosts: Seq[HttpHost]) {
+  def open(): RestClient = {
+    try {
+      RestClient.builder(hosts: _*).build()
+    } catch {
+      case e: Throwable =>
+        throw new StorageClientException(e.getMessage, e)
+    }
+  }
+}
+
+class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
+    with Logging {
+  override val prefix = "ES"
+
+  val client = ESClient(ESUtils.getHttpHosts(config))
+}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/package.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/package.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/package.scala
new file mode 100644
index 0000000..5cb423a
--- /dev/null
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch5/package.scala
@@ -0,0 +1,25 @@
+/*
+ * 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.predictionio.data.storage
+
+/** Elasticsearch implementation of storage traits, supporting meta data only
+  *
+  * @group Implementation
+  */
+package object elasticsearch5 {}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/project/Build.scala
----------------------------------------------------------------------
diff --git a/project/Build.scala b/project/Build.scala
index a8d730b..5420004 100644
--- a/project/Build.scala
+++ b/project/Build.scala
@@ -19,11 +19,11 @@ import sbt._
 import Keys._
 
 object PIOBuild extends Build {
+  val elasticsearch5Version = SettingKey[String](
+    "elasticsearch5-version",
+    "The version of Elasticsearch 5.x used for building.")
   val elasticsearchVersion = SettingKey[String](
     "elasticsearch-version",
-    "The version of Elasticsearch used for building.")
-  val elasticsearch1Version = SettingKey[String](
-    "elasticsearch1-version",
     "The version of Elasticsearch 1.x used for building.")
   val json4sVersion = SettingKey[String](
     "json4s-version",

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/tools/src/main/scala/org/apache/predictionio/tools/commands/Management.scala
----------------------------------------------------------------------
diff --git a/tools/src/main/scala/org/apache/predictionio/tools/commands/Management.scala b/tools/src/main/scala/org/apache/predictionio/tools/commands/Management.scala
index 10aca41..877da06 100644
--- a/tools/src/main/scala/org/apache/predictionio/tools/commands/Management.scala
+++ b/tools/src/main/scala/org/apache/predictionio/tools/commands/Management.scala
@@ -108,7 +108,7 @@ object Management extends EitherLogging {
     info("Inspecting Apache Spark...")
     val sparkHomePath = Common.getSparkHome(sparkHome)
     if (new File(s"$sparkHomePath/bin/spark-submit").exists) {
-      info(s"Apache Spark is installed at $sparkHome")
+      info(s"Apache Spark is installed at $sparkHomePath")
       val sparkMinVersion = "1.3.0"
       pioStatus = pioStatus.copy(
         sparkHome = sparkHomePath,


[10/10] incubator-predictionio git commit: [PIO-49] Rename Elasticsearch packages

Posted by do...@apache.org.
[PIO-49] Rename Elasticsearch packages

Rename elasticsearch1 back to elasticsearch to main backward
compatibility with existing configuration files

New ES5+ support now lives under elasticsearch5 package

Includes a minor fix to the "pio status" output not showing Spark's
proper location


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

Branch: refs/heads/feature/es5
Commit: c64941b6e11666ea160f4d19bd4a2302f988d9dd
Parents: d039dda
Author: Donald Szeto <do...@apache.org>
Authored: Sat Feb 11 13:57:44 2017 -0800
Committer: Donald Szeto <do...@apache.org>
Committed: Sat Feb 11 13:57:44 2017 -0800

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 build.sbt                                       |   4 +-
 conf/pio-env.sh.template                        |  18 +-
 core/build.sbt                                  |   4 +-
 data/build.sbt                                  |   8 +-
 .../storage/elasticsearch/ESAccessKeys.scala    | 154 ++++------
 .../data/storage/elasticsearch/ESApps.scala     | 176 ++++--------
 .../data/storage/elasticsearch/ESChannels.scala | 144 ++++------
 .../elasticsearch/ESEngineInstances.scala       | 246 +++++-----------
 .../elasticsearch/ESEvaluationInstances.scala   | 186 +++++-------
 .../storage/elasticsearch/ESEventsUtil.scala    | 125 --------
 .../data/storage/elasticsearch/ESLEvents.scala  | 286 -------------------
 .../data/storage/elasticsearch/ESPEvents.scala  | 145 ----------
 .../storage/elasticsearch/ESSequences.scala     |  71 ++---
 .../data/storage/elasticsearch/ESUtils.scala    | 157 ++--------
 .../storage/elasticsearch/StorageClient.scala   |  40 +--
 .../storage/elasticsearch1/ESAccessKeys.scala   | 119 --------
 .../data/storage/elasticsearch1/ESApps.scala    | 130 ---------
 .../storage/elasticsearch1/ESChannels.scala     | 117 --------
 .../elasticsearch1/ESEngineInstances.scala      | 158 ----------
 .../elasticsearch1/ESEngineManifests.scala      |  84 ------
 .../elasticsearch1/ESEvaluationInstances.scala  | 136 ---------
 .../storage/elasticsearch1/ESSequences.scala    |  64 -----
 .../data/storage/elasticsearch1/ESUtils.scala   |  48 ----
 .../storage/elasticsearch1/StorageClient.scala  |  50 ----
 .../data/storage/elasticsearch1/package.scala   |  25 --
 .../storage/elasticsearch5/ESAccessKeys.scala   | 175 ++++++++++++
 .../data/storage/elasticsearch5/ESApps.scala    | 194 +++++++++++++
 .../storage/elasticsearch5/ESChannels.scala     | 165 +++++++++++
 .../elasticsearch5/ESEngineInstances.scala      | 248 ++++++++++++++++
 .../elasticsearch5/ESEvaluationInstances.scala  | 194 +++++++++++++
 .../storage/elasticsearch5/ESEventsUtil.scala   | 125 ++++++++
 .../data/storage/elasticsearch5/ESLEvents.scala | 286 +++++++++++++++++++
 .../data/storage/elasticsearch5/ESPEvents.scala | 145 ++++++++++
 .../storage/elasticsearch5/ESSequences.scala    |  79 +++++
 .../data/storage/elasticsearch5/ESUtils.scala   | 163 +++++++++++
 .../storage/elasticsearch5/StorageClient.scala  |  44 +++
 .../data/storage/elasticsearch5/package.scala   |  25 ++
 project/Build.scala                             |   6 +-
 .../tools/commands/Management.scala             |   2 +-
 40 files changed, 2233 insertions(+), 2317 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 4d62999..68dee42 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -63,8 +63,8 @@ env:
   matrix:
     - BUILD_TYPE=Unit
     - BUILD_TYPE=Integration METADATA_REP=PGSQL EVENTDATA_REP=PGSQL MODELDATA_REP=PGSQL
-    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH1 EVENTDATA_REP=HBASE MODELDATA_REP=LOCALFS
-    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH1 EVENTDATA_REP=PGSQL MODELDATA_REP=HDFS
+    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH EVENTDATA_REP=HBASE MODELDATA_REP=LOCALFS
+    - BUILD_TYPE=Integration METADATA_REP=ELASTICSEARCH EVENTDATA_REP=PGSQL MODELDATA_REP=HDFS
 
 before_install:
   - unset SBT_OPTS JVM_OPTS

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/build.sbt
----------------------------------------------------------------------
diff --git a/build.sbt b/build.sbt
index ce626ce..1e9eb8a 100644
--- a/build.sbt
+++ b/build.sbt
@@ -34,9 +34,9 @@ fork in (ThisBuild, run) := true
 javacOptions in (ThisBuild, compile) ++= Seq("-source", "1.7", "-target", "1.7",
   "-Xlint:deprecation", "-Xlint:unchecked")
 
-elasticsearchVersion in ThisBuild := "5.1.2"
+elasticsearch5Version in ThisBuild := "5.1.2"
 
-elasticsearch1Version in ThisBuild := "1.7.6"
+elasticsearchVersion in ThisBuild := "1.7.6"
 
 json4sVersion in ThisBuild := "3.2.10"
 

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/conf/pio-env.sh.template
----------------------------------------------------------------------
diff --git a/conf/pio-env.sh.template b/conf/pio-env.sh.template
index 8f5d7b1..f56f137 100644
--- a/conf/pio-env.sh.template
+++ b/conf/pio-env.sh.template
@@ -84,17 +84,17 @@ PIO_STORAGE_SOURCES_PGSQL_PASSWORD=pio
 # PIO_STORAGE_SOURCES_MYSQL_PASSWORD=pio
 
 # Elasticsearch Example
+# PIO_STORAGE_SOURCES_ELASTICSEARCH5_TYPE=elasticsearch5
+# PIO_STORAGE_SOURCES_ELASTICSEARCH5_HOSTS=localhost
+# PIO_STORAGE_SOURCES_ELASTICSEARCH5_PORTS=9200
+# PIO_STORAGE_SOURCES_ELASTICSEARCH5_SCHEMES=http
+# PIO_STORAGE_SOURCES_ELASTICSEARCH5_HOME=$PIO_HOME/vendors/elasticsearch-5.1.2
+# Elasticsearch 1.x Example
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_TYPE=elasticsearch
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_CLUSTERNAME=<elasticsearch_cluster_name>
 # PIO_STORAGE_SOURCES_ELASTICSEARCH_HOSTS=localhost
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
-# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-5.1.2
-# Elasticsearch 1.x Example
-# PIO_STORAGE_SOURCES_ELASTICSEARCH1_TYPE=elasticsearch1
-# PIO_STORAGE_SOURCES_ELASTICSEARCH1_CLUSTERNAME=<elasticsearch_cluster_name>
-# PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOSTS=localhost
-# PIO_STORAGE_SOURCES_ELASTICSEARCH1_PORTS=9300
-# PIO_STORAGE_SOURCES_ELASTICSEARCH1_HOME=$PIO_HOME/vendors/elasticsearch-1.7.6
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9300
+# PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$PIO_HOME/vendors/elasticsearch-1.7.6
 
 # Local File System Example
 # PIO_STORAGE_SOURCES_LOCALFS_TYPE=localfs

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/core/build.sbt
----------------------------------------------------------------------
diff --git a/core/build.sbt b/core/build.sbt
index 305075e..b1f589d 100644
--- a/core/build.sbt
+++ b/core/build.sbt
@@ -32,8 +32,8 @@ libraryDependencies ++= Seq(
   "org.apache.spark"        %% "spark-core"       % sparkVersion.value % "provided",
   "org.apache.spark"        %% "spark-sql"        % sparkVersion.value % "provided",
   "org.clapper"             %% "grizzled-slf4j"   % "1.0.2",
-  "org.elasticsearch.client" % "rest"             % elasticsearchVersion.value,
-  "org.elasticsearch"        % "elasticsearch"    % elasticsearch1Version.value,
+  "org.elasticsearch.client" % "rest"             % elasticsearch5Version.value,
+  "org.elasticsearch"        % "elasticsearch"    % elasticsearchVersion.value,
   "org.json4s"              %% "json4s-native"    % json4sVersion.value,
   "org.json4s"              %% "json4s-ext"       % json4sVersion.value,
   "org.scalaj"              %% "scalaj-http"      % "1.1.6",

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/build.sbt
----------------------------------------------------------------------
diff --git a/data/build.sbt b/data/build.sbt
index 75d3c09..306153a 100644
--- a/data/build.sbt
+++ b/data/build.sbt
@@ -43,10 +43,10 @@ libraryDependencies ++= Seq(
   "org.apache.spark"        %% "spark-core"     % sparkVersion.value % "provided",
   "org.apache.spark"        %% "spark-sql"      % sparkVersion.value % "provided",
   "org.clapper"             %% "grizzled-slf4j" % "1.0.2",
-  "org.elasticsearch.client" % "rest"           % elasticsearchVersion.value,
-  "org.elasticsearch"        % "elasticsearch"  % elasticsearch1Version.value,
-  "org.elasticsearch"        % "elasticsearch-spark-13_2.10" % elasticsearchVersion.value % "provided",
-  "org.elasticsearch"        % "elasticsearch-hadoop-mr" % elasticsearchVersion.value,
+  "org.elasticsearch.client" % "rest"           % elasticsearch5Version.value,
+  "org.elasticsearch"        % "elasticsearch"  % elasticsearchVersion.value,
+  "org.elasticsearch"       %% "elasticsearch-spark-13" % elasticsearch5Version.value % "provided",
+  "org.elasticsearch"        % "elasticsearch-hadoop-mr" % elasticsearch5Version.value,
   "org.json4s"              %% "json4s-native"  % json4sVersion.value,
   "org.json4s"              %% "json4s-ext"     % json4sVersion.value,
   "org.postgresql"           % "postgresql"     % "9.4-1204-jdbc41",

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
index 9156fab..077168a 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
@@ -15,45 +15,44 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters.mapAsJavaMapConverter
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.StorageClientConfig
 import org.apache.predictionio.data.storage.AccessKey
 import org.apache.predictionio.data.storage.AccessKeys
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
 import org.json4s.JsonDSL._
+import org.json4s._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
+import scala.util.Random
 
 /** Elasticsearch implementation of AccessKeys. */
-class ESAccessKeys(client: ESClient, config: StorageClientConfig, index: String)
+class ESAccessKeys(client: Client, config: StorageClientConfig, index: String)
     extends AccessKeys with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "accesskeys"
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
-    val mappingJson =
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
       (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
         ("properties" ->
-          ("key" -> ("type" -> "keyword")) ~
-          ("events" -> ("type" -> "keyword"))))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+          ("key" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("events" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
   }
 
   def insert(accessKey: AccessKey): Option[String] = {
@@ -62,114 +61,59 @@ class ESAccessKeys(client: ESClient, config: StorageClientConfig, index: String)
     Some(key)
   }
 
-  def get(id: String): Option[AccessKey] = {
-    val restClient = client.open()
+  def get(key: String): Option[AccessKey] = {
     try {
-      val response = restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "found").extract[Boolean] match {
-        case true =>
-          Some((jsonResponse \ "_source").extract[AccessKey])
-        case _ =>
-          None
-      }
+      val response = client.prepareGet(
+        index,
+        estype,
+        key).get()
+      Some(read[AccessKey](response.getSourceAsString))
     } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => None
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            None
-        }
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/$key", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
+      case e: NullPointerException => None
     }
   }
 
   def getAll(): Seq[AccessKey] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("match_all" -> List.empty))
-      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[AccessKey](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[AccessKey]()
     }
   }
 
   def getByAppid(appid: Int): Seq[AccessKey] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("term" ->
-            ("appid" -> appid)))
-      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("appid", appid))
+      ESUtils.getAll[AccessKey](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[AccessKey]()
     }
   }
 
   def update(accessKey: AccessKey): Unit = {
-    val id = accessKey.key
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write(accessKey), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "created" =>
-        case "updated" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareIndex(index, estype, accessKey.key).setSource(write(accessKey)).get()
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
     }
   }
 
-  def delete(id: String): Unit = {
-    val restClient = client.open()
+  def delete(key: String): Unit = {
     try {
-      val response = restClient.performRequest(
-        "DELETE",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "deleted" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/id")
-      }
+      client.prepareDelete(index, estype, key).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
index 0379c90..3781a4b 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
@@ -15,180 +15,116 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters.mapAsJavaMapConverter
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
+import org.apache.predictionio.data.storage.StorageClientConfig
 import org.apache.predictionio.data.storage.App
 import org.apache.predictionio.data.storage.Apps
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
 import org.json4s.JsonDSL._
+import org.json4s._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
-
 /** Elasticsearch implementation of Items. */
-class ESApps(client: ESClient, config: StorageClientConfig, index: String)
-    extends Apps with Logging {
+class ESApps(client: Client, config: StorageClientConfig, index: String)
+  extends Apps with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "apps"
   private val seq = new ESSequences(client, config, index)
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
-    val mappingJson =
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
       (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
         ("properties" ->
-          ("id" -> ("type" -> "keyword")) ~
-          ("name" -> ("type" -> "keyword"))))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
   }
 
   def insert(app: App): Option[Int] = {
     val id =
       if (app.id == 0) {
-        var roll = seq.genNext(estype)
-        while (!get(roll).isEmpty) roll = seq.genNext(estype)
+        var roll = seq.genNext("apps")
+        while (!get(roll).isEmpty) roll = seq.genNext("apps")
         roll
-      } else app.id
-    update(app.copy(id = id))
+      }
+      else app.id
+    val realapp = app.copy(id = id)
+    update(realapp)
     Some(id)
   }
 
   def get(id: Int): Option[App] = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "found").extract[Boolean] match {
-        case true =>
-          Some((jsonResponse \ "_source").extract[App])
-        case _ =>
-          None
-      }
+      val response = client.prepareGet(
+        index,
+        estype,
+        id.toString).get()
+      Some(read[App](response.getSourceAsString))
     } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => None
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            None
-        }
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/$id", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
+      case e: NullPointerException => None
     }
   }
 
   def getByName(name: String): Option[App] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("term" ->
-            ("name" -> name)))
-      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/_search",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "hits" \ "total").extract[Long] match {
-        case 0 => None
-        case _ =>
-          val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
-          val result = (results.head \ "_source").extract[App]
-          Some(result)
+      val response = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("name", name)).get
+      val hits = response.getHits().hits()
+      if (hits.size > 0) {
+        Some(read[App](hits.head.getSourceAsString))
+      } else {
+        None
       }
     } catch {
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/_search", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
     }
   }
 
   def getAll(): Seq[App] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("match_all" -> List.empty))
-      ESUtils.getAll[App](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[App](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[App]()
     }
   }
 
   def update(app: App): Unit = {
-    val id = app.id.toString
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write(app), ContentType.APPLICATION_JSON);
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "created" =>
-        case "updated" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      val response = client.prepareIndex(index, estype, app.id.toString).
+        setSource(write(app)).get()
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
     }
   }
 
   def delete(id: Int): Unit = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "DELETE",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "deleted" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareDelete(index, estype, id.toString).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
index b319c26..52697fd 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
@@ -15,151 +15,103 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters.mapAsJavaMapConverter
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.Channel
 import org.apache.predictionio.data.storage.Channels
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders.termFilter
+import org.json4s.DefaultFormats
 import org.json4s.JsonDSL._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
-
-class ESChannels(client: ESClient, config: StorageClientConfig, index: String)
+class ESChannels(client: Client, config: StorageClientConfig, index: String)
     extends Channels with Logging {
+
   implicit val formats = DefaultFormats.lossless
   private val estype = "channels"
   private val seq = new ESSequences(client, config, index)
+  private val seqName = "channels"
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
-    val mappingJson =
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
       (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
         ("properties" ->
-          ("name" -> ("type" -> "keyword"))))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+          ("name" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
   }
 
   def insert(channel: Channel): Option[Int] = {
     val id =
       if (channel.id == 0) {
-        var roll = seq.genNext(estype)
-        while (!get(roll).isEmpty) roll = seq.genNext(estype)
+        var roll = seq.genNext(seqName)
+        while (!get(roll).isEmpty) roll = seq.genNext(seqName)
         roll
       } else channel.id
 
-    if (update(channel.copy(id = id))) Some(id) else None
+    val realChannel = channel.copy(id = id)
+    if (update(realChannel)) Some(id) else None
   }
 
   def get(id: Int): Option[Channel] = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "found").extract[Boolean] match {
-        case true =>
-          Some((jsonResponse \ "_source").extract[Channel])
-        case _ =>
-          None
-      }
+      val response = client.prepareGet(
+        index,
+        estype,
+        id.toString).get()
+      Some(read[Channel](response.getSourceAsString))
     } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => None
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            None
-        }
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/$id", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
+      case e: NullPointerException => None
     }
   }
 
   def getByAppid(appid: Int): Seq[Channel] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("term" ->
-            ("appid" -> appid)))
-      ESUtils.getAll[Channel](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype).
+        setPostFilter(termFilter("appid", appid))
+      ESUtils.getAll[Channel](client, builder)
     } catch {
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq[Channel]()
     }
   }
 
   def update(channel: Channel): Boolean = {
-    val id = channel.id.toString
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write(channel), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava,
-        entity)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "created" => true
-        case "updated" => true
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-          false
-      }
+      val response = client.prepareIndex(index, estype, channel.id.toString).
+        setSource(write(channel)).get()
+      true
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         false
-    } finally {
-      restClient.close()
     }
   }
 
   def delete(id: Int): Unit = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "DELETE",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "deleted" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareDelete(index, estype, id.toString).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
index 68cdeac..21690bf 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
@@ -15,234 +15,144 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters.mapAsJavaMapConverter
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.EngineInstance
 import org.apache.predictionio.data.storage.EngineInstanceSerializer
 import org.apache.predictionio.data.storage.EngineInstances
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.elasticsearch.search.sort.SortOrder
 import org.json4s.JsonDSL._
+import org.json4s._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
-
-class ESEngineInstances(client: ESClient, config: StorageClientConfig, index: String)
-    extends EngineInstances with Logging {
+class ESEngineInstances(client: Client, config: StorageClientConfig, index: String)
+  extends EngineInstances with Logging {
   implicit val formats = DefaultFormats + new EngineInstanceSerializer
   private val estype = "engine_instances"
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
-    val mappingJson =
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
       (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
         ("properties" ->
-          ("status" -> ("type" -> "keyword")) ~
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
           ("startTime" -> ("type" -> "date")) ~
           ("endTime" -> ("type" -> "date")) ~
-          ("engineId" -> ("type" -> "keyword")) ~
-          ("engineVersion" -> ("type" -> "keyword")) ~
-          ("engineVariant" -> ("type" -> "keyword")) ~
-          ("engineFactory" -> ("type" -> "keyword")) ~
-          ("batch" -> ("type" -> "keyword")) ~
-          ("dataSourceParams" -> ("type" -> "keyword")) ~
-          ("preparatorParams" -> ("type" -> "keyword")) ~
-          ("algorithmsParams" -> ("type" -> "keyword")) ~
-          ("servingParams" -> ("type" -> "keyword")) ~
-          ("status" -> ("type" -> "keyword"))))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+          ("engineId" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineVersion" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineVariant" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineFactory" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("batch" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("dataSourceParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("preparatorParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("algorithmsParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("servingParams" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
   }
 
   def insert(i: EngineInstance): String = {
-    val id = i.id match {
-      case x if x.isEmpty =>
-        @scala.annotation.tailrec
-        def generateId(newId: Option[String]): String = {
-          newId match {
-            case Some(x) => x
-            case _ => generateId(preInsert())
-          }
-        }
-        generateId(preInsert())
-      case x => x
-    }
-
-    update(i.copy(id = id))
-    id
-  }
-
-  def preInsert(): Option[String] = {
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity("{}", ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "created" =>
-          Some((jsonResponse \ "_id").extract[String])
-        case _ =>
-          error(s"[$result] Failed to create $index/$estype")
-          None
-      }
+      val response = client.prepareIndex(index, estype).
+        setSource(write(i)).get
+      response.getId
     } catch {
-      case e: IOException =>
-        error(s"Failed to create $index/$estype", e)
-        None
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        ""
     }
   }
 
   def get(id: String): Option[EngineInstance] = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "found").extract[Boolean] match {
-        case true =>
-          Some((jsonResponse \ "_source").extract[EngineInstance])
-        case _ =>
-          None
+      val response = client.prepareGet(index, estype, id).get
+      if (response.isExists) {
+        Some(read[EngineInstance](response.getSourceAsString))
+      } else {
+        None
       }
     } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => None
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            None
-        }
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/$id", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
     }
   }
 
   def getAll(): Seq[EngineInstance] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("match_all" -> List.empty))
-      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[EngineInstance](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
     }
   }
 
   def getCompleted(
-    engineId: String,
-    engineVersion: String,
-    engineVariant: String): Seq[EngineInstance] = {
-    val restClient = client.open()
+      engineId: String,
+      engineVersion: String,
+      engineVariant: String): Seq[EngineInstance] = {
     try {
-      val json =
-        ("query" ->
-          ("bool" ->
-            ("must" -> List(
-              ("term" ->
-                ("status" -> "COMPLETED")),
-              ("term" ->
-                ("engineId" -> engineId)),
-              ("term" ->
-                ("engineVersion" -> engineVersion)),
-              ("term" ->
-                ("engineVariant" -> engineVariant)))))) ~
-              ("sort" -> List(
-                ("startTime" ->
-                  ("order" -> "desc"))))
-      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
+        andFilter(
+          termFilter("status", "COMPLETED"),
+          termFilter("engineId", engineId),
+          termFilter("engineVersion", engineVersion),
+          termFilter("engineVariant", engineVariant))).
+        addSort("startTime", SortOrder.DESC)
+      ESUtils.getAll[EngineInstance](client, builder)
     } catch {
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
     }
   }
 
   def getLatestCompleted(
-    engineId: String,
-    engineVersion: String,
-    engineVariant: String): Option[EngineInstance] =
+      engineId: String,
+      engineVersion: String,
+      engineVariant: String): Option[EngineInstance] =
     getCompleted(
       engineId,
       engineVersion,
       engineVariant).headOption
 
   def update(i: EngineInstance): Unit = {
-    val id = i.id
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "created" =>
-        case "updated" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException => error(e.getMessage)
     }
   }
 
   def delete(id: String): Unit = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "DELETE",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "deleted" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      val response = client.prepareDelete(index, estype, id).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException => error(e.getMessage)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
index 1f798f0..85bf820 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
@@ -15,180 +15,122 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters._
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.EvaluationInstance
 import org.apache.predictionio.data.storage.EvaluationInstanceSerializer
 import org.apache.predictionio.data.storage.EvaluationInstances
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.StorageClientException
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
+import org.elasticsearch.index.query.FilterBuilders._
+import org.elasticsearch.search.sort.SortOrder
 import org.json4s.JsonDSL._
+import org.json4s._
 import org.json4s.native.JsonMethods._
+import org.json4s.native.Serialization.read
 import org.json4s.native.Serialization.write
 
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
-
-class ESEvaluationInstances(client: ESClient, config: StorageClientConfig, index: String)
-    extends EvaluationInstances with Logging {
+class ESEvaluationInstances(client: Client, config: StorageClientConfig, index: String)
+  extends EvaluationInstances with Logging {
   implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
   private val estype = "evaluation_instances"
-  private val seq = new ESSequences(client, config, index)
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
-    val mappingJson =
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
+    val json =
       (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
         ("properties" ->
-          ("status" -> ("type" -> "keyword")) ~
+          ("status" -> ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
           ("startTime" -> ("type" -> "date")) ~
           ("endTime" -> ("type" -> "date")) ~
-          ("evaluationClass" -> ("type" -> "keyword")) ~
-          ("engineParamsGeneratorClass" -> ("type" -> "keyword")) ~
-          ("batch" -> ("type" -> "keyword")) ~
-          ("evaluatorResults" -> ("type" -> "text") ~ ("index" -> "no")) ~
-          ("evaluatorResultsHTML" -> ("type" -> "text") ~ ("index" -> "no")) ~
-          ("evaluatorResultsJSON" -> ("type" -> "text") ~ ("index" -> "no"))))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+          ("evaluationClass" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("engineParamsGeneratorClass" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("batch" ->
+            ("type" -> "string") ~ ("index" -> "not_analyzed")) ~
+          ("evaluatorResults" ->
+            ("type" -> "string") ~ ("index" -> "no")) ~
+          ("evaluatorResultsHTML" ->
+            ("type" -> "string") ~ ("index" -> "no")) ~
+          ("evaluatorResultsJSON" ->
+            ("type" -> "string") ~ ("index" -> "no"))))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(json))).get
   }
 
   def insert(i: EvaluationInstance): String = {
-    val id = i.id match {
-      case x if x.isEmpty =>
-        var roll = seq.genNext(estype).toString
-        while (!get(roll).isEmpty) roll = seq.genNext(estype).toString
-        roll
-      case x => x
+    try {
+      val response = client.prepareIndex(index, estype).
+        setSource(write(i)).get
+      response.getId
+    } catch {
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        ""
     }
-
-    update(i.copy(id = id))
-    id
   }
 
   def get(id: String): Option[EvaluationInstance] = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      (jsonResponse \ "found").extract[Boolean] match {
-        case true =>
-          Some((jsonResponse \ "_source").extract[EvaluationInstance])
-        case _ =>
-          None
+      val response = client.prepareGet(index, estype, id).get
+      if (response.isExists) {
+        Some(read[EvaluationInstance](response.getSourceAsString))
+      } else {
+        None
       }
     } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => None
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            None
-        }
-      case e: IOException =>
-        error(s"Failed to access to /$index/$estype/$id", e)
+      case e: ElasticsearchException =>
+        error(e.getMessage)
         None
-    } finally {
-      restClient.close()
     }
   }
 
   def getAll(): Seq[EvaluationInstance] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("match_all" -> List.empty))
-      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype)
+      ESUtils.getAll[EvaluationInstance](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
     }
   }
 
   def getCompleted(): Seq[EvaluationInstance] = {
-    val restClient = client.open()
     try {
-      val json =
-        ("query" ->
-          ("term" ->
-            ("status" -> "EVALCOMPLETED"))) ~
-            ("sort" ->
-              ("startTime" ->
-                ("order" -> "desc")))
-      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
+      val builder = client.prepareSearch(index).setTypes(estype).setPostFilter(
+        termFilter("status", "EVALCOMPLETED")).
+        addSort("startTime", SortOrder.DESC)
+      ESUtils.getAll[EvaluationInstance](client, builder)
     } catch {
-      case e: IOException =>
-        error("Failed to access to /$index/$estype/_search", e)
-        Nil
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        Seq()
     }
   }
 
   def update(i: EvaluationInstance): Unit = {
-    val id = i.id
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava,
-        entity)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "created" =>
-        case "updated" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareUpdate(index, estype, i.id).setDoc(write(i)).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException => error(e.getMessage)
     }
   }
 
   def delete(id: String): Unit = {
-    val restClient = client.open()
     try {
-      val response = restClient.performRequest(
-        "DELETE",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava)
-      val json = parse(EntityUtils.toString(response.getEntity))
-      val result = (json \ "result").extract[String]
-      result match {
-        case "deleted" =>
-        case _ =>
-          error(s"[$result] Failed to update $index/$estype/$id")
-      }
+      client.prepareDelete(index, estype, id).get
     } catch {
-      case e: IOException =>
-        error(s"Failed to update $index/$estype/$id", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException => error(e.getMessage)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
deleted file mode 100644
index f2ab7c2..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEventsUtil.scala
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch
-
-import org.apache.hadoop.io.DoubleWritable
-import org.apache.hadoop.io.LongWritable
-import org.apache.hadoop.io.MapWritable
-import org.apache.hadoop.io.Text
-import org.apache.predictionio.data.storage.DataMap
-import org.apache.predictionio.data.storage.Event
-import org.apache.predictionio.data.storage.EventValidation
-import org.joda.time.DateTime
-import org.joda.time.DateTimeZone
-import org.json4s._
-
-object ESEventsUtil {
-
-  implicit val formats = DefaultFormats
-
-  def resultToEvent(id: Text, result: MapWritable, appId: Int): Event = {
-
-    def getStringCol(col: String): String = {
-      val r = result.get(new Text(col)).asInstanceOf[Text]
-      require(r != null,
-        s"Failed to get value for column ${col}. " +
-          s"StringBinary: ${r.getBytes()}.")
-
-      r.toString()
-    }
-
-    def getOptStringCol(col: String): Option[String] = {
-      val r = result.get(new Text(col))
-      if (r == null) {
-        None
-      } else {
-        Some(r.asInstanceOf[Text].toString())
-      }
-    }
-
-    val tmp = result
-      .get(new Text("properties")).asInstanceOf[MapWritable]
-      .get(new Text("fields")).asInstanceOf[MapWritable]
-      .get(new Text("rating"))
-
-    val rating =
-      if (tmp.isInstanceOf[DoubleWritable]) tmp.asInstanceOf[DoubleWritable]
-      else if (tmp.isInstanceOf[LongWritable]) {
-        new DoubleWritable(tmp.asInstanceOf[LongWritable].get().toDouble)
-      }
-      else null
-
-    val properties: DataMap =
-      if (rating != null) DataMap(s"""{"rating":${rating.get().toString}}""")
-      else DataMap()
-
-
-    val eventId = Some(getStringCol("eventId"))
-    val event = getStringCol("event")
-    val entityType = getStringCol("entityType")
-    val entityId = getStringCol("entityId")
-    val targetEntityType = getOptStringCol("targetEntityType")
-    val targetEntityId = getOptStringCol("targetEntityId")
-    val prId = getOptStringCol("prId")
-    val eventTimeZone = getOptStringCol("eventTimeZone")
-      .map(DateTimeZone.forID(_))
-      .getOrElse(EventValidation.defaultTimeZone)
-    val eventTime = new DateTime(
-      getStringCol("eventTime"), eventTimeZone)
-    val creationTimeZone = getOptStringCol("creationTimeZone")
-      .map(DateTimeZone.forID(_))
-      .getOrElse(EventValidation.defaultTimeZone)
-    val creationTime: DateTime = new DateTime(
-      getStringCol("creationTime"), creationTimeZone)
-
-
-    Event(
-      eventId = eventId,
-      event = event,
-      entityType = entityType,
-      entityId = entityId,
-      targetEntityType = targetEntityType,
-      targetEntityId = targetEntityId,
-      properties = properties,
-      eventTime = eventTime,
-      tags = Seq(),
-      prId = prId,
-      creationTime = creationTime
-    )
-  }
-
-  def eventToPut(event: Event, appId: Int): Seq[Map[String, Any]] = {
-    Seq(
-      Map(
-        "eventId" -> event.eventId,
-        "event" -> event.event,
-        "entityType" -> event.entityType,
-        "entityId" -> event.entityId,
-        "targetEntityType" -> event.targetEntityType,
-        "targetEntityId" -> event.targetEntityId,
-        "properties" -> event.properties,
-        "eventTime" -> event.eventTime,
-        "tags" -> event.tags,
-        "prId" -> event.prId,
-        "creationTime" -> event.creationTime
-      )
-    )
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
deleted file mode 100644
index b4f7dc5..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
-
-import scala.collection.JavaConverters._
-import scala.concurrent.ExecutionContext
-import scala.concurrent.Future
-
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
-import org.apache.predictionio.data.storage.Event
-import org.apache.predictionio.data.storage.LEvents
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.elasticsearch.client.RestClient
-import org.joda.time.DateTime
-import org.json4s._
-import org.json4s.JsonDSL._
-import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.write
-import org.json4s.ext.JodaTimeSerializers
-
-import grizzled.slf4j.Logging
-import org.elasticsearch.client.ResponseException
-
-class ESLEvents(val client: ESClient, config: StorageClientConfig, val index: String)
-    extends LEvents with Logging {
-  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
-  private val seq = new ESSequences(client, config, index)
-  private val seqName = "events"
-
-  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
-    channelId.map { ch =>
-      s"${appId}_${ch}"
-    }.getOrElse {
-      s"${appId}"
-    }
-  }
-
-  override def init(appId: Int, channelId: Option[Int] = None): Boolean = {
-    val estype = getEsType(appId, channelId)
-    val restClient = client.open()
-    try {
-      ESUtils.createIndex(restClient, index)
-      val json =
-        (estype ->
-          ("_all" -> ("enabled" -> 0)) ~
-          ("properties" ->
-            ("name" -> ("type" -> "keyword")) ~
-            ("eventId" -> ("type" -> "keyword")) ~
-            ("event" -> ("type" -> "keyword")) ~
-            ("entityType" -> ("type" -> "keyword")) ~
-            ("entityId" -> ("type" -> "keyword")) ~
-            ("targetEntityType" -> ("type" -> "keyword")) ~
-            ("targetEntityId" -> ("type" -> "keyword")) ~
-            ("properties" ->
-              ("type" -> "nested") ~
-              ("properties" ->
-                ("fields" -> ("type" -> "nested") ~
-                  ("properties" ->
-                    ("user" -> ("type" -> "long")) ~
-                    ("num" -> ("type" -> "long")))))) ~
-                    ("eventTime" -> ("type" -> "date")) ~
-                    ("tags" -> ("type" -> "keyword")) ~
-                    ("prId" -> ("type" -> "keyword")) ~
-                    ("creationTime" -> ("type" -> "date"))))
-      ESUtils.createMapping(restClient, index, estype, compact(render(json)))
-    } finally {
-      restClient.close()
-    }
-    true
-  }
-
-  override def remove(appId: Int, channelId: Option[Int] = None): Boolean = {
-    val estype = getEsType(appId, channelId)
-    val restClient = client.open()
-    try {
-      val json =
-        ("query" ->
-          ("match_all" -> List.empty))
-      val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-      restClient.performRequest(
-        "POST",
-        s"/$index/$estype/_delete_by_query",
-        Map.empty[String, String].asJava,
-        entity).getStatusLine.getStatusCode match {
-          case 200 => true
-          case _ =>
-            error(s"Failed to remove $index/$estype")
-            false
-        }
-    } catch {
-      case e: Exception =>
-        error(s"Failed to remove $index/$estype", e)
-        false
-    } finally {
-      restClient.close()
-    }
-  }
-
-  override def close(): Unit = {
-    // nothing
-  }
-
-  override def futureInsert(
-    event: Event,
-    appId: Int,
-    channelId: Option[Int])(implicit ec: ExecutionContext): Future[String] = {
-    Future {
-      val estype = getEsType(appId, channelId)
-      val restClient = client.open()
-      try {
-        val id = event.eventId.getOrElse {
-          var roll = seq.genNext(seqName)
-          while (exists(restClient, estype, roll)) roll = seq.genNext(seqName)
-          roll.toString
-        }
-        val json = write(event.copy(eventId = Some(id)))
-        val entity = new NStringEntity(json, ContentType.APPLICATION_JSON);
-        val response = restClient.performRequest(
-          "POST",
-          s"/$index/$estype/$id",
-          Map.empty[String, String].asJava,
-          entity)
-        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-        val result = (jsonResponse \ "result").extract[String]
-        result match {
-          case "created" => id
-          case "updated" => id
-          case _ =>
-            error(s"[$result] Failed to update $index/$estype/$id")
-            ""
-        }
-      } catch {
-        case e: IOException =>
-          error(s"Failed to update $index/$estype/<id>", e)
-          ""
-      } finally {
-        restClient.close()
-      }
-    }
-  }
-
-  private def exists(restClient: RestClient, estype: String, id: Int): Boolean = {
-    try {
-      restClient.performRequest(
-        "GET",
-        s"/$index/$estype/$id",
-        Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
-          case 200 => true
-          case _ => false
-        }
-    } catch {
-      case e: ResponseException =>
-        e.getResponse.getStatusLine.getStatusCode match {
-          case 404 => false
-          case _ =>
-            error(s"Failed to access to /$index/$estype/$id", e)
-            false
-        }
-      case e: IOException =>
-        error(s"Failed to access to $index/$estype/$id", e)
-        false
-    }
-  }
-
-  override def futureGet(
-    eventId: String,
-    appId: Int,
-    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Option[Event]] = {
-    Future {
-      val estype = getEsType(appId, channelId)
-      val restClient = client.open()
-      try {
-        val json =
-          ("query" ->
-            ("term" ->
-              ("eventId" -> eventId)))
-        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-        val response = restClient.performRequest(
-          "POST",
-          s"/$index/$estype/_search",
-          Map.empty[String, String].asJava,
-          entity)
-        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-        (jsonResponse \ "hits" \ "total").extract[Long] match {
-          case 0 => None
-          case _ =>
-            val results = (jsonResponse \ "hits" \ "hits").extract[Seq[JValue]]
-            val result = (results.head \ "_source").extract[Event]
-            Some(result)
-        }
-      } catch {
-        case e: IOException =>
-          error("Failed to access to /$index/$estype/_search", e)
-          None
-      } finally {
-        restClient.close()
-      }
-    }
-  }
-
-  override def futureDelete(
-    eventId: String,
-    appId: Int,
-    channelId: Option[Int])(implicit ec: ExecutionContext): Future[Boolean] = {
-    Future {
-      val estype = getEsType(appId, channelId)
-      val restClient = client.open()
-      try {
-        val json =
-          ("query" ->
-            ("term" ->
-              ("eventId" -> eventId)))
-        val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-        val response = restClient.performRequest(
-          "POST",
-          s"/$index/$estype/_delete_by_query",
-          Map.empty[String, String].asJava)
-        val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-        val result = (jsonResponse \ "result").extract[String]
-        result match {
-          case "deleted" => true
-          case _ =>
-            error(s"[$result] Failed to update $index/$estype:$eventId")
-            false
-        }
-      } catch {
-        case e: IOException =>
-          error(s"Failed to update $index/$estype:$eventId", e)
-          false
-      } finally {
-        restClient.close()
-      }
-    }
-  }
-
-  override def futureFind(
-    appId: Int,
-    channelId: Option[Int] = None,
-    startTime: Option[DateTime] = None,
-    untilTime: Option[DateTime] = None,
-    entityType: Option[String] = None,
-    entityId: Option[String] = None,
-    eventNames: Option[Seq[String]] = None,
-    targetEntityType: Option[Option[String]] = None,
-    targetEntityId: Option[Option[String]] = None,
-    limit: Option[Int] = None,
-    reversed: Option[Boolean] = None)
-    (implicit ec: ExecutionContext): Future[Iterator[Event]] = {
-    Future {
-      val estype = getEsType(appId, channelId)
-      val restClient = client.open()
-      try {
-        val query = ESUtils.createEventQuery(
-          startTime, untilTime, entityType, entityId,
-          eventNames, targetEntityType, targetEntityId, None)
-        ESUtils.getAll[Event](restClient, index, estype, query).toIterator
-      } catch {
-        case e: IOException =>
-          error(e.getMessage)
-          Iterator[Event]()
-      } finally {
-        restClient.close()
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
deleted file mode 100644
index 5784b3f..0000000
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.predictionio.data.storage.elasticsearch
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.MapWritable
-import org.apache.hadoop.io.Text
-import org.apache.predictionio.data.storage.Event
-import org.apache.predictionio.data.storage.PEvents
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.spark.SparkContext
-import org.apache.spark.rdd.RDD
-import org.elasticsearch.client.RestClient
-import org.elasticsearch.hadoop.mr.EsInputFormat
-import org.elasticsearch.spark._
-import org.joda.time.DateTime
-import java.io.IOException
-import org.apache.http.util.EntityUtils
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.entity.ContentType
-import org.json4s._
-import org.json4s.JsonDSL._
-import org.json4s.native.JsonMethods._
-import org.json4s.ext.JodaTimeSerializers
-
-
-class ESPEvents(client: ESClient, config: StorageClientConfig, index: String)
-    extends PEvents {
-  implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
-
-  def getEsType(appId: Int, channelId: Option[Int] = None): String = {
-    channelId.map { ch =>
-      s"${appId}_${ch}"
-    }.getOrElse {
-      s"${appId}"
-    }
-  }
-
-  def getESNodes(): String = {
-    val hosts = config.properties.get("HOSTS").
-      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
-    val ports = config.properties.get("PORTS").
-      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
-    val schemes = config.properties.get("SCHEMES").
-      map(_.split(",").toSeq).getOrElse(Seq("http"))
-    (hosts, ports, schemes).zipped.map(
-      (h, p, s) => s"$s://$h:$p").mkString(",")
-  }
-
-  override def find(
-    appId: Int,
-    channelId: Option[Int] = None,
-    startTime: Option[DateTime] = None,
-    untilTime: Option[DateTime] = None,
-    entityType: Option[String] = None,
-    entityId: Option[String] = None,
-    eventNames: Option[Seq[String]] = None,
-    targetEntityType: Option[Option[String]] = None,
-    targetEntityId: Option[Option[String]] = None)(sc: SparkContext): RDD[Event] = {
-
-    val query = ESUtils.createEventQuery(
-      startTime, untilTime, entityType, entityId,
-      eventNames, targetEntityType, targetEntityId, None)
-
-    val estype = getEsType(appId, channelId)
-    val conf = new Configuration()
-    conf.set("es.resource", s"$index/$estype")
-    conf.set("es.query", query)
-    conf.set("es.nodes", getESNodes())
-
-    val rdd = sc.newAPIHadoopRDD(conf, classOf[EsInputFormat[Text, MapWritable]],
-      classOf[Text], classOf[MapWritable]).map {
-        case (key, doc) => {
-          ESEventsUtil.resultToEvent(key, doc, appId)
-        }
-      }
-
-    rdd
-  }
-
-  override def write(
-    events: RDD[Event],
-    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
-    val estype = getEsType(appId, channelId)
-    events.map { event =>
-      ESEventsUtil.eventToPut(event, appId)
-    }.saveToEs(s"$index/$estype")
-  }
-
-  override def delete(
-    eventIds: RDD[String],
-    appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
-    val estype = getEsType(appId, channelId)
-    val restClient = client.open()
-    try {
-      eventIds.foreachPartition { iter =>
-        iter.foreach { eventId =>
-          try {
-            val json =
-              ("query" ->
-                ("term" ->
-                  ("eventId" -> eventId)))
-            val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-            val response = restClient.performRequest(
-              "POST",
-              s"/$index/$estype/_delete_by_query",
-              Map.empty[String, String].asJava)
-            val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-            val result = (jsonResponse \ "result").extract[String]
-            result match {
-              case "deleted" => true
-              case _ =>
-                logger.error(s"[$result] Failed to update $index/$estype:$eventId")
-                false
-            }
-          } catch {
-            case e: IOException =>
-              logger.error(s"Failed to update $index/$estype:$eventId", e)
-              false
-          }
-        }
-      }
-    } finally {
-      restClient.close()
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
index 4eb8cd7..5c9e170 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
@@ -15,65 +15,50 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
-
-import java.io.IOException
 
-import scala.collection.JavaConverters._
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.Header
-import org.apache.http.entity.ContentType
-import org.apache.http.nio.entity.NStringEntity
-import org.apache.http.util.EntityUtils
+import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.predictionio.data.storage.StorageClientException
-import org.elasticsearch.client.RestClient
-import org.json4s._
+import org.elasticsearch.ElasticsearchException
+import org.elasticsearch.client.Client
 import org.json4s.JsonDSL._
+import org.json4s._
 import org.json4s.native.JsonMethods._
-import org.json4s.native.Serialization.write
-
-import grizzled.slf4j.Logging
 
-class ESSequences(client: ESClient, config: StorageClientConfig, index: String) extends Logging {
+class ESSequences(client: Client, config: StorageClientConfig, index: String) extends Logging {
   implicit val formats = DefaultFormats
   private val estype = "sequences"
 
-  val restClient = client.open()
-  try {
-    ESUtils.createIndex(restClient, index)
+  val indices = client.admin.indices
+  val indexExistResponse = indices.prepareExists(index).get
+  if (!indexExistResponse.isExists) {
+    // val settingsJson =
+    //   ("number_of_shards" -> 1) ~
+    //   ("auto_expand_replicas" -> "0-all")
+    indices.prepareCreate(index).get
+  }
+  val typeExistResponse = indices.prepareTypesExists(index).setTypes(estype).get
+  if (!typeExistResponse.isExists) {
     val mappingJson =
       (estype ->
-        ("_all" -> ("enabled" -> 0)))
-    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
-  } finally {
-    restClient.close()
+        ("_source" -> ("enabled" -> 0)) ~
+        ("_all" -> ("enabled" -> 0)) ~
+        ("_type" -> ("index" -> "no")) ~
+        ("enabled" -> 0))
+    indices.preparePutMapping(index).setType(estype).
+      setSource(compact(render(mappingJson))).get
   }
 
   def genNext(name: String): Int = {
-    val restClient = client.open()
     try {
-      val entity = new NStringEntity(write("n" -> name), ContentType.APPLICATION_JSON)
-      val response = restClient.performRequest(
-        "POST",
-        s"/$index/$estype/$name",
-        Map.empty[String, String].asJava,
-        entity)
-      val jsonResponse = parse(EntityUtils.toString(response.getEntity))
-      val result = (jsonResponse \ "result").extract[String]
-      result match {
-        case "created" =>
-          (jsonResponse \ "_version").extract[Int]
-        case "updated" =>
-          (jsonResponse \ "_version").extract[Int]
-        case _ =>
-          throw new IllegalStateException(s"[$result] Failed to update $index/$estype/$name")
-      }
+      val response = client.prepareIndex(index, estype, name).
+        setSource(compact(render("n" -> name))).get
+      response.getVersion().toInt
     } catch {
-      case e: IOException =>
-        throw new StorageClientException(s"Failed to update $index/$estype/$name", e)
-    } finally {
-      restClient.close()
+      case e: ElasticsearchException =>
+        error(e.getMessage)
+        0
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
index db841b6..f5c99bf 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
@@ -15,149 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.predictionio.data.storage.elasticsearch
 
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
+package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.entity.ContentType
-import org.apache.http.entity.StringEntity
-import org.apache.http.nio.entity.NStringEntity
-import org.elasticsearch.client.RestClient
-import org.json4s._
-import org.json4s.JsonDSL._
-import org.json4s.native.JsonMethods._
+import org.elasticsearch.action.search.SearchRequestBuilder
+import org.elasticsearch.client.Client
+import org.elasticsearch.common.unit.TimeValue
+import org.json4s.Formats
 import org.json4s.native.Serialization.read
-import org.apache.http.util.EntityUtils
-import org.joda.time.DateTime
-import org.joda.time.format.DateTimeFormat
-import org.joda.time.DateTimeZone
-import org.apache.predictionio.data.storage.StorageClientConfig
-import org.apache.http.HttpHost
+
+import scala.collection.mutable.ArrayBuffer
 
 object ESUtils {
-  val scrollLife = "1m"
+  val scrollLife = new TimeValue(60000)
 
-  def getAll[T: Manifest](
-    client: RestClient,
-    index: String,
-    estype: String,
-    query: String)(
+  def getAll[T : Manifest](
+      client: Client,
+      builder: SearchRequestBuilder)(
       implicit formats: Formats): Seq[T] = {
-
-    @scala.annotation.tailrec
-    def scroll(scrollId: String, hits: Seq[JValue], results: Seq[T]): Seq[T] = {
-      if (hits.isEmpty) results
-      else {
-        val json = ("scroll" -> scrollLife) ~ ("scroll_id" -> scrollId)
-        val scrollBody = new StringEntity(compact(render(json)))
-        val response = client.performRequest(
-          "POST",
-          "/_search/scroll",
-          Map[String, String](),
-          scrollBody)
-        val responseJValue = parse(EntityUtils.toString(response.getEntity))
-        scroll((responseJValue \ "_scroll_id").extract[String],
-          (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
-          hits.map(h => (h \ "_source").extract[T]) ++ results)
-      }
+    val results = ArrayBuffer[T]()
+    var response = builder.setScroll(scrollLife).get
+    var hits = response.getHits().hits()
+    results ++= hits.map(h => read[T](h.getSourceAsString))
+    while (hits.size > 0) {
+      response = client.prepareSearchScroll(response.getScrollId).
+        setScroll(scrollLife).get
+      hits = response.getHits().hits()
+      results ++= hits.map(h => read[T](h.getSourceAsString))
     }
-
-    val response = client.performRequest(
-      "POST",
-      s"/$index/$estype/_search",
-      Map("scroll" -> scrollLife),
-      new StringEntity(query))
-    val responseJValue = parse(EntityUtils.toString(response.getEntity))
-    scroll((responseJValue \ "_scroll_id").extract[String],
-        (responseJValue \ "hits" \ "hits").extract[Seq[JValue]],
-        Nil)
-  }
-
-  def createIndex(
-    client: RestClient,
-    index: String): Unit = {
-    client.performRequest(
-      "HEAD",
-      s"/$index",
-      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
-        case 404 =>
-          client.performRequest(
-            "PUT",
-            s"/$index",
-            Map.empty[String, String].asJava)
-        case 200 =>
-        case _ =>
-          throw new IllegalStateException(s"/$index is invalid.")
-      }
-  }
-
-  def createMapping(
-    client: RestClient,
-    index: String,
-    estype: String,
-    json: String): Unit = {
-    client.performRequest(
-      "HEAD",
-      s"/$index/_mapping/$estype",
-      Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
-        case 404 =>
-          val entity = new NStringEntity(json, ContentType.APPLICATION_JSON)
-          client.performRequest(
-            "PUT",
-            s"/$index/_mapping/$estype",
-            Map.empty[String, String].asJava,
-            entity)
-        case 200 =>
-        case _ =>
-          throw new IllegalStateException(s"/$index/$estype is invalid: $json")
-      }
-  }
-
-  def createEventQuery(
-    startTime: Option[DateTime] = None,
-    untilTime: Option[DateTime] = None,
-    entityType: Option[String] = None,
-    entityId: Option[String] = None,
-    eventNames: Option[Seq[String]] = None,
-    targetEntityType: Option[Option[String]] = None,
-    targetEntityId: Option[Option[String]] = None,
-    reversed: Option[Boolean] = None): String = {
-    val mustQueries = Seq(
-      startTime.map(x => {
-        val v = DateTimeFormat
-          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
-        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
-      }),
-      untilTime.map(x => {
-        val v = DateTimeFormat
-          .forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ").print(x.withZone(DateTimeZone.UTC))
-        s"""{"range":{"eventTime":{"gte":"${v}"}}}"""
-      }),
-      entityType.map(x => s"""{"term":{"entityType":"${x}"}}"""),
-      entityId.map(x => s"""{"term":{"entityId":"${x}"}}"""),
-      targetEntityType.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityType":"${x}"}}""")),
-      targetEntityId.flatMap(xx => xx.map(x => s"""{"term":{"targetEntityId":"${x}"}}""")),
-      eventNames
-        .map { xx => xx.map(x => "\"%s\"".format(x)) }
-        .map(x => s"""{"terms":{"event":[${x.mkString(",")}]}}""")).flatten.mkString(",")
-    val sortOrder = reversed.map(x => x match {
-      case true => "desc"
-      case _ => "asc"
-    })
-    s"""{
-       |"query":{"bool":{"must":[${mustQueries}]}},
-       |"sort":[{"eventTime":{"order":"${sortOrder}"}}]
-       |}""".stripMargin
-  }
-
-  def getHttpHosts(config: StorageClientConfig): Seq[HttpHost] = {
-    val hosts = config.properties.get("HOSTS").
-      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
-    val ports = config.properties.get("PORTS").
-      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
-    val schemes = config.properties.get("SCHEMES").
-      map(_.split(",").toSeq).getOrElse(Seq("http"))
-    (hosts, ports, schemes).zipped.map((h, p, s) => new HttpHost(h, p, s))
+    results
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/c64941b6/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
index 647d180..75ac2b0 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
@@ -15,30 +15,36 @@
  * limitations under the License.
  */
 
+
 package org.apache.predictionio.data.storage.elasticsearch
 
-import org.apache.http.HttpHost
+import grizzled.slf4j.Logging
 import org.apache.predictionio.data.storage.BaseStorageClient
 import org.apache.predictionio.data.storage.StorageClientConfig
 import org.apache.predictionio.data.storage.StorageClientException
-import org.elasticsearch.client.RestClient
-
-import grizzled.slf4j.Logging
-
-case class ESClient(hosts: Seq[HttpHost]) {
-  def open(): RestClient = {
-    try {
-      RestClient.builder(hosts: _*).build()
-    } catch {
-      case e: Throwable =>
-        throw new StorageClientException(e.getMessage, e)
-    }
-  }
-}
+import org.elasticsearch.client.transport.TransportClient
+import org.elasticsearch.common.settings.ImmutableSettings
+import org.elasticsearch.common.transport.InetSocketTransportAddress
+import org.elasticsearch.transport.ConnectTransportException
 
 class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
     with Logging {
   override val prefix = "ES"
-
-  val client = ESClient(ESUtils.getHttpHosts(config))
+  val client = try {
+    val hosts = config.properties.get("HOSTS").
+      map(_.split(",").toSeq).getOrElse(Seq("localhost"))
+    val ports = config.properties.get("PORTS").
+      map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9300))
+    val settings = ImmutableSettings.settingsBuilder()
+      .put("cluster.name", config.properties.getOrElse("CLUSTERNAME", "elasticsearch"))
+    val transportClient = new TransportClient(settings)
+    (hosts zip ports) foreach { hp =>
+      transportClient.addTransportAddress(
+        new InetSocketTransportAddress(hp._1, hp._2))
+    }
+    transportClient
+  } catch {
+    case e: ConnectTransportException =>
+      throw new StorageClientException(e.getMessage, e)
+  }
 }



[03/10] incubator-predictionio git commit: add ESClient to close RestClient

Posted by do...@apache.org.
add ESClient to close RestClient


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

Branch: refs/heads/feature/es5
Commit: d4e75ab5441c2d7278c0cc55c6f1d3b51c9479a0
Parents: 36b79d7
Author: Shinsuke Sugaya <sh...@yahoo.co.jp>
Authored: Mon Jan 16 17:28:54 2017 +0900
Committer: Shinsuke Sugaya <sh...@yahoo.co.jp>
Committed: Mon Jan 16 17:28:54 2017 +0900

----------------------------------------------------------------------
 .../predictionio/workflow/CreateWorkflow.scala  | 74 +++++++--------
 .../storage/elasticsearch/ESAccessKeys.scala    | 48 +++++++---
 .../data/storage/elasticsearch/ESApps.scala     | 48 +++++++---
 .../data/storage/elasticsearch/ESChannels.scala | 42 ++++++---
 .../elasticsearch/ESEngineInstances.scala       | 75 ++++++++++-----
 .../elasticsearch/ESEngineManifests.scala       | 23 +++--
 .../elasticsearch/ESEvaluationInstances.scala   | 62 +++++++-----
 .../data/storage/elasticsearch/ESLEvents.scala  | 99 ++++++++++++--------
 .../data/storage/elasticsearch/ESPEvents.scala  | 12 +--
 .../storage/elasticsearch/ESSequences.scala     | 22 +++--
 .../data/storage/elasticsearch/ESUtils.scala    |  6 +-
 .../storage/elasticsearch/StorageClient.scala   | 24 +++--
 12 files changed, 330 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala b/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
index edfc1b6..899ace2 100644
--- a/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
+++ b/core/src/main/scala/org/apache/predictionio/workflow/CreateWorkflow.scala
@@ -223,40 +223,36 @@ object CreateWorkflow extends Logging {
         engineFactoryObj.engineParams(wfc.engineParamsKey)
       }
 
-      try {
-        val engineInstance = EngineInstance(
-          id = "",
-          status = "INIT",
-          startTime = DateTime.now,
-          endTime = DateTime.now,
-          engineId = wfc.engineId,
-          engineVersion = wfc.engineVersion,
-          engineVariant = variantId,
-          engineFactory = engineFactory,
-          batch = wfc.batch,
-          env = pioEnvVars,
-          sparkConf = workflowParams.sparkEnv,
-          dataSourceParams =
-            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.dataSourceParams),
-          preparatorParams =
-            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.preparatorParams),
-          algorithmsParams =
-            JsonExtractor.paramsToJson(wfc.jsonExtractor, engineParams.algorithmParamsList),
-          servingParams =
-            JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.servingParams))
+      val engineInstance = EngineInstance(
+        id = "",
+        status = "INIT",
+        startTime = DateTime.now,
+        endTime = DateTime.now,
+        engineId = wfc.engineId,
+        engineVersion = wfc.engineVersion,
+        engineVariant = variantId,
+        engineFactory = engineFactory,
+        batch = wfc.batch,
+        env = pioEnvVars,
+        sparkConf = workflowParams.sparkEnv,
+        dataSourceParams =
+          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.dataSourceParams),
+        preparatorParams =
+          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.preparatorParams),
+        algorithmsParams =
+          JsonExtractor.paramsToJson(wfc.jsonExtractor, engineParams.algorithmParamsList),
+        servingParams =
+          JsonExtractor.paramToJson(wfc.jsonExtractor, engineParams.servingParams))
 
-        val engineInstanceId = Storage.getMetaDataEngineInstances.insert(
-          engineInstance)
+      val engineInstanceId = Storage.getMetaDataEngineInstances.insert(
+        engineInstance)
 
-        CoreWorkflow.runTrain(
-          env = pioEnvVars,
-          params = workflowParams,
-          engine = trainableEngine,
-          engineParams = engineParams,
-          engineInstance = engineInstance.copy(id = engineInstanceId))
-      } finally {
-        Storage.getLEvents().close()
-      }
+      CoreWorkflow.runTrain(
+        env = pioEnvVars,
+        params = workflowParams,
+        engine = trainableEngine,
+        engineParams = engineParams,
+        engineInstance = engineInstance.copy(id = engineInstanceId))
     } else {
       val workflowParams = WorkflowParams(
         verbose = wfc.verbosity,
@@ -271,15 +267,11 @@ object CreateWorkflow extends Logging {
         env = pioEnvVars,
         sparkConf = workflowParams.sparkEnv
       )
-      try {
-        Workflow.runEvaluation(
-          evaluation = evaluation.get,
-          engineParamsGenerator = engineParamsGenerator.get,
-          evaluationInstance = evaluationInstance,
-          params = workflowParams)
-      } finally {
-        Storage.getLEvents().close()
-      }
+      Workflow.runEvaluation(
+        evaluation = evaluation.get,
+        engineParamsGenerator = engineParamsGenerator.get,
+        evaluationInstance = evaluationInstance,
+        params = workflowParams)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
index 2c69cf4..9156fab 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESAccessKeys.scala
@@ -37,19 +37,24 @@ import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
 /** Elasticsearch implementation of AccessKeys. */
-class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: String)
+class ESAccessKeys(client: ESClient, config: StorageClientConfig, index: String)
     extends AccessKeys with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "accesskeys"
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0)) ~
-      ("properties" ->
-        ("key" -> ("type" -> "keyword")) ~
-        ("events" -> ("type" -> "keyword"))))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("key" -> ("type" -> "keyword")) ~
+          ("events" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def insert(accessKey: AccessKey): Option[String] = {
     val key = if (accessKey.key.isEmpty) generateKey else accessKey.key
@@ -58,8 +63,9 @@ class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: Strin
   }
 
   def get(id: String): Option[AccessKey] = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -81,41 +87,50 @@ class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: Strin
       case e: IOException =>
         error("Failed to access to /$index/$estype/$key", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getAll(): Seq[AccessKey] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" -> List.empty))
-      ESUtils.getAll[AccessKey](client, index, estype, compact(render(json)))
+      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def getByAppid(appid: Int): Seq[AccessKey] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("term" ->
             ("appid" -> appid)))
-      ESUtils.getAll[AccessKey](client, index, estype, compact(render(json)))
+      ESUtils.getAll[AccessKey](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def update(accessKey: AccessKey): Unit = {
     val id = accessKey.key
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(accessKey), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -131,12 +146,15 @@ class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: Strin
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 
   def delete(id: String): Unit = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -150,6 +168,8 @@ class ESAccessKeys(client: RestClient, config: StorageClientConfig, index: Strin
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/id", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
index 7a65379..0379c90 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESApps.scala
@@ -37,20 +37,25 @@ import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
 /** Elasticsearch implementation of Items. */
-class ESApps(client: RestClient, config: StorageClientConfig, index: String)
+class ESApps(client: ESClient, config: StorageClientConfig, index: String)
     extends Apps with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "apps"
   private val seq = new ESSequences(client, config, index)
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0))~
-      ("properties" ->
-        ("id" -> ("type" -> "keyword")) ~
-        ("name" -> ("type" -> "keyword"))))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("id" -> ("type" -> "keyword")) ~
+          ("name" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def insert(app: App): Option[Int] = {
     val id =
@@ -64,8 +69,9 @@ class ESApps(client: RestClient, config: StorageClientConfig, index: String)
   }
 
   def get(id: Int): Option[App] = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -87,17 +93,20 @@ class ESApps(client: RestClient, config: StorageClientConfig, index: String)
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/$id", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getByName(name: String): Option[App] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("term" ->
             ("name" -> name)))
       val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/_search",
         Map.empty[String, String].asJava,
@@ -114,27 +123,33 @@ class ESApps(client: RestClient, config: StorageClientConfig, index: String)
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/_search", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getAll(): Seq[App] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" -> List.empty))
-      ESUtils.getAll[App](client, index, estype, compact(render(json)))
+      ESUtils.getAll[App](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def update(app: App): Unit = {
     val id = app.id.toString
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(app), ContentType.APPLICATION_JSON);
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -150,12 +165,15 @@ class ESApps(client: RestClient, config: StorageClientConfig, index: String)
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 
   def delete(id: Int): Unit = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -169,6 +187,8 @@ class ESApps(client: RestClient, config: StorageClientConfig, index: String)
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/id", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
index c90d668..b319c26 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESChannels.scala
@@ -36,19 +36,24 @@ import org.json4s.native.Serialization.write
 import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
-class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
+class ESChannels(client: ESClient, config: StorageClientConfig, index: String)
     extends Channels with Logging {
   implicit val formats = DefaultFormats.lossless
   private val estype = "channels"
   private val seq = new ESSequences(client, config, index)
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0))~
-      ("properties" ->
-        ("name" -> ("type" -> "keyword"))))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("name" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def insert(channel: Channel): Option[Int] = {
     val id =
@@ -62,8 +67,9 @@ class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
   }
 
   def get(id: Int): Option[Channel] = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -85,28 +91,34 @@ class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/$id", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getByAppid(appid: Int): Seq[Channel] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("term" ->
             ("appid" -> appid)))
-      ESUtils.getAll[Channel](client, index, estype, compact(render(json)))
+      ESUtils.getAll[Channel](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def update(channel: Channel): Boolean = {
     val id = channel.id.toString
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(channel), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -124,12 +136,15 @@ class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
         false
+    } finally {
+      restClient.close()
     }
   }
 
   def delete(id: Int): Unit = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -143,7 +158,8 @@ class ESChannels(client: RestClient, config: StorageClientConfig, index: String)
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
index 08f87f3..68cdeac 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineInstances.scala
@@ -37,30 +37,35 @@ import org.json4s.native.Serialization.write
 import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
-class ESEngineInstances(client: RestClient, config: StorageClientConfig, index: String)
+class ESEngineInstances(client: ESClient, config: StorageClientConfig, index: String)
     extends EngineInstances with Logging {
   implicit val formats = DefaultFormats + new EngineInstanceSerializer
   private val estype = "engine_instances"
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0))~
-      ("properties" ->
-        ("status" -> ("type" -> "keyword")) ~
-        ("startTime" -> ("type" -> "date")) ~
-        ("endTime" -> ("type" -> "date")) ~
-        ("engineId" -> ("type" -> "keyword")) ~
-        ("engineVersion" -> ("type" -> "keyword")) ~
-        ("engineVariant" -> ("type" -> "keyword")) ~
-        ("engineFactory" -> ("type" -> "keyword")) ~
-        ("batch" -> ("type" -> "keyword")) ~
-        ("dataSourceParams" -> ("type" -> "keyword")) ~
-        ("preparatorParams" -> ("type" -> "keyword")) ~
-        ("algorithmsParams" -> ("type" -> "keyword")) ~
-        ("servingParams" -> ("type" -> "keyword")) ~
-        ("status" -> ("type" -> "keyword"))))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("status" -> ("type" -> "keyword")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("engineId" -> ("type" -> "keyword")) ~
+          ("engineVersion" -> ("type" -> "keyword")) ~
+          ("engineVariant" -> ("type" -> "keyword")) ~
+          ("engineFactory" -> ("type" -> "keyword")) ~
+          ("batch" -> ("type" -> "keyword")) ~
+          ("dataSourceParams" -> ("type" -> "keyword")) ~
+          ("preparatorParams" -> ("type" -> "keyword")) ~
+          ("algorithmsParams" -> ("type" -> "keyword")) ~
+          ("servingParams" -> ("type" -> "keyword")) ~
+          ("status" -> ("type" -> "keyword"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def insert(i: EngineInstance): String = {
     val id = i.id match {
@@ -81,9 +86,10 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
   }
 
   def preInsert(): Option[String] = {
+    val restClient = client.open()
     try {
       val entity = new NStringEntity("{}", ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/",
         Map.empty[String, String].asJava,
@@ -101,12 +107,15 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
       case e: IOException =>
         error(s"Failed to create $index/$estype", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def get(id: String): Option[EngineInstance] = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -128,19 +137,24 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/$id", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getAll(): Seq[EngineInstance] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" -> List.empty))
-      ESUtils.getAll[EngineInstance](client, index, estype, compact(render(json)))
+      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
@@ -148,6 +162,7 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
     engineId: String,
     engineVersion: String,
     engineVariant: String): Seq[EngineInstance] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
@@ -164,11 +179,13 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
               ("sort" -> List(
                 ("startTime" ->
                   ("order" -> "desc"))))
-      ESUtils.getAll[EngineInstance](client, index, estype, compact(render(json)))
+      ESUtils.getAll[EngineInstance](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
@@ -183,9 +200,10 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
 
   def update(i: EngineInstance): Unit = {
     val id = i.id
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -201,12 +219,15 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 
   def delete(id: String): Unit = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -220,6 +241,8 @@ class ESEngineInstances(client: RestClient, config: StorageClientConfig, index:
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
index a965c71..ae4d86b 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEngineManifests.scala
@@ -37,7 +37,7 @@ import org.json4s.native.Serialization.write
 import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
-class ESEngineManifests(client: RestClient, config: StorageClientConfig, index: String)
+class ESEngineManifests(client: ESClient, config: StorageClientConfig, index: String)
     extends EngineManifests with Logging {
   implicit val formats = DefaultFormats + new EngineManifestSerializer
   private val estype = "engine_manifests"
@@ -45,9 +45,10 @@ class ESEngineManifests(client: RestClient, config: StorageClientConfig, index:
 
   def insert(engineManifest: EngineManifest): Unit = {
     val id = esid(engineManifest.id, engineManifest.version)
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(engineManifest), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -63,13 +64,16 @@ class ESEngineManifests(client: RestClient, config: StorageClientConfig, index:
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 
   def get(id: String, version: String): Option[EngineManifest] = {
     val esId = esid(id, version)
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$esId",
         Map.empty[String, String].asJava)
@@ -91,20 +95,24 @@ class ESEngineManifests(client: RestClient, config: StorageClientConfig, index:
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/$esId", e)
         None
+    } finally {
+      restClient.close()
     }
-
   }
 
   def getAll(): Seq[EngineManifest] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" ->  List.empty))
-      ESUtils.getAll[EngineManifest](client, index, estype, compact(render(json)))
+      ESUtils.getAll[EngineManifest](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
@@ -113,8 +121,9 @@ class ESEngineManifests(client: RestClient, config: StorageClientConfig, index:
 
   def delete(id: String, version: String): Unit = {
     val esId = esid(id, version)
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$esId",
         Map.empty[String, String].asJava)
@@ -128,6 +137,8 @@ class ESEngineManifests(client: RestClient, config: StorageClientConfig, index:
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$esId", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
index 0e71f79..1f798f0 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESEvaluationInstances.scala
@@ -38,27 +38,32 @@ import org.json4s.native.Serialization.write
 import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
-class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, index: String)
+class ESEvaluationInstances(client: ESClient, config: StorageClientConfig, index: String)
     extends EvaluationInstances with Logging {
   implicit val formats = DefaultFormats + new EvaluationInstanceSerializer
   private val estype = "evaluation_instances"
   private val seq = new ESSequences(client, config, index)
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0))~
-      ("properties" ->
-        ("status" -> ("type" -> "keyword")) ~
-        ("startTime" -> ("type" -> "date")) ~
-        ("endTime" -> ("type" -> "date")) ~
-        ("evaluationClass" -> ("type" -> "keyword")) ~
-        ("engineParamsGeneratorClass" -> ("type" -> "keyword")) ~
-        ("batch" -> ("type" -> "keyword")) ~
-        ("evaluatorResults" -> ("type" -> "text") ~ ("index" -> "no")) ~
-        ("evaluatorResultsHTML" -> ("type" -> "text") ~ ("index" -> "no")) ~
-        ("evaluatorResultsJSON" -> ("type" -> "text") ~ ("index" -> "no"))))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)) ~
+        ("properties" ->
+          ("status" -> ("type" -> "keyword")) ~
+          ("startTime" -> ("type" -> "date")) ~
+          ("endTime" -> ("type" -> "date")) ~
+          ("evaluationClass" -> ("type" -> "keyword")) ~
+          ("engineParamsGeneratorClass" -> ("type" -> "keyword")) ~
+          ("batch" -> ("type" -> "keyword")) ~
+          ("evaluatorResults" -> ("type" -> "text") ~ ("index" -> "no")) ~
+          ("evaluatorResultsHTML" -> ("type" -> "text") ~ ("index" -> "no")) ~
+          ("evaluatorResultsJSON" -> ("type" -> "text") ~ ("index" -> "no"))))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def insert(i: EvaluationInstance): String = {
     val id = i.id match {
@@ -74,8 +79,9 @@ class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, ind
   }
 
   def get(id: String): Option[EvaluationInstance] = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -97,23 +103,29 @@ class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, ind
       case e: IOException =>
         error(s"Failed to access to /$index/$estype/$id", e)
         None
+    } finally {
+      restClient.close()
     }
   }
 
   def getAll(): Seq[EvaluationInstance] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" -> List.empty))
-      ESUtils.getAll[EvaluationInstance](client, index, estype, compact(render(json)))
+      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def getCompleted(): Seq[EvaluationInstance] = {
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
@@ -122,19 +134,22 @@ class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, ind
             ("sort" ->
               ("startTime" ->
                 ("order" -> "desc")))
-      ESUtils.getAll[EvaluationInstance](client, index, estype, compact(render(json)))
+      ESUtils.getAll[EvaluationInstance](restClient, index, estype, compact(render(json)))
     } catch {
       case e: IOException =>
         error("Failed to access to /$index/$estype/_search", e)
         Nil
+    } finally {
+      restClient.close()
     }
   }
 
   def update(i: EvaluationInstance): Unit = {
     val id = i.id
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write(i), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava,
@@ -150,12 +165,15 @@ class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, ind
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 
   def delete(id: String): Unit = {
+    val restClient = client.open()
     try {
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "DELETE",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava)
@@ -169,6 +187,8 @@ class ESEvaluationInstances(client: RestClient, config: StorageClientConfig, ind
     } catch {
       case e: IOException =>
         error(s"Failed to update $index/$estype/$id", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
index ef25204..b4f7dc5 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESLEvents.scala
@@ -40,7 +40,7 @@ import org.json4s.ext.JodaTimeSerializers
 import grizzled.slf4j.Logging
 import org.elasticsearch.client.ResponseException
 
-class ESLEvents(val client: RestClient, config: StorageClientConfig, val index: String)
+class ESLEvents(val client: ESClient, config: StorageClientConfig, val index: String)
     extends LEvents with Logging {
   implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
   private val seq = new ESSequences(client, config, index)
@@ -56,41 +56,47 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
 
   override def init(appId: Int, channelId: Option[Int] = None): Boolean = {
     val estype = getEsType(appId, channelId)
-    ESUtils.createIndex(client, index)
-    val json =
-      (estype ->
-        ("_all" -> ("enabled" -> 0)) ~
-        ("properties" ->
-          ("name" -> ("type" -> "keyword")) ~
-          ("eventId" -> ("type" -> "keyword")) ~
-          ("event" -> ("type" -> "keyword")) ~
-          ("entityType" -> ("type" -> "keyword")) ~
-          ("entityId" -> ("type" -> "keyword")) ~
-          ("targetEntityType" -> ("type" -> "keyword")) ~
-          ("targetEntityId" -> ("type" -> "keyword")) ~
+    val restClient = client.open()
+    try {
+      ESUtils.createIndex(restClient, index)
+      val json =
+        (estype ->
+          ("_all" -> ("enabled" -> 0)) ~
           ("properties" ->
-            ("type" -> "nested") ~
+            ("name" -> ("type" -> "keyword")) ~
+            ("eventId" -> ("type" -> "keyword")) ~
+            ("event" -> ("type" -> "keyword")) ~
+            ("entityType" -> ("type" -> "keyword")) ~
+            ("entityId" -> ("type" -> "keyword")) ~
+            ("targetEntityType" -> ("type" -> "keyword")) ~
+            ("targetEntityId" -> ("type" -> "keyword")) ~
             ("properties" ->
-              ("fields" -> ("type" -> "nested") ~
-                ("properties" ->
-                  ("user" -> ("type" -> "long")) ~
-                  ("num" -> ("type" -> "long")))))) ~
-                  ("eventTime" -> ("type" -> "date")) ~
-                  ("tags" -> ("type" -> "keyword")) ~
-                  ("prId" -> ("type" -> "keyword")) ~
-                  ("creationTime" -> ("type" -> "date"))))
-    ESUtils.createMapping(client, index, estype, compact(render(json)))
+              ("type" -> "nested") ~
+              ("properties" ->
+                ("fields" -> ("type" -> "nested") ~
+                  ("properties" ->
+                    ("user" -> ("type" -> "long")) ~
+                    ("num" -> ("type" -> "long")))))) ~
+                    ("eventTime" -> ("type" -> "date")) ~
+                    ("tags" -> ("type" -> "keyword")) ~
+                    ("prId" -> ("type" -> "keyword")) ~
+                    ("creationTime" -> ("type" -> "date"))))
+      ESUtils.createMapping(restClient, index, estype, compact(render(json)))
+    } finally {
+      restClient.close()
+    }
     true
   }
 
   override def remove(appId: Int, channelId: Option[Int] = None): Boolean = {
     val estype = getEsType(appId, channelId)
+    val restClient = client.open()
     try {
       val json =
         ("query" ->
           ("match_all" -> List.empty))
       val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-      client.performRequest(
+      restClient.performRequest(
         "POST",
         s"/$index/$estype/_delete_by_query",
         Map.empty[String, String].asJava,
@@ -104,14 +110,13 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
       case e: Exception =>
         error(s"Failed to remove $index/$estype", e)
         false
+    } finally {
+      restClient.close()
     }
   }
 
   override def close(): Unit = {
-    try client.close() catch {
-      case e: Exception =>
-        error("Failed to close client.", e)
-    }
+    // nothing
   }
 
   override def futureInsert(
@@ -120,15 +125,16 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
     channelId: Option[Int])(implicit ec: ExecutionContext): Future[String] = {
     Future {
       val estype = getEsType(appId, channelId)
-      val id = event.eventId.getOrElse {
-        var roll = seq.genNext(seqName)
-        while (exists(estype, roll)) roll = seq.genNext(seqName)
-        roll.toString
-      }
-      val json = write(event.copy(eventId = Some(id)))
+      val restClient = client.open()
       try {
+        val id = event.eventId.getOrElse {
+          var roll = seq.genNext(seqName)
+          while (exists(restClient, estype, roll)) roll = seq.genNext(seqName)
+          roll.toString
+        }
+        val json = write(event.copy(eventId = Some(id)))
         val entity = new NStringEntity(json, ContentType.APPLICATION_JSON);
-        val response = client.performRequest(
+        val response = restClient.performRequest(
           "POST",
           s"/$index/$estype/$id",
           Map.empty[String, String].asJava,
@@ -144,15 +150,17 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
         }
       } catch {
         case e: IOException =>
-          error(s"Failed to update $index/$estype/$id: $json", e)
+          error(s"Failed to update $index/$estype/<id>", e)
           ""
+      } finally {
+        restClient.close()
       }
     }
   }
 
-  private def exists(estype: String, id: Int): Boolean = {
+  private def exists(restClient: RestClient, estype: String, id: Int): Boolean = {
     try {
-      client.performRequest(
+      restClient.performRequest(
         "GET",
         s"/$index/$estype/$id",
         Map.empty[String, String].asJava).getStatusLine.getStatusCode match {
@@ -179,13 +187,14 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
     channelId: Option[Int])(implicit ec: ExecutionContext): Future[Option[Event]] = {
     Future {
       val estype = getEsType(appId, channelId)
+      val restClient = client.open()
       try {
         val json =
           ("query" ->
             ("term" ->
               ("eventId" -> eventId)))
         val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-        val response = client.performRequest(
+        val response = restClient.performRequest(
           "POST",
           s"/$index/$estype/_search",
           Map.empty[String, String].asJava,
@@ -202,6 +211,8 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
         case e: IOException =>
           error("Failed to access to /$index/$estype/_search", e)
           None
+      } finally {
+        restClient.close()
       }
     }
   }
@@ -212,13 +223,14 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
     channelId: Option[Int])(implicit ec: ExecutionContext): Future[Boolean] = {
     Future {
       val estype = getEsType(appId, channelId)
+      val restClient = client.open()
       try {
         val json =
           ("query" ->
             ("term" ->
               ("eventId" -> eventId)))
         val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-        val response = client.performRequest(
+        val response = restClient.performRequest(
           "POST",
           s"/$index/$estype/_delete_by_query",
           Map.empty[String, String].asJava)
@@ -234,6 +246,8 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
         case e: IOException =>
           error(s"Failed to update $index/$estype:$eventId", e)
           false
+      } finally {
+        restClient.close()
       }
     }
   }
@@ -253,15 +267,18 @@ class ESLEvents(val client: RestClient, config: StorageClientConfig, val index:
     (implicit ec: ExecutionContext): Future[Iterator[Event]] = {
     Future {
       val estype = getEsType(appId, channelId)
+      val restClient = client.open()
       try {
         val query = ESUtils.createEventQuery(
           startTime, untilTime, entityType, entityId,
           eventNames, targetEntityType, targetEntityId, None)
-        ESUtils.getAll[Event](client, index, estype, query).toIterator
+        ESUtils.getAll[Event](restClient, index, estype, query).toIterator
       } catch {
         case e: IOException =>
           error(e.getMessage)
           Iterator[Event]()
+      } finally {
+        restClient.close()
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
index 0e3eec8..5784b3f 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESPEvents.scala
@@ -41,16 +41,10 @@ import org.json4s.native.JsonMethods._
 import org.json4s.ext.JodaTimeSerializers
 
 
-class ESPEvents(client: RestClient, config: StorageClientConfig, index: String)
+class ESPEvents(client: ESClient, config: StorageClientConfig, index: String)
     extends PEvents {
   implicit val formats = DefaultFormats.lossless ++ JodaTimeSerializers.all
 
-  // client is not used.
-  try client.close() catch {
-    case e: Exception =>
-      logger.error("Failed to close client.", e)
-  }
-
   def getEsType(appId: Int, channelId: Option[Int] = None): String = {
     channelId.map { ch =>
       s"${appId}_${ch}"
@@ -114,7 +108,7 @@ class ESPEvents(client: RestClient, config: StorageClientConfig, index: String)
     eventIds: RDD[String],
     appId: Int, channelId: Option[Int])(sc: SparkContext): Unit = {
     val estype = getEsType(appId, channelId)
-    val restClient = ESUtils.createRestClient(config)
+    val restClient = client.open()
     try {
       eventIds.foreachPartition { iter =>
         iter.foreach { eventId =>
@@ -124,7 +118,7 @@ class ESPEvents(client: RestClient, config: StorageClientConfig, index: String)
                 ("term" ->
                   ("eventId" -> eventId)))
             val entity = new NStringEntity(compact(render(json)), ContentType.APPLICATION_JSON)
-            val response = client.performRequest(
+            val response = restClient.performRequest(
               "POST",
               s"/$index/$estype/_delete_by_query",
               Map.empty[String, String].asJava)

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
index c067f3a..4eb8cd7 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESSequences.scala
@@ -35,20 +35,26 @@ import org.json4s.native.Serialization.write
 
 import grizzled.slf4j.Logging
 
-class ESSequences(client: RestClient, config: StorageClientConfig, index: String) extends Logging {
+class ESSequences(client: ESClient, config: StorageClientConfig, index: String) extends Logging {
   implicit val formats = DefaultFormats
   private val estype = "sequences"
 
-  ESUtils.createIndex(client, index)
-  val mappingJson =
-    (estype ->
-      ("_all" -> ("enabled" -> 0)))
-  ESUtils.createMapping(client, index, estype, compact(render(mappingJson)))
+  val restClient = client.open()
+  try {
+    ESUtils.createIndex(restClient, index)
+    val mappingJson =
+      (estype ->
+        ("_all" -> ("enabled" -> 0)))
+    ESUtils.createMapping(restClient, index, estype, compact(render(mappingJson)))
+  } finally {
+    restClient.close()
+  }
 
   def genNext(name: String): Int = {
+    val restClient = client.open()
     try {
       val entity = new NStringEntity(write("n" -> name), ContentType.APPLICATION_JSON)
-      val response = client.performRequest(
+      val response = restClient.performRequest(
         "POST",
         s"/$index/$estype/$name",
         Map.empty[String, String].asJava,
@@ -66,6 +72,8 @@ class ESSequences(client: RestClient, config: StorageClientConfig, index: String
     } catch {
       case e: IOException =>
         throw new StorageClientException(s"Failed to update $index/$estype/$name", e)
+    } finally {
+      restClient.close()
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
index 68e3f57..db841b6 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/ESUtils.scala
@@ -151,15 +151,13 @@ object ESUtils {
        |}""".stripMargin
   }
 
-  def createRestClient(config: StorageClientConfig): RestClient = {
+  def getHttpHosts(config: StorageClientConfig): Seq[HttpHost] = {
     val hosts = config.properties.get("HOSTS").
       map(_.split(",").toSeq).getOrElse(Seq("localhost"))
     val ports = config.properties.get("PORTS").
       map(_.split(",").toSeq.map(_.toInt)).getOrElse(Seq(9200))
     val schemes = config.properties.get("SCHEMES").
       map(_.split(",").toSeq).getOrElse(Seq("http"))
-    val httpHosts = (hosts, ports, schemes).zipped.map(
-      (h, p, s) => new HttpHost(h, p, s))
-    RestClient.builder(httpHosts: _*).build()
+    (hosts, ports, schemes).zipped.map((h, p, s) => new HttpHost(h, p, s))
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/d4e75ab5/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
----------------------------------------------------------------------
diff --git a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
index 912d467..647d180 100644
--- a/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
+++ b/data/src/main/scala/org/apache/predictionio/data/storage/elasticsearch/StorageClient.scala
@@ -17,22 +17,28 @@
 
 package org.apache.predictionio.data.storage.elasticsearch
 
-import grizzled.slf4j.Logging
+import org.apache.http.HttpHost
 import org.apache.predictionio.data.storage.BaseStorageClient
 import org.apache.predictionio.data.storage.StorageClientConfig
 import org.apache.predictionio.data.storage.StorageClientException
-import java.net.InetAddress
 import org.elasticsearch.client.RestClient
-import org.apache.http.HttpHost
+
+import grizzled.slf4j.Logging
+
+case class ESClient(hosts: Seq[HttpHost]) {
+  def open(): RestClient = {
+    try {
+      RestClient.builder(hosts: _*).build()
+    } catch {
+      case e: Throwable =>
+        throw new StorageClientException(e.getMessage, e)
+    }
+  }
+}
 
 class StorageClient(val config: StorageClientConfig) extends BaseStorageClient
     with Logging {
   override val prefix = "ES"
 
-  val client = try {
-    ESUtils.createRestClient(config)
-  } catch {
-    case e: Throwable =>
-      throw new StorageClientException(e.getMessage, e)
-  }
+  val client = ESClient(ESUtils.getHttpHosts(config))
 }


[04/10] incubator-predictionio git commit: update docker files

Posted by do...@apache.org.
update docker files


Project: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/commit/48e18b56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/tree/48e18b56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-predictionio/diff/48e18b56

Branch: refs/heads/feature/es5
Commit: 48e18b566f58a0b25e79129c7a2b75b9d775ab3e
Parents: d4e75ab
Author: Shinsuke Sugaya <sh...@yahoo.co.jp>
Authored: Mon Jan 16 19:27:58 2017 +0900
Committer: Shinsuke Sugaya <sh...@yahoo.co.jp>
Committed: Mon Jan 16 19:27:58 2017 +0900

----------------------------------------------------------------------
 tests/Dockerfile                       | 4 ++--
 tests/docker-files/env-conf/pio-env.sh | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/48e18b56/tests/Dockerfile
----------------------------------------------------------------------
diff --git a/tests/Dockerfile b/tests/Dockerfile
index a455d5d..1f87554 100644
--- a/tests/Dockerfile
+++ b/tests/Dockerfile
@@ -18,7 +18,7 @@
 from ubuntu
 
 ENV SPARK_VERSION 1.4.0
-ENV ELASTICSEARCH_VERSION 1.4.4
+ENV ELASTICSEARCH_VERSION 5.1.2
 ENV HBASE_VERSION 1.0.0
 
 RUN echo "== Updating system =="
@@ -48,7 +48,7 @@ RUN rm spark-${SPARK_VERSION}-bin-hadoop2.6.tgz
 ENV SPARK_HOME /vendors/spark-${SPARK_VERSION}-bin-hadoop2.6
 
 RUN echo "== Installing Elasticsearch =="
-RUN wget https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
+RUN wget https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
 RUN tar zxvfC elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz /vendors
 RUN rm elasticsearch-${ELASTICSEARCH_VERSION}.tar.gz
 ENV ELASTICSEARCH_HOME /vendors/elasticsearch-${ELASTICSEARCH_VERSION}

http://git-wip-us.apache.org/repos/asf/incubator-predictionio/blob/48e18b56/tests/docker-files/env-conf/pio-env.sh
----------------------------------------------------------------------
diff --git a/tests/docker-files/env-conf/pio-env.sh b/tests/docker-files/env-conf/pio-env.sh
index 4ffec46..7ea2164 100644
--- a/tests/docker-files/env-conf/pio-env.sh
+++ b/tests/docker-files/env-conf/pio-env.sh
@@ -85,9 +85,9 @@ PIO_STORAGE_SOURCES_PGSQL_PASSWORD=pio
 
 # Elasticsearch Example
 PIO_STORAGE_SOURCES_ELASTICSEARCH_TYPE=elasticsearch
-#PIO_STORAGE_SOURCES_ELASTICSEARCH_CLUSTERNAME=pio
 PIO_STORAGE_SOURCES_ELASTICSEARCH_HOSTS=localhost
-PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9300
+PIO_STORAGE_SOURCES_ELASTICSEARCH_PORTS=9200
+PIO_STORAGE_SOURCES_ELASTICSEARCH_SCHEMES=http
 PIO_STORAGE_SOURCES_ELASTICSEARCH_HOME=$ELASTICSEARCH_HOME
 
 # Local File System Example