You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ja...@apache.org on 2016/04/20 19:33:35 UTC

[01/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Repository: incubator-geode
Updated Branches:
  refs/heads/develop f12ece597 -> ddee87fea


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/LocatorHelperTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/LocatorHelperTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/LocatorHelperTest.scala
deleted file mode 100644
index c775784..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/LocatorHelperTest.scala
+++ /dev/null
@@ -1,168 +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 unittest.io.pivotal.geode.spark.connector
-
-import java.net.InetAddress
-
-import io.pivotal.geode.spark.connector.internal.LocatorHelper
-import org.scalatest.FunSuite
-
-class LocatorHelperTest extends FunSuite {
-
-  test("locatorStr2HostPortPair hostname w/o domain") {
-    val (host, port) = ("localhost", 10334)
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:$port").get ==(host, port))
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host[$port]").get ==(host, port))
-  }
-
-  test("locatorStr2HostPortPair hostname w/ domain") {
-    val (host, port) = ("localhost", 10334)
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:$port").get ==(host, port))
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host[$port]").get ==(host, port))
-  }
-
-  test("locatorStr2HostPortPair w/ invalid host name") {
-    // empty or null locatorStr
-    assert(LocatorHelper.locatorStr2HostPortPair("").isFailure)
-    assert(LocatorHelper.locatorStr2HostPortPair(null).isFailure)
-    // host name has leading `.`
-    assert(LocatorHelper.locatorStr2HostPortPair(".localhost.1234").isFailure)
-    // host name has leading and/or tail white space
-    assert(LocatorHelper.locatorStr2HostPortPair(" localhost.1234").isFailure)
-    assert(LocatorHelper.locatorStr2HostPortPair("localhost .1234").isFailure)
-    assert(LocatorHelper.locatorStr2HostPortPair(" localhost .1234").isFailure)
-    // host name contain invalid characters
-    assert(LocatorHelper.locatorStr2HostPortPair("local%host.1234").isFailure)
-    assert(LocatorHelper.locatorStr2HostPortPair("localhost*.1234").isFailure)
-    assert(LocatorHelper.locatorStr2HostPortPair("^localhost.1234").isFailure)
-  }
-
-  test("locatorStr2HostPortPair w/ valid port") {
-    val host = "192.168.0.1"
-    // port has 2, 3, 4, 5 digits
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:20").get ==(host, 20))
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:300").get ==(host, 300))
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:4000").get ==(host, 4000))
-    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:50000").get ==(host, 50000))
-  }
-  
-  test("locatorStr2HostPortPair w/ invalid port") {
-    // port number is less than 2 digits
-    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.9").isFailure)
-    // port number is more than 5 digits
-    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.100000").isFailure)
-    // port number is invalid
-    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.1xx1").isFailure)
-  }
-  
-  test("parseLocatorsString with valid locator(s)") {
-    val (host1, port1) = ("localhost", 10334)
-    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1") == Seq((host1, port1)))
-    val (host2, port2) = ("localhost2", 10335)
-    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1,$host2:$port2") == Seq((host1, port1),(host2, port2)))
-    val (host3, port3) = ("localhost2", 10336)
-    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1,$host2:$port2,$host3:$port3") == 
-      Seq((host1, port1),(host2, port2),(host3, port3)))
-  }
-
-  test("parseLocatorsString with invalid locator(s)") {
-    // empty and null locatorsStr
-    intercept[Exception] { LocatorHelper.parseLocatorsString("") }
-    intercept[Exception] { LocatorHelper.parseLocatorsString(null) }
-    // 1 bad locatorStr
-    intercept[Exception] { LocatorHelper.parseLocatorsString("local%host.1234") }
-    // 1 good locatorStr and 1 bad locatorStr
-    intercept[Exception] { LocatorHelper.parseLocatorsString("localhost:2345,local%host.1234") }
-    intercept[Exception] { LocatorHelper.parseLocatorsString("local^host:2345,localhost.1234") }
-  }
-
-  test("pickPreferredGeodeServers: shared servers and one gf-server per host") {
-    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
-    val servers = Seq(srv1, srv2, srv3, srv4)
-    verifyPickPreferredGeodeServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
-    verifyPickPreferredGeodeServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
-    verifyPickPreferredGeodeServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
-    verifyPickPreferredGeodeServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
-  }
-
-  test("pickPreferredGeodeServers: shared servers, one gf-server per host, un-sorted list") {
-    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
-    val servers = Seq(srv4, srv2, srv3, srv1)
-    verifyPickPreferredGeodeServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
-    verifyPickPreferredGeodeServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
-    verifyPickPreferredGeodeServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
-    verifyPickPreferredGeodeServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
-  }
-
-  test("pickPreferredGeodeServers: shared servers and two gf-server per host") {
-    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
-    val servers = Seq(srv1, srv2, srv3, srv4)
-    verifyPickPreferredGeodeServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
-    verifyPickPreferredGeodeServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
-    verifyPickPreferredGeodeServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
-    verifyPickPreferredGeodeServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
-  }
-
-  test("pickPreferredGeodeServers: shared servers, two gf-server per host, un-sorted server list") {
-    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
-    val servers = Seq(srv1, srv4, srv3, srv2)
-    verifyPickPreferredGeodeServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
-    verifyPickPreferredGeodeServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
-    verifyPickPreferredGeodeServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
-    verifyPickPreferredGeodeServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
-  }
-
-  test("pickPreferredGeodeServers: no shared servers and one gf-server per host") {
-    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
-    val servers = Seq(srv1, srv2, srv3, srv4)
-    verifyPickPreferredGeodeServers(servers, "host5", "<driver>", Seq(srv1, srv2, srv3))
-    verifyPickPreferredGeodeServers(servers, "host6", "0", Seq(srv2, srv3, srv4))
-    verifyPickPreferredGeodeServers(servers, "host7", "1", Seq(srv3, srv4, srv1))
-    verifyPickPreferredGeodeServers(servers, "host8", "2", Seq(srv4, srv1, srv2))
-  }
-
-  test("pickPreferredGeodeServers: no shared servers, one gf-server per host, and less gf-server") {
-    val (srv1, srv2) = (("host1", 4001), ("host2", 4002))
-    val servers = Seq(srv1, srv2)
-    verifyPickPreferredGeodeServers(servers, "host5", "<driver>", Seq(srv1, srv2))
-    verifyPickPreferredGeodeServers(servers, "host6", "0", Seq(srv2, srv1))
-    verifyPickPreferredGeodeServers(servers, "host7", "1", Seq(srv1, srv2))
-    verifyPickPreferredGeodeServers(servers, "host8", "2", Seq(srv2, srv1))
-
-
-    println("host name: " + InetAddress.getLocalHost.getHostName)
-    println("canonical host name: " + InetAddress.getLocalHost.getCanonicalHostName)
-    println("canonical host name 2: " + InetAddress.getByName(InetAddress.getLocalHost.getHostName).getCanonicalHostName)
-  }
-
-  test("pickPreferredGeodeServers: ad-hoc") {
-    val (srv4, srv5, srv6) = (
-      ("w2-gst-pnq-04.gemstone.com", 40411), ("w2-gst-pnq-05.gemstone.com", 40411), ("w2-gst-pnq-06.gemstone.com", 40411))
-    val servers = Seq(srv6, srv5, srv4)
-    verifyPickPreferredGeodeServers(servers, "w2-gst-pnq-03.gemstone.com", "<driver>", Seq(srv4, srv5, srv6))
-    verifyPickPreferredGeodeServers(servers, "w2-gst-pnq-04.gemstone.com", "1", Seq(srv4, srv5, srv6))
-    verifyPickPreferredGeodeServers(servers, "w2-gst-pnq-05.gemstone.com", "0", Seq(srv5, srv6, srv4))
-    verifyPickPreferredGeodeServers(servers, "w2-gst-pnq-06.gemstone.com", "2", Seq(srv6, srv4, srv5))
-  }
-  
-  def verifyPickPreferredGeodeServers(
-    servers: Seq[(String, Int)], hostName: String, executorId: String, expectation: Seq[(String, Int)]): Unit = {
-    val result = LocatorHelper.pickPreferredGeodeServers(servers, hostName, executorId)
-    assert(result == expectation, s"pick servers for $hostName:$executorId")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRDDPartitionerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRDDPartitionerTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRDDPartitionerTest.scala
deleted file mode 100644
index f53c178..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRDDPartitionerTest.scala
+++ /dev/null
@@ -1,190 +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 unittest.io.pivotal.geode.spark.connector.rdd
-
-import com.gemstone.gemfire.distributed.internal.ServerLocation
-import io.pivotal.geode.spark.connector.internal.RegionMetadata
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRDDPartitioner._
-import io.pivotal.geode.spark.connector.GeodeConnection
-import io.pivotal.geode.spark.connector.internal.rdd._
-import org.apache.spark.Partition
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{Matchers, FunSuite}
-
-import java.util.{HashSet => JHashSet, HashMap => JHashMap}
-
-import scala.collection.mutable
-
-class GeodeRDDPartitionerTest extends FunSuite with Matchers with MockitoSugar {
-
-  val emptyServerBucketMap: JHashMap[ServerLocation, JHashSet[Integer]] = new JHashMap()
-
-  def toJavaServerBucketMap(map: Map[(String, Int), Set[Int]]): JHashMap[ServerLocation, JHashSet[Integer]] = {
-    import scala.collection.JavaConversions._
-    val tmp = map.map {case ((host, port), set) => (new ServerLocation(host, port), set.map(Integer.valueOf))}
-    (new JHashMap[ServerLocation, JHashSet[Integer]]() /: tmp) { case (acc, (s, jset)) => acc.put(s, new JHashSet(jset)); acc }
-  }
-  
-  val map: mutable.Map[(String, Int), mutable.Set[Int]] = mutable.Map(
-    ("s0",1) -> mutable.Set.empty, ("s1",2) -> mutable.Set(0), ("s2",3) -> mutable.Set(1, 2), ("s3",4) -> mutable.Set(3, 4, 5))
-
-  
-  // update this test whenever change default setting 
-  test("default partitioned region partitioner") {
-    assert(GeodeRDDPartitioner.defaultPartitionedRegionPartitioner === ServerSplitsPartitioner)
-  }
-
-  // update this test whenever change default setting 
-  test("default replicated region partitioner") {
-    assert(GeodeRDDPartitioner.defaultReplicatedRegionPartitioner === OnePartitionPartitioner)
-  }
-  
-  test("GeodeRDDPartitioner.apply method") {
-    import io.pivotal.geode.spark.connector.internal.rdd.GeodeRDDPartitioner._
-    for ((name, partitioner) <- partitioners) assert(GeodeRDDPartitioner(name) == partitioner)
-    assert(GeodeRDDPartitioner("dummy") == GeodeRDDPartitioner.defaultPartitionedRegionPartitioner)
-    assert(GeodeRDDPartitioner() == GeodeRDDPartitioner.defaultPartitionedRegionPartitioner)
-  }
-  
-  test("OnePartitionPartitioner") {
-    val mockConnection = mock[GeodeConnection]
-    val partitions = OnePartitionPartitioner.partitions[String, String](mockConnection, null, Map.empty)
-    verifySinglePartition(partitions)
-  }
-
-  def verifySinglePartition(partitions: Array[Partition]): Unit = {
-    assert(1 == partitions.size)
-    assert(partitions(0).index === 0)
-    assert(partitions(0).isInstanceOf[GeodeRDDPartition])
-    assert(partitions(0).asInstanceOf[GeodeRDDPartition].bucketSet.isEmpty)
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=1 & no empty bucket") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 6, 1)
-    verifyPartitions(partitions, List(
-      (Set(0, 1, 2, 3), Seq("server1")), (Set(4, 5), Seq("server2"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=1 & 1 empty bucket") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 7, 1)
-    verifyPartitions(partitions, List(
-      (Set(0, 1, 2, 3, 6), Seq("server1")), (Set(4, 5), Seq("server2"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=1 & 2 empty bucket") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 8, 1)
-    verifyPartitions(partitions, List(
-      (Set(0, 1, 2, 3, 6), Seq("server1")), (Set(4, 5, 7), Seq("server2"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=1 & 5 empty bucket") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 11, 1)
-    verifyPartitions(partitions, List(
-      (Set(0, 1, 2, 3, 6, 7, 8), Seq("server1")), (Set(4, 5, 9, 10), Seq("server2"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=1, 4 empty-bucket, non-continuous IDs") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "server1" -> mutable.Set(1, 3), "server2" -> mutable.Set(5,7))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 8, 1)
-    verifyPartitions(partitions, List(
-      (Set(0, 1, 2, 3), Seq("server1")), (Set(4, 5, 6, 7), Seq("server2"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=2, no empty buckets, 3 servers have 1, 2, and 3 buckets") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "s1" -> mutable.Set(0), "s2" -> mutable.Set(1, 2), "s3" -> mutable.Set(3, 4, 5))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 6, 2)
-    // partitions.foreach(println)
-    verifyPartitions(partitions, List(
-      (Set(0), Seq("s1")), (Set(1), Seq("s2")), (Set(2), Seq("s2")), (Set(3, 4), Seq("s3")), (Set(5), Seq("s3"))))
-  }
-
-  test("ServerSplitsPartitioner.doPartitions(): n=3, no empty buckets, 4 servers have 0, 2, 3, and 4 buckets") {
-    val map: List[(String, mutable.Set[Int])] = List(
-      "s0" -> mutable.Set.empty, "s1" -> mutable.Set(0, 1), "s2" -> mutable.Set(2, 3, 4), "s3" -> mutable.Set(5, 6, 7, 8))
-    val partitions = ServerSplitsPartitioner.doPartitions(map, 9, 3)
-    // partitions.foreach(println)
-    verifyPartitions(partitions, List(
-      (Set(0), Seq("s1")), (Set(1), Seq("s1")), (Set(2), Seq("s2")), (Set(3), Seq("s2")), (Set(4), Seq("s2")),
-      (Set(5, 6), Seq("s3")), (Set(7, 8), Seq("s3")) ))
-  }
-
-  test("ServerSplitsPartitioner.partitions(): metadata = None ") {
-    val regionPath = "test"
-    val mockConnection = mock[GeodeConnection]
-    intercept[RuntimeException] { ServerSplitsPartitioner.partitions[String, String](mockConnection, null, Map.empty) }
-  }
-
-  test("ServerSplitsPartitioner.partitions(): replicated region ") {
-    val regionPath = "test"
-    val mockConnection = mock[GeodeConnection]
-    val md = new RegionMetadata(regionPath, false, 11, null)
-    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
-    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map.empty)
-    verifySinglePartition(partitions)
-  }
-
-  test("ServerSplitsPartitioner.partitions(): partitioned region w/o data ") {
-    val regionPath = "test"
-    val mockConnection = mock[GeodeConnection]
-    val md = new RegionMetadata(regionPath, true, 6, emptyServerBucketMap)
-    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
-    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map.empty)
-    verifySinglePartition(partitions)
-  }
-
-  test("ServerSplitsPartitioner.partitions(): partitioned region w/ some data ") {
-    import io.pivotal.geode.spark.connector.NumberPartitionsPerServerPropKey
-    val regionPath = "test"
-    val mockConnection = mock[GeodeConnection]
-    val map: Map[(String, Int), Set[Int]] = Map(
-      ("s0",1) -> Set.empty, ("s1",2) -> Set(0), ("s2",3) -> Set(1, 2), ("s3",4) -> Set(3, 4, 5))
-    val md = new RegionMetadata(regionPath, true, 6, toJavaServerBucketMap(map))
-    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
-    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map(NumberPartitionsPerServerPropKey->"2"))
-    // partitions.foreach(println)
-    verifyPartitions(partitions, List(
-      (Set(0), Seq("s1")), (Set(1), Seq("s2")), (Set(2), Seq("s2")), (Set(3, 4), Seq("s3")), (Set(5), Seq("s3"))))
-  }
-  
-  // Note: since the order of partitions is not pre-determined, we have to verify partition id
-  // and contents separately
-  def verifyPartitions(partitions: Array[Partition], expPartitions: List[(Set[Int], Seq[String])]): Unit = {
-    // 1. check size
-    assert(partitions.size == expPartitions.size)
-    // 2. check IDs are 0 to n-1
-    (0 until partitions.size).toList.zip(partitions).foreach { case (id, p) => assert(id == p.index) }
-
-    // 3. get all pairs of bucket set and its locations, and compare to the expected pairs
-    val list = partitions.map { e =>
-      val p = e.asInstanceOf[GeodeRDDPartition]
-      (p.bucketSet, p.locations)
-    }
-    expPartitions.foreach(e => assert(list.contains(e)))    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRegionRDDTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRegionRDDTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRegionRDDTest.scala
deleted file mode 100644
index 046ceac..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/rdd/GeodeRegionRDDTest.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 unittest.io.pivotal.geode.spark.connector.rdd
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.internal.RegionMetadata
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodeRDDPartition, GeodeRegionRDD}
-import io.pivotal.geode.spark.connector.{GeodeConnectionConf, GeodeConnection}
-import org.apache.spark.{TaskContext, Partition, SparkContext}
-import org.mockito.Mockito._
-import org.mockito.Matchers.{eq => mockEq, any => mockAny}
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{Matchers, FunSuite}
-
-import scala.reflect.ClassTag
-
-class GeodeRegionRDDTest extends FunSuite with Matchers with MockitoSugar {
-
-  /** create common mocks, not all mocks are used by all tests */
-  def createMocks[K, V](regionPath: String)(implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]])
-    : (String, Region[K,V], GeodeConnectionConf, GeodeConnection) = {
-    val mockConnection = mock[GeodeConnection]
-    val mockRegion = mock[Region[K, V]]
-    val mockConnConf = mock[GeodeConnectionConf]
-    when(mockConnConf.getConnection).thenReturn(mockConnection)
-    when(mockConnection.getRegionProxy[K, V](regionPath)).thenReturn(mockRegion)
-    when(mockConnConf.locators).thenReturn(Seq.empty)
-    (regionPath, mockRegion, mockConnConf, mockConnection)
-  }
-  
-  test("create GeodeRDD with non-existing region") {
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    when(mockConnConf.getConnection).thenReturn(mockConnection)
-    when(mockConnection.validateRegion[String,String](regionPath)).thenThrow(new RuntimeException)
-    val mockSparkContext = mock[SparkContext]
-    intercept[RuntimeException] { GeodeRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf) }
-    verify(mockConnConf).getConnection
-    verify(mockConnection).validateRegion[String, String](regionPath)
-  }
-  
-  test("getPartitions with non-existing region") {
-    // region exists when RDD is created, but get removed before getPartitions() is invoked
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(None)
-    val mockSparkContext = mock[SparkContext]
-    intercept[RuntimeException] { GeodeRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf).getPartitions }
-  }
-
-  test("getPartitions with replicated region and not preferred env") {
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    implicit val mockConnConf2 = mockConnConf
-    val mockSparkContext = mock[SparkContext]
-    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, false, 0, null)))
-    val partitions = GeodeRegionRDD(mockSparkContext, regionPath, mockConnConf).partitions
-    verifySinglePartition(partitions)
-  }
-
-  def verifySinglePartition(partitions: Array[Partition]): Unit = {
-    assert(1 == partitions.size)
-    assert(partitions(0).index === 0)
-    assert(partitions(0).isInstanceOf[GeodeRDDPartition])
-    assert(partitions(0).asInstanceOf[GeodeRDDPartition].bucketSet.isEmpty)
-  }
-
-  test("getPartitions with replicated region and preferred OnePartitionPartitioner") {
-    // since it's replicated region, so OnePartitionPartitioner will be used, i.e., override preferred partitioner
-    import io.pivotal.geode.spark.connector.{PreferredPartitionerPropKey, OnePartitionPartitionerName}
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, false, 0, null)))
-    implicit val mockConnConf2 = mockConnConf
-    val mockSparkContext = mock[SparkContext]
-    val env = Map(PreferredPartitionerPropKey -> OnePartitionPartitionerName)
-    val partitions = GeodeRegionRDD(mockSparkContext, regionPath, mockConnConf, env).partitions
-    verifySinglePartition(partitions)
-  }
-
-  test("getPartitions with partitioned region and not preferred env") {
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    implicit val mockConnConf2 = mockConnConf
-    val mockSparkContext = mock[SparkContext]
-    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, true, 2, null)))
-    val partitions = GeodeRegionRDD(mockSparkContext, regionPath, mockConnConf).partitions
-    verifySinglePartition(partitions)
-  }
-
-  test("GeodeRDD.compute() method") {
-    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
-    implicit val mockConnConf2 = mockConnConf
-    val mockIter = mock[Iterator[(String, String)]]
-    val partition = GeodeRDDPartition(0, Set.empty)
-    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, true, 2, null)))
-    when(mockConnection.getRegionData[String, String](regionPath, None, partition)).thenReturn(mockIter)
-    val mockSparkContext = mock[SparkContext]
-    val rdd = GeodeRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf)
-    val partitions = rdd.partitions
-    assert(1 == partitions.size)
-    val mockTaskContext = mock[TaskContext]
-    rdd.compute(partitions(0), mockTaskContext)        
-    verify(mockConnection).getRegionData[String, String](mockEq(regionPath), mockEq(None), mockEq(partition))
-    // verify(mockConnection).getRegionData[String, String](regionPath, Set.empty.asInstanceOf[Set[Int]], "geodeRDD 0.0")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/Emp.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/Emp.java b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/Emp.java
deleted file mode 100644
index 03e15a0..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/Emp.java
+++ /dev/null
@@ -1,95 +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 demo;
-
-import java.io.Serializable;
-
-/**
- * This is a demo class used in doc/?.md
- */
-public class Emp implements Serializable {
-
-  private int id;
-  
-  private String lname;
-
-  private String fname;
-
-  private int age;
-
-  private String loc;
-
-  public Emp(int id, String lname, String fname, int age, String loc) {
-    this.id = id;
-    this.lname = lname;
-    this.fname = fname;
-    this.age = age;
-    this.loc = loc;
-  }
-
-  public int getId() {
-    return id;
-  }
-
-  public String getLname() {
-    return lname;
-  }
-
-  public String getFname() {
-    return fname;
-  }
-
-  public int getAge() {
-    return age;
-  }
-
-  public String getLoc() {
-    return loc;
-  }
-
-  @Override
-  public String toString() {
-    return "Emp(" + id + ", " + lname + ", " + fname + ", " + age + ", " + loc + ")";
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    Emp emp = (Emp) o;
-
-    if (age != emp.age) return false;
-    if (id != emp.id) return false;
-    if (fname != null ? !fname.equals(emp.fname) : emp.fname != null) return false;
-    if (lname != null ? !lname.equals(emp.lname) : emp.lname != null) return false;
-    if (loc != null ? !loc.equals(emp.loc) : emp.loc != null) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = id;
-    result = 31 * result + (lname != null ? lname.hashCode() : 0);
-    result = 31 * result + (fname != null ? fname.hashCode() : 0);
-    result = 31 * result + age;
-    result = 31 * result + (loc != null ? loc.hashCode() : 0);
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
deleted file mode 100644
index adcf072..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
+++ /dev/null
@@ -1,59 +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 demo;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.sql.DataFrame;
-import org.apache.spark.sql.SQLContext;
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
-
-
-/**
- * This Spark application demonstrates how to get region data from Geode using Geode
- * OQL Java API. The result is a Spark DataFrame.
- * <p>
- * In order to run it, you will need to start a Geode cluster, and run demo PairRDDSaveJavaDemo
- * first to create some data in the region.
- * <p>
- * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
- * should be generated under geode-spark-demos/basic-demos/target/scala-2.10/.
- * Then run the following command to start a Spark job:
- * <pre>
- *   <path to spark>/bin/spark-submit --master=local[2] --class demo.OQLJavaDemo \
- *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
- * </pre>
- */
-public class OQLJavaDemo {
-
-  public static void main(String[] argv) {
-
-    if (argv.length != 1) {
-      System.err.printf("Usage: OQLJavaDemo <locators>\n");
-      return;
-    }
-
-    SparkConf conf = new SparkConf().setAppName("OQLJavaDemo");
-    conf.set(GeodeLocatorPropKey, argv[0]); // "192.168.1.47[10335]"
-    JavaSparkContext sc = new JavaSparkContext(conf);
-    SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
-    DataFrame df = javaFunctions(sqlContext).geodeOQL("select * from /str_str_region");
-    System.out.println("======= DataFrame =======\n");
-    df.show();
-    sc.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
deleted file mode 100644
index 52d2a99..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
+++ /dev/null
@@ -1,86 +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 demo;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import scala.Tuple2;
-import java.util.*;
-
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
-
-/**
- * This Spark application demonstrates how to save a RDD to Geode using Geode Spark
- * Connector with Java.
- * <p/>
- * In order to run it, you will need to start Geode cluster, and create the following region
- * with GFSH:
- * <pre>
- * gfsh> create region --name=str_str_region --type=REPLICATE \
- *         --key-constraint=java.lang.String --value-constraint=java.lang.String
- * </pre>
- * 
- * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
- * should be generated under geode-spark-demos/basic-demos/target/scala-2.10/.
- * Then run the following command to start a Spark job:
- * <pre>
- *   <path to spark>/bin/spark-submit --master=local[2] --class demo.PairRDDSaveJavaDemo \
- *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
- * </pre>
- * 
- * Verify the data was saved to Geode with GFSH:
- * <pre>gfsh> query --query="select * from /str_str_region.entrySet"  </pre>
- */
-public class PairRDDSaveJavaDemo {
-
-  public static void main(String[] argv) {
-
-    if (argv.length != 1) {
-      System.err.printf("Usage: PairRDDSaveJavaDemo <locators>\n");
-      return;
-    }
-
-    SparkConf conf = new SparkConf().setAppName("PairRDDSaveJavaDemo");
-    conf.set(GeodeLocatorPropKey, argv[0]);
-    JavaSparkContext sc = new JavaSparkContext(conf);
-    GeodeConnectionConf connConf = GeodeConnectionConf.apply(conf);
-
-    List<Tuple2<String, String>> data = new ArrayList<>();
-    data.add(new Tuple2<>("7", "seven"));
-    data.add(new Tuple2<>("8", "eight"));
-    data.add(new Tuple2<>("9", "nine"));
-
-    List<Tuple2<String, String>> data2 = new ArrayList<Tuple2<String, String>>();
-    data2.add(new Tuple2<>("11", "eleven"));
-    data2.add(new Tuple2<>("12", "twelve"));
-    data2.add(new Tuple2<>("13", "thirteen"));
-
-    // method 1: generate JavaPairRDD directly
-    JavaPairRDD<String, String> rdd1 =  sc.parallelizePairs(data);
-    javaFunctions(rdd1).saveToGeode("str_str_region", connConf);
-
-    // method 2: convert JavaRDD<Tuple2<K,V>> to JavaPairRDD<K, V>
-    JavaRDD<Tuple2<String, String>> rdd2 =  sc.parallelize(data2);
-    javaFunctions(toJavaPairRDD(rdd2)).saveToGeode("str_str_region", connConf);
-       
-    sc.stop();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
deleted file mode 100644
index 1125de5..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
+++ /dev/null
@@ -1,85 +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 demo;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.PairFunction;
-import scala.Tuple2;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
-
-/**
- * This Spark application demonstrates how to save a RDD to Geode using Geode Spark
- * Connector with Java.
- * <p/>
- * In order to run it, you will need to start Geode cluster, and create the following region
- * with GFSH:
- * <pre>
- * gfsh> create region --name=str_int_region --type=REPLICATE \
- *         --key-constraint=java.lang.String --value-constraint=java.lang.Integer
- * </pre>
- *
- * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
- * should be generated under geode-spark-demos/basic-demos/target/scala-2.10/.
- * Then run the following command to start a Spark job:
- * <pre>
- *   <path to spark>/bin/spark-submit --master=local[2] --class demo.RDDSaveJavaDemo \
- *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
- * </pre>
- *
- * Verify the data was saved to Geode with GFSH:
- * <pre>gfsh> query --query="select * from /str_int_region.entrySet"  </pre>
- */
-public class RDDSaveJavaDemo {
-
-  public static void main(String[] argv) {
-
-    if (argv.length != 1) {
-      System.err.printf("Usage: RDDSaveJavaDemo <locators>\n");
-      return;
-    }
-
-    SparkConf conf = new SparkConf().setAppName("RDDSaveJavaDemo");
-    conf.set(GeodeLocatorPropKey, argv[0]);
-    JavaSparkContext sc = new JavaSparkContext(conf);
-
-    List<String> data = new ArrayList<String>();
-    data.add("abcdefg");
-    data.add("abcdefgh");
-    data.add("abcdefghi");
-    JavaRDD<String> rdd =  sc.parallelize(data);
-
-    GeodeConnectionConf connConf = GeodeConnectionConf.apply(conf);
-
-    PairFunction<String, String, Integer> func =  new PairFunction<String, String, Integer>() {
-      @Override public Tuple2<String, Integer> call(String s) throws Exception {
-        return new Tuple2<String, Integer>(s, s.length());
-      }
-    };
-
-    javaFunctions(rdd).saveToGeode("str_int_region", func, connConf);
-
-    sc.stop();
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
deleted file mode 100644
index 1ce8ceb..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
+++ /dev/null
@@ -1,57 +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 demo;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
-
-/**
- * This Spark application demonstrates how to expose a region in Geode as a RDD using Geode
- * Spark Connector with Java.
- * <p>
- * In order to run it, you will need to start Geode cluster, and run demo PairRDDSaveJavaDemo
- * first to create some data in the region.
- * <p>
- * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
- * should be generated under geode-spark-demos/basic-demos/target/scala-2.10/.
- * Then run the following command to start a Spark job:
- * <pre>
- *   <path to spark>/bin/spark-submit --master=local[2] --class demo.RegionToRDDJavaDemo \
- *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
- * </pre>
- */
-public class RegionToRDDJavaDemo {
-
-  public static void main(String[] argv) {
-
-    if (argv.length != 1) {
-      System.err.printf("Usage: RegionToRDDJavaDemo <locators>\n");
-      return;
-    }
-    
-    SparkConf conf = new SparkConf().setAppName("RegionToRDDJavaDemo"); 
-    conf.set(GeodeLocatorPropKey, argv[0]);
-    JavaSparkContext sc = new JavaSparkContext(conf);
-
-    JavaPairRDD<String, String> rdd = javaFunctions(sc).geodeRegion("str_str_region");
-    System.out.println("=== geodeRegion =======\n" + rdd.collect() + "\n=========================");
-    
-    sc.stop();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala b/geode-spark-connector/geode-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
deleted file mode 100644
index 810b380..0000000
--- a/geode-spark-connector/geode-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
+++ /dev/null
@@ -1,75 +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 demo
-
-import org.apache.spark.SparkConf
-import org.apache.spark.streaming.{Seconds, StreamingContext}
-import io.pivotal.geode.spark.connector.GeodeLocatorPropKey
-import io.pivotal.geode.spark.connector.streaming._
-
-/**
- * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
- * <p><p>
- * In order to run it, you will need to start Geode cluster, and create the following region
- * with GFSH:
- * <pre>
- * gfsh> create region --name=str_int_region --type=REPLICATE \
- *         --key-constraint=java.lang.String --value-constraint=java.lang.Integer
- * </pre> 
- *
- * <p>To run this on your local machine, you need to first run a net cat server
- *    `$ nc -lk 9999`
- * and then run the example
- *    `$ bin/spark-submit --master=local[2] --class demo.NetworkWordCount <path to>/basic-demos_2.10-0.5.0.jar localhost 9999 locatorHost:port`
- * 
- * <p><p> check result that was saved to Geode with GFSH:
- * <pre>gfsh> query --query="select * from /str_int_region.entrySet"  </pre>
- */
-object NetworkWordCount {
-  
-  def main(args: Array[String]) {
-    if (args.length < 3) {
-      System.err.println("Usage: NetworkWordCount <hostname> <port> <geode locator>")
-      System.exit(1)
-    }
-
-    val updateFunc = (values: Seq[Int], state: Option[Int]) => {
-      val currentCount = values.foldLeft(0)(_ + _)
-      val previousCount = state.getOrElse(0)
-      Some(currentCount + previousCount)
-    }
-    
-    // Create the context with a 1 second batch size
-    val sparkConf = new SparkConf().setAppName("NetworkWordCount").set(GeodeLocatorPropKey, args(2))
-    val ssc = new StreamingContext(sparkConf, Seconds(1))
-    ssc.checkpoint(".")
-    
-    // Create a socket stream on target ip:port and count the
-    // words in input stream of \n delimited text (eg. generated by 'nc')
-    // Note that no duplication in storage level only for running locally.
-    // Replication necessary in distributed scenario for fault tolerance.
-    val lines = ssc.socketTextStream(args(0), args(1).toInt)
-    val words = lines.flatMap(_.split(" "))
-    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
-    val runningCounts = wordCounts.updateStateByKey[Int](updateFunc)
-    // runningCounts.print()
-    runningCounts.saveToGeode("str_int_region")
-    ssc.start()
-    ssc.awaitTermination()
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/project/Dependencies.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/project/Dependencies.scala b/geode-spark-connector/project/Dependencies.scala
index b2931dd..c77c158 100644
--- a/geode-spark-connector/project/Dependencies.scala
+++ b/geode-spark-connector/project/Dependencies.scala
@@ -22,7 +22,7 @@ object Dependencies {
   object Compile {
     val sparkStreaming = "org.apache.spark" %% "spark-streaming" % "1.3.0" 
     val sparkSql = "org.apache.spark" %% "spark-sql" % "1.3.0"
-    val geode = "org.apache.geode" % "geode-core" % "1.0.0-incubating.M2-SNAPSHOT" excludeAll(ExclusionRule(organization = "org.jboss.netty") )
+    val gemfire = "org.apache.geode" % "geode-core" % "1.0.0-incubating.M2-SNAPSHOT" excludeAll(ExclusionRule(organization = "org.jboss.netty") )
   }
 
   object Test {
@@ -37,9 +37,9 @@ object Dependencies {
 
   val unitTests = Seq(scalaTest, mockito, junit, novoCode)
 
-  val connector = unitTests ++ Seq(sparkStreaming, sparkSql, geode)
+  val connector = unitTests ++ Seq(sparkStreaming, sparkSql, gemfire)
 
-  val functions = Seq(geode, junit)
+  val functions = Seq(gemfire, junit)
  
-  val demos = Seq(sparkStreaming, sparkSql, geode)
+  val demos = Seq(sparkStreaming, sparkSql, gemfire)
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/project/GemFireSparkBuild.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/project/GemFireSparkBuild.scala b/geode-spark-connector/project/GemFireSparkBuild.scala
new file mode 100644
index 0000000..89d8e0b
--- /dev/null
+++ b/geode-spark-connector/project/GemFireSparkBuild.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+import sbt._
+import sbt.Keys._
+import scoverage.ScoverageSbtPlugin._
+import scoverage.ScoverageSbtPlugin
+
+object GemFireSparkConnectorBuild extends Build {
+  import Settings._
+  import Dependencies._ 
+
+  lazy val root = Project(
+    id = "root", 
+    base =file("."), 
+    aggregate = Seq(gemfireFunctions, gemfireSparkConnector,demos),
+    settings = commonSettings ++ Seq( 
+     name := "GemFire Connector for Apache Spark",
+     publishArtifact :=  false,
+     publishLocal := { },
+     publish := { }
+    )
+  )
+ 
+  lazy val gemfireFunctions = Project(
+    id = "gemfire-functions",
+    base = file("gemfire-functions"),
+    settings = commonSettings ++ Seq(libraryDependencies ++= Dependencies.functions,
+      resolvers ++= gfcResolvers,
+      description := "Required GemFire Functions to be deployed onto the GemFire Cluster before using the GemFire Spark Connector"
+    )
+  ).configs(IntegrationTest)
+  
+  lazy val gemfireSparkConnector = Project(
+    id = "gemfire-spark-connector",
+    base = file("gemfire-spark-connector"),
+    settings = gfcSettings ++ Seq(libraryDependencies ++= Dependencies.connector,
+      resolvers ++= gfcResolvers,
+      description := "A library that exposes GemFire regions as Spark RDDs, writes Spark RDDs to GemFire regions, and executes OQL queries from Spark Applications to GemFire"
+    )
+  ).dependsOn(gemfireFunctions).configs(IntegrationTest)
+
+ 
+  /******** Demo Project Definitions ********/ 
+  lazy val demoPath = file("gemfire-spark-demos")
+
+  lazy val demos = Project ( 
+    id = "gemfire-spark-demos",
+    base = demoPath,
+    settings = demoSettings,
+    aggregate = Seq(basicDemos)
+  )
+ 
+  lazy val basicDemos = Project (
+    id = "basic-demos",
+    base = demoPath / "basic-demos",
+    settings = demoSettings ++ Seq(libraryDependencies ++= Dependencies.demos,
+      resolvers ++= gfcResolvers,
+      description := "Sample applications that demonstrates functionality of the GemFire Spark Connector"
+    )
+  ).dependsOn(gemfireSparkConnector)
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/project/GeodeSparkBuild.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/project/GeodeSparkBuild.scala b/geode-spark-connector/project/GeodeSparkBuild.scala
deleted file mode 100644
index 07cae51..0000000
--- a/geode-spark-connector/project/GeodeSparkBuild.scala
+++ /dev/null
@@ -1,76 +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.
- */
-import sbt._
-import sbt.Keys._
-import scoverage.ScoverageSbtPlugin._
-import scoverage.ScoverageSbtPlugin
-
-object GeodeSparkConnectorBuild extends Build {
-  import Settings._
-  import Dependencies._ 
-
-  lazy val root = Project(
-    id = "root", 
-    base =file("."), 
-    aggregate = Seq(geodeFunctions, geodeSparkConnector,demos),
-    settings = commonSettings ++ Seq( 
-     name := "Geode Connector for Apache Spark",
-     publishArtifact :=  false,
-     publishLocal := { },
-     publish := { }
-    )
-  )
- 
-  lazy val geodeFunctions = Project(
-    id = "geode-functions",
-    base = file("geode-functions"),
-    settings = commonSettings ++ Seq(libraryDependencies ++= Dependencies.functions,
-      resolvers ++= gfcResolvers,
-      description := "Required Geode Functions to be deployed onto the Geode Cluster before using the Geode Spark Connector"
-    )
-  ).configs(IntegrationTest)
-  
-  lazy val geodeSparkConnector = Project(
-    id = "geode-spark-connector",
-    base = file("geode-spark-connector"),
-    settings = gfcSettings ++ Seq(libraryDependencies ++= Dependencies.connector,
-      resolvers ++= gfcResolvers,
-      description := "A library that exposes Geode regions as Spark RDDs, writes Spark RDDs to Geode regions, and executes OQL queries from Spark Applications to Geode"
-    )
-  ).dependsOn(geodeFunctions).configs(IntegrationTest)
-
- 
-  /******** Demo Project Definitions ********/ 
-  lazy val demoPath = file("geode-spark-demos")
-
-  lazy val demos = Project ( 
-    id = "geode-spark-demos",
-    base = demoPath,
-    settings = demoSettings,
-    aggregate = Seq(basicDemos)
-  )
- 
-  lazy val basicDemos = Project (
-    id = "basic-demos",
-    base = demoPath / "basic-demos",
-    settings = demoSettings ++ Seq(libraryDependencies ++= Dependencies.demos,
-      resolvers ++= gfcResolvers,
-      description := "Sample applications that demonstrates functionality of the Geode Spark Connector"
-    )
-  ).dependsOn(geodeSparkConnector)
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/project/Settings.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/project/Settings.scala b/geode-spark-connector/project/Settings.scala
index 9aefa9b..796541c 100644
--- a/geode-spark-connector/project/Settings.scala
+++ b/geode-spark-connector/project/Settings.scala
@@ -24,7 +24,7 @@ object Settings extends Build {
     organization := "io.pivotal",
     version := "0.5.0",
     scalaVersion := "2.10.4",
-    organization := "io.pivotal.geode.spark",
+    organization := "io.pivotal.gemfire.spark",
     organizationHomepage := Some(url("http://www.pivotal.io/"))
   ) 
 
@@ -43,7 +43,7 @@ object Settings extends Build {
   val gfcITSettings = inConfig(IntegrationTest)(Defaults.itSettings) ++
     Seq(parallelExecution in IntegrationTest := false, fork in IntegrationTest := true)
 
-  val gfcCompileSettings = inConfig(Compile)(Defaults.compileSettings) ++ Seq(unmanagedSourceDirectories in Compile += baseDirectory.value /"../geode-functions/src")
+  val gfcCompileSettings = inConfig(Compile)(Defaults.compileSettings) ++ Seq(unmanagedSourceDirectories in Compile += baseDirectory.value /"../gemfire-functions/src")
 
   val gfcSettings = commonSettings ++ gfcITSettings ++ gfcCompileSettings 
 


[03/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnection.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnection.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnection.scala
deleted file mode 100644
index ff4cd17..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnection.scala
+++ /dev/null
@@ -1,67 +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 io.pivotal.geode.spark.connector
-
-import com.gemstone.gemfire.cache.execute.ResultCollector
-import com.gemstone.gemfire.cache.query.Query
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.internal.RegionMetadata
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRDDPartition
-
-
-trait GeodeConnection {
-
-  /**
-   * Validate region existence and key/value type constraints, throw RuntimeException
-   * if region does not exist or key and/or value type do(es) not match.
-   * @param regionPath the full path of region
-   */
-  def validateRegion[K, V](regionPath: String): Unit
-
-  /**
-   * Get Region proxy for the given region
-   * @param regionPath the full path of region
-   */
-  def getRegionProxy[K, V](regionPath: String): Region[K, V]
-
-  /**
-   * Retrieve region meta data for the given region. 
-   * @param regionPath: the full path of the region
-   * @return Some[RegionMetadata] if region exists, None otherwise
-   */
-  def getRegionMetadata[K, V](regionPath: String): Option[RegionMetadata]
-
-  /** 
-   * Retrieve region data for the given region and bucket set 
-   * @param regionPath: the full path of the region
-   * @param whereClause: the set of bucket IDs
-   * @param split: Geode RDD Partition instance
-   */
-  def getRegionData[K, V](regionPath: String, whereClause: Option[String], split: GeodeRDDPartition): Iterator[(K, V)]
-
-  def executeQuery(regionPath: String, bucketSet: Set[Int], queryString: String): Object
-  /** 
-   * Create a geode OQL query
-   * @param queryString Geode OQL query string
-   */
-  def getQuery(queryString: String): Query
-
-  /** Close the connection */
-  def close(): Unit
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionConf.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionConf.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionConf.scala
deleted file mode 100644
index 38d9e07..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionConf.scala
+++ /dev/null
@@ -1,73 +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 io.pivotal.geode.spark.connector
-
-import org.apache.spark.SparkConf
-import io.pivotal.geode.spark.connector.internal.{DefaultGeodeConnectionManager, LocatorHelper}
-
-/**
- * Stores configuration of a connection to Geode cluster. It is serializable and can
- * be safely sent over network.
- *
- * @param locators Geode locator host:port pairs, the default is (localhost,10334)
- * @param geodeProps The initial geode properties to be used.
- * @param connectionManager GeodeConnectionFactory instance
- */
-class GeodeConnectionConf(
-   val locators: Seq[(String, Int)], 
-   val geodeProps: Map[String, String] = Map.empty,
-   connectionManager: GeodeConnectionManager = new DefaultGeodeConnectionManager
-  ) extends Serializable {
-
-  /** require at least 1 pair of (host,port) */
-  require(locators.nonEmpty)
-  
-  def getConnection: GeodeConnection = connectionManager.getConnection(this)
-  
-}
-
-object GeodeConnectionConf {
-
-  /**
-   * create GeodeConnectionConf object based on locator string and optional GeodeConnectionFactory
-   * @param locatorStr Geode cluster locator string
-   * @param connectionManager GeodeConnection factory
-   */
-  def apply(locatorStr: String, geodeProps: Map[String, String] = Map.empty)
-    (implicit connectionManager: GeodeConnectionManager = new DefaultGeodeConnectionManager): GeodeConnectionConf = {
-    new GeodeConnectionConf(LocatorHelper.parseLocatorsString(locatorStr), geodeProps, connectionManager)
-  }
-
-  /**
-   * create GeodeConnectionConf object based on SparkConf. Note that implicit can
-   * be used to control what GeodeConnectionFactory instance to use if desired
-   * @param conf a SparkConf instance 
-   */
-  def apply(conf: SparkConf): GeodeConnectionConf = {
-    val locatorStr = conf.getOption(GeodeLocatorPropKey).getOrElse(
-      throw new RuntimeException(s"SparkConf does not contain property $GeodeLocatorPropKey"))
-    // SparkConf only holds properties whose key starts with "spark.", In order to
-    // put geode properties in SparkConf, all geode properties are prefixes with
-    // "spark.geode.". This prefix was removed before the properties were put in `geodeProp`
-    val prefix = "spark.geode."
-    val geodeProps = conf.getAll.filter {
-        case (k, v) => k.startsWith(prefix) && k != GeodeLocatorPropKey
-      }.map { case (k, v) => (k.substring(prefix.length), v) }.toMap
-    apply(locatorStr, geodeProps)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionManager.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionManager.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionManager.scala
deleted file mode 100644
index bf678f0..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeConnectionManager.scala
+++ /dev/null
@@ -1,31 +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 io.pivotal.geode.spark.connector
-
-/**
- * GeodeConnectionFactory provide an common interface that manages Geode
- * connections, and it's serializable. Each factory instance will handle
- * connection instance creation and connection pool management.
- */
-trait GeodeConnectionManager extends Serializable {
-
-  /** get connection for the given connector */
-  def getConnection(connConf: GeodeConnectionConf): GeodeConnection
-
-  /** close the connection */
-  def closeConnection(connConf: GeodeConnectionConf): Unit
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployer.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployer.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployer.scala
deleted file mode 100644
index 6e93b05..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployer.scala
+++ /dev/null
@@ -1,81 +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 io.pivotal.geode.spark.connector
-
-import java.io.File
-import java.net.URL
-import org.apache.commons.httpclient.methods.PostMethod
-import org.apache.commons.httpclient.methods.multipart.{FilePart, Part, MultipartRequestEntity}
-import org.apache.commons.httpclient.HttpClient
-import org.apache.spark.Logging
-
-object GeodeFunctionDeployer {
-  def main(args: Array[String]) {
-    new GeodeFunctionDeployer(new HttpClient()).commandLineRun(args)
-  }
-}
-
-class GeodeFunctionDeployer(val httpClient:HttpClient) extends Logging {
-
-  def deploy(host: String, port: Int, jarLocation: String): String =
-    deploy(host + ":" + port, jarLocation)
-  
-  def deploy(host: String, port: Int, jar:File): String =
-    deploy(host + ":" + port, jar)
-  
-  def deploy(jmxHostAndPort: String, jarLocation: String): String =
-    deploy(jmxHostAndPort, jarFileHandle(jarLocation))
-  
-  def deploy(jmxHostAndPort: String, jar: File): String = {
-    val urlString = constructURLString(jmxHostAndPort)
-    val filePost: PostMethod = new PostMethod(urlString)
-    val parts: Array[Part] = new Array[Part](1)
-    parts(0) = new FilePart("resources", jar)
-    filePost.setRequestEntity(new MultipartRequestEntity(parts, filePost.getParams))
-    val status: Int = httpClient.executeMethod(filePost)
-    "Deployed Jar with status:" + status
-  }
-
-  private[connector] def constructURLString(jmxHostAndPort: String) =
-    "http://" + jmxHostAndPort + "/gemfire/v1/deployed"
-
-  private[connector]def jarFileHandle(jarLocation: String) = {
-    val f: File = new File(jarLocation)
-    if (!f.exists()) {
-      val errorMessage: String = "Invalid jar file:" + f.getAbsolutePath
-      logInfo(errorMessage)
-      throw new RuntimeException(errorMessage)
-    }
-    f
-  }
-  
-  def commandLineRun(args: Array[String]):Unit = {
-    val (hostPort: String, jarFile: String) =
-    if (args.length < 2) {
-      logInfo("JMX Manager Host and Port (example: localhost:7070):")
-      val bufferedReader = new java.io.BufferedReader(new java.io.InputStreamReader(System.in))
-      val jmxHostAndPort = bufferedReader.readLine()
-      logInfo("Location of geode-functions.jar:")
-      val functionJarLocation = bufferedReader.readLine()
-      (jmxHostAndPort, functionJarLocation)
-    } else {
-      (args(0), args(1))
-    }
-    val status = deploy(hostPort, jarFile)
-    logInfo(status)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeKryoRegistrator.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeKryoRegistrator.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeKryoRegistrator.scala
deleted file mode 100644
index 8c0aeca..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeKryoRegistrator.scala
+++ /dev/null
@@ -1,29 +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 io.pivotal.geode.spark.connector
-
-import com.esotericsoftware.kryo.Kryo
-import io.pivotal.geode.spark.connector.internal.oql.UndefinedSerializer
-import org.apache.spark.serializer.KryoRegistrator
-import com.gemstone.gemfire.cache.query.internal.Undefined
-
-class GeodeKryoRegistrator extends KryoRegistrator{
-
-  override def registerClasses(kyro: Kryo): Unit = {
-    kyro.addDefaultSerializer(classOf[Undefined], classOf[UndefinedSerializer])
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodePairRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodePairRDDFunctions.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodePairRDDFunctions.scala
deleted file mode 100644
index ba5d2df..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodePairRDDFunctions.scala
+++ /dev/null
@@ -1,140 +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 io.pivotal.geode.spark.connector
-
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodeOuterJoinRDD, GeodeJoinRDD, GeodePairRDDWriter}
-import org.apache.spark.Logging
-import org.apache.spark.api.java.function.Function
-import org.apache.spark.rdd.RDD
-
-/**
- * Extra gemFire functions on RDDs of (key, value) pairs through an implicit conversion.
- * Import `io.pivotal.geode.spark.connector._` at the top of your program to
- * use these functions.
- */
-class GeodePairRDDFunctions[K, V](val rdd: RDD[(K, V)]) extends Serializable with Logging {
-
-  /**
-   * Save the RDD of pairs to Geode key-value store without any conversion
-   * @param regionPath the full path of region that the RDD is stored
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */
-  def saveToGeode(
-      regionPath: String, 
-      connConf: GeodeConnectionConf = defaultConnectionConf, 
-      opConf: Map[String, String] = Map.empty): Unit = {    
-    connConf.getConnection.validateRegion[K, V](regionPath)
-    if (log.isDebugEnabled)
-      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
-    else
-      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
-    val writer = new GeodePairRDDWriter[K, V](regionPath, connConf, opConf)
-    rdd.sparkContext.runJob(rdd, writer.write _)
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in `this`
-   * RDD and the Geode `Region[K, V2]`. Each pair of elements will be returned
-   * as a ((k, v), v2) tuple, where (k, v) is in `this` RDD and (k, v2) is in the
-   * Geode region.
-   *
-   *@param regionPath the region path of the Geode region
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K2 the key type of the Geode region
-   * @tparam V2 the value type of the Geode region
-   * @return RDD[T, V]
-   */
-  def joinGeodeRegion[K2 <: K, V2](
-    regionPath: String, connConf: GeodeConnectionConf = defaultConnectionConf): GeodeJoinRDD[(K, V), K, V2] = {
-    new GeodeJoinRDD[(K, V), K, V2](rdd, null, regionPath, connConf)
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in `this` RDD
-   * and the Geode `Region[K2, V2]`. The join key from RDD element is generated by
-   * `func(K, V) => K2`, and the key from the Geode region is jus the key of the
-   * key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
-   * where (k, v) is in `this` RDD and (k2, v2) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K2 the key type of the Geode region
-   * @tparam V2 the value type of the Geode region
-   * @return RDD[(K, V), V2]
-   */
-  def joinGeodeRegion[K2, V2](
-    regionPath: String, func: ((K, V)) => K2, connConf: GeodeConnectionConf = defaultConnectionConf): GeodeJoinRDD[(K, V), K2, V2] =
-    new GeodeJoinRDD[(K, V), K2, V2](rdd, func, regionPath, connConf)
-
-  /** This version of joinGeodeRegion(...) is just for Java API. */
-  private[connector] def joinGeodeRegion[K2, V2](
-    regionPath: String, func: Function[(K, V), K2], connConf: GeodeConnectionConf): GeodeJoinRDD[(K, V), K2, V2] = {
-    new GeodeJoinRDD[(K, V), K2, V2](rdd, func.call, regionPath, connConf)
-  }
-
-  /**
-   * Perform a left outer join of `this` RDD and the Geode `Region[K, V2]`.
-   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key k.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K2 the key type of the Geode region
-   * @tparam V2 the value type of the Geode region
-   * @return RDD[ (K, V), Option[V] ]
-   */
-  def outerJoinGeodeRegion[K2 <: K, V2](
-    regionPath: String, connConf: GeodeConnectionConf = defaultConnectionConf): GeodeOuterJoinRDD[(K, V), K, V2] = {
-    new GeodeOuterJoinRDD[(K, V), K, V2](rdd, null, regionPath, connConf)
-  }
-
-  /**
-   * Perform a left outer join of `this` RDD and the Geode `Region[K2, V2]`.
-   * The join key from RDD element is generated by `func(K, V) => K2`, and the
-   * key from region is jus the key of the key/value pair.
-   *
-   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key `func(k, v)`.
-   *
-   *@param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K2 the key type of the Geode region
-   * @tparam V2 the value type of the Geode region
-   * @return RDD[ (K, V), Option[V] ]
-   */
-  def outerJoinGeodeRegion[K2, V2](
-    regionPath: String, func: ((K, V)) => K2, connConf: GeodeConnectionConf = defaultConnectionConf): GeodeOuterJoinRDD[(K, V), K2, V2] = {
-    new GeodeOuterJoinRDD[(K, V), K2, V2](rdd, func, regionPath, connConf)
-  }
-
-  /** This version of outerJoinGeodeRegion(...) is just for Java API. */
-  private[connector] def outerJoinGeodeRegion[K2, V2](
-    regionPath: String, func: Function[(K, V), K2], connConf: GeodeConnectionConf): GeodeOuterJoinRDD[(K, V), K2, V2] = {
-    new GeodeOuterJoinRDD[(K, V), K2, V2](rdd, func.call, regionPath, connConf)
-  }
-
-  private[connector] def defaultConnectionConf: GeodeConnectionConf =
-    GeodeConnectionConf(rdd.sparkContext.getConf)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeRDDFunctions.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeRDDFunctions.scala
deleted file mode 100644
index 2e5c92a..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeRDDFunctions.scala
+++ /dev/null
@@ -1,120 +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 io.pivotal.geode.spark.connector
-
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodeOuterJoinRDD, GeodeJoinRDD, GeodeRDDWriter}
-import org.apache.spark.Logging
-import org.apache.spark.api.java.function.{PairFunction, Function}
-import org.apache.spark.rdd.RDD
-
-/**
- * Extra gemFire functions on non-Pair RDDs through an implicit conversion.
- * Import `io.pivotal.geode.spark.connector._` at the top of your program to 
- * use these functions.  
- */
-class GeodeRDDFunctions[T](val rdd: RDD[T]) extends Serializable with Logging {
-
-  /**
-   * Save the non-pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored  
-   * @param func the function that converts elements of RDD to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */
-  def saveToGeode[K, V](
-      regionPath: String, 
-      func: T => (K, V), 
-      connConf: GeodeConnectionConf = defaultConnectionConf,
-      opConf: Map[String, String] = Map.empty): Unit = {
-    connConf.getConnection.validateRegion[K, V](regionPath)
-    if (log.isDebugEnabled)
-      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
-    else
-      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
-    val writer = new GeodeRDDWriter[T, K, V](regionPath, connConf, opConf)
-    rdd.sparkContext.runJob(rdd, writer.write(func) _)
-  }
-
-  /** This version of saveToGeode(...) is just for Java API. */
-  private[connector] def saveToGeode[K, V](
-      regionPath: String, 
-      func: PairFunction[T, K, V], 
-      connConf: GeodeConnectionConf, 
-      opConf: Map[String, String]): Unit = {
-    saveToGeode[K, V](regionPath, func.call _, connConf, opConf)
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in `this` RDD
-   * and the Geode `Region[K, V]`. The join key from RDD element is generated by
-   * `func(T) => K`, and the key from the Geode region is just the key of the
-   * key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a (t, v) tuple, 
-   * where (t) is in `this` RDD and (k, v) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generate region key from RDD element T
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K the key type of the Geode region
-   * @tparam V the value type of the Geode region
-   * @return RDD[T, V]
-   */
-  def joinGeodeRegion[K, V](regionPath: String, func: T => K, 
-    connConf: GeodeConnectionConf = defaultConnectionConf): GeodeJoinRDD[T, K, V] = {
-    new GeodeJoinRDD[T, K, V](rdd, func, regionPath, connConf)    
-  }
-
-  /** This version of joinGeodeRegion(...) is just for Java API. */
-  private[connector] def joinGeodeRegion[K, V](
-    regionPath: String, func: Function[T, K], connConf: GeodeConnectionConf): GeodeJoinRDD[T, K, V] = {
-    joinGeodeRegion(regionPath, func.call _, connConf)
-  }
-
-  /**
-   * Perform a left outer join of `this` RDD and the Geode `Region[K, V]`.
-   * The join key from RDD element is generated by `func(T) => K`, and the
-   * key from region is just the key of the key/value pair.
-   *
-   * For each element (t) in `this` RDD, the resulting RDD will either contain
-   * all pairs (t, Some(v)) for v in the Geode region, or the pair
-   * (t, None) if no element in the Geode region have key `func(t)`
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generate region key from RDD element T
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @tparam K the key type of the Geode region
-   * @tparam V the value type of the Geode region
-   * @return RDD[ T, Option[V] ]
-   */
-  def outerJoinGeodeRegion[K, V](regionPath: String, func: T => K,
-    connConf: GeodeConnectionConf = defaultConnectionConf): GeodeOuterJoinRDD[T, K, V] = {
-    new GeodeOuterJoinRDD[T, K, V](rdd, func, regionPath, connConf)
-  }
-
-  /** This version of outerJoinGeodeRegion(...) is just for Java API. */
-  private[connector] def outerJoinGeodeRegion[K, V](
-    regionPath: String, func: Function[T, K], connConf: GeodeConnectionConf): GeodeOuterJoinRDD[T, K, V] = {
-    outerJoinGeodeRegion(regionPath, func.call _, connConf)
-  }
-
-  private[connector] def defaultConnectionConf: GeodeConnectionConf =
-    GeodeConnectionConf(rdd.sparkContext.getConf)
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSQLContextFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSQLContextFunctions.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSQLContextFunctions.scala
deleted file mode 100644
index 83aab7a..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSQLContextFunctions.scala
+++ /dev/null
@@ -1,42 +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 io.pivotal.geode.spark.connector
-
-import io.pivotal.geode.spark.connector.internal.oql.{OQLRelation, QueryRDD}
-import org.apache.spark.Logging
-import org.apache.spark.sql.{DataFrame, SQLContext}
-
-/**
- * Provide Geode OQL specific functions
- */
-class GeodeSQLContextFunctions(@transient sqlContext: SQLContext) extends Serializable with Logging {
-
-  /**
-   * Expose a Geode OQL query result as a DataFrame
-   * @param query the OQL query string.
-   */
-  def geodeOQL(
-    query: String,
-    connConf: GeodeConnectionConf = GeodeConnectionConf(sqlContext.sparkContext.getConf)): DataFrame = {
-    logInfo(s"OQL query = $query")
-    val rdd = new QueryRDD[Object](sqlContext.sparkContext, query, connConf)
-    sqlContext.baseRelationToDataFrame(OQLRelation(rdd)(sqlContext))
-  }
-
-  private[connector] def defaultConnectionConf: GeodeConnectionConf =
-    GeodeConnectionConf(sqlContext.sparkContext.getConf)
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSparkContextFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSparkContextFunctions.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSparkContextFunctions.scala
deleted file mode 100644
index 617cb33..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/GeodeSparkContextFunctions.scala
+++ /dev/null
@@ -1,39 +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 io.pivotal.geode.spark.connector
-
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRegionRDD
-import org.apache.spark.SparkContext
-
-import scala.reflect.ClassTag
-
-/** Provides Geode specific methods on `SparkContext` */
-class GeodeSparkContextFunctions(@transient sc: SparkContext) extends Serializable {
-
-  /**
-   * Expose a Geode region as a GeodeRDD
-   * @param regionPath the full path of the region
-   * @param connConf the GeodeConnectionConf that can be used to access the region
-   * @param opConf use this to specify preferred partitioner
-   *        and its parameters. The implementation will use it if it's applicable
-   */
-  def geodeRegion[K: ClassTag, V: ClassTag] (
-    regionPath: String, connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf),
-    opConf: Map[String, String] = Map.empty): GeodeRegionRDD[K, V] =
-    GeodeRegionRDD[K, V](sc, regionPath, connConf, opConf)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnection.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnection.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnection.scala
deleted file mode 100644
index 52f9961..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnection.scala
+++ /dev/null
@@ -1,164 +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 io.pivotal.geode.spark.connector.internal
-
-import java.net.InetAddress
-
-import com.gemstone.gemfire.cache.client.{ClientCache, ClientCacheFactory, ClientRegionShortcut}
-import com.gemstone.gemfire.cache.execute.{FunctionException, FunctionService}
-import com.gemstone.gemfire.cache.query.Query
-import com.gemstone.gemfire.cache.{Region, RegionService}
-import com.gemstone.gemfire.internal.cache.execute.InternalExecution
-import io.pivotal.geode.spark.connector.internal.oql.QueryResultCollector
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRDDPartition
-import org.apache.spark.{SparkEnv, Logging}
-import io.pivotal.geode.spark.connector.GeodeConnection
-import io.pivotal.geode.spark.connector.internal.geodefunctions._
-import java.util.{Set => JSet, List => JList }
-
-/**
- * Default GeodeConnection implementation. The instance of this should be
- * created by DefaultGeodeConnectionFactory
- * @param locators pairs of host/port of locators
- * @param gemFireProps The initial geode properties to be used.
- */
-private[connector] class DefaultGeodeConnection (
-  locators: Seq[(String, Int)], gemFireProps: Map[String, String] = Map.empty) 
-  extends GeodeConnection with Logging {
-
-  private val clientCache = initClientCache()
-
-  /** Register Geode functions to the Geode cluster */
-  FunctionService.registerFunction(RetrieveRegionMetadataFunction.getInstance())
-  FunctionService.registerFunction(RetrieveRegionFunction.getInstance())
-
-  private def initClientCache() : ClientCache = {
-    try {
-      val ccf = getClientCacheFactory
-      ccf.create()
-    } catch {
-      case e: Exception =>
-        logError(s"""Failed to init ClientCache, locators=${locators.mkString(",")}, Error: $e""")
-        throw new RuntimeException(e)
-    }
-  }
-  
-  private def getClientCacheFactory: ClientCacheFactory = {
-    import io.pivotal.geode.spark.connector.map2Properties
-    val ccf = new ClientCacheFactory(gemFireProps)
-    ccf.setPoolReadTimeout(30000)
-    val servers = LocatorHelper.getAllGeodeServers(locators)
-    if (servers.isDefined && servers.get.size > 0) {
-      val sparkIp = System.getenv("SPARK_LOCAL_IP")
-      val hostName = if (sparkIp != null) InetAddress.getByName(sparkIp).getCanonicalHostName
-                     else InetAddress.getLocalHost.getCanonicalHostName
-      val executorId = SparkEnv.get.executorId      
-      val pickedServers = LocatorHelper.pickPreferredGeodeServers(servers.get, hostName, executorId)
-      logInfo(s"""Init ClientCache: severs=${pickedServers.mkString(",")}, host=$hostName executor=$executorId props=$gemFireProps""")
-      logDebug(s"""Init ClientCache: all-severs=${pickedServers.mkString(",")}""")
-      pickedServers.foreach{ case (host, port)  => ccf.addPoolServer(host, port) }
-    } else {
-      logInfo(s"""Init ClientCache: locators=${locators.mkString(",")}, props=$gemFireProps""")
-      locators.foreach { case (host, port)  => ccf.addPoolLocator(host, port) }
-    }
-    ccf
-  }
-
-  /** close the clientCache */
-  override def close(): Unit =
-    if (! clientCache.isClosed) clientCache.close()
-
-  /** ----------------------------------------- */
-  /** implementation of GeodeConnection trait */
-  /** ----------------------------------------- */
-
-  override def getQuery(queryString: String): Query =
-    clientCache.asInstanceOf[RegionService].getQueryService.newQuery(queryString)
-
-  override def validateRegion[K, V](regionPath: String): Unit = {
-    val md = getRegionMetadata[K, V](regionPath)
-    if (! md.isDefined) throw new RuntimeException(s"The region named $regionPath was not found")
-  }
-
-  def getRegionMetadata[K, V](regionPath: String): Option[RegionMetadata] = {
-    import scala.collection.JavaConversions.setAsJavaSet
-    val region = getRegionProxy[K, V](regionPath)
-    val set0: JSet[Integer] = Set[Integer](0)
-    val exec = FunctionService.onRegion(region).asInstanceOf[InternalExecution].withBucketFilter(set0)
-    exec.setWaitOnExceptionFlag(true)
-    try {
-      val collector = exec.execute(RetrieveRegionMetadataFunction.ID)
-      val r = collector.getResult.asInstanceOf[JList[RegionMetadata]]
-      logDebug(r.get(0).toString)
-      Some(r.get(0))
-    } catch {
-      case e: FunctionException => 
-        if (e.getMessage.contains(s"The region named /$regionPath was not found")) None
-        else throw e
-    }
-  }
-
-  def getRegionProxy[K, V](regionPath: String): Region[K, V] = {
-    val region1: Region[K, V] = clientCache.getRegion(regionPath).asInstanceOf[Region[K, V]]
-    if (region1 != null) region1
-    else DefaultGeodeConnection.regionLock.synchronized {
-      val region2 = clientCache.getRegion(regionPath).asInstanceOf[Region[K, V]]
-      if (region2 != null) region2
-      else clientCache.createClientRegionFactory[K, V](ClientRegionShortcut.PROXY).create(regionPath)
-    }
-  }
-
-  override def getRegionData[K, V](regionPath: String, whereClause: Option[String], split: GeodeRDDPartition): Iterator[(K, V)] = {
-    val region = getRegionProxy[K, V](regionPath)
-    val desc = s"""RDD($regionPath, "${whereClause.getOrElse("")}", ${split.index})"""
-    val args : Array[String] = Array[String](whereClause.getOrElse(""), desc)
-    val collector = new StructStreamingResultCollector(desc)
-        // RetrieveRegionResultCollector[(K, V)]
-    import scala.collection.JavaConversions.setAsJavaSet
-    val exec = FunctionService.onRegion(region).withArgs(args).withCollector(collector).asInstanceOf[InternalExecution]
-      .withBucketFilter(split.bucketSet.map(Integer.valueOf))
-    exec.setWaitOnExceptionFlag(true)
-    exec.execute(RetrieveRegionFunction.ID)
-    collector.getResult.map{objs: Array[Object] => (objs(0).asInstanceOf[K], objs(1).asInstanceOf[V])}
-  }
-
-  override def executeQuery(regionPath: String, bucketSet: Set[Int], queryString: String) = {
-    import scala.collection.JavaConversions.setAsJavaSet
-    FunctionService.registerFunction(QueryFunction.getInstance())
-    val collector = new QueryResultCollector
-    val region = getRegionProxy(regionPath)
-    val args: Array[String] = Array[String](queryString, bucketSet.toString)
-    val exec = FunctionService.onRegion(region).withCollector(collector).asInstanceOf[InternalExecution]
-      .withBucketFilter(bucketSet.map(Integer.valueOf))
-      .withArgs(args)
-    exec.execute(QueryFunction.ID)
-    collector.getResult
-  }
-}
-
-private[connector] object DefaultGeodeConnection {
-  /** a lock object only used by getRegionProxy...() */
-  private val regionLock = new Object
-}
-
-/** The purpose of this class is making unit test DefaultGeodeConnectionManager easier */
-class DefaultGeodeConnectionFactory {
-
-  def newConnection(locators: Seq[(String, Int)], gemFireProps: Map[String, String] = Map.empty) =
-    new DefaultGeodeConnection(locators, gemFireProps)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnectionManager.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnectionManager.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnectionManager.scala
deleted file mode 100644
index eb67cda..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/DefaultGeodeConnectionManager.scala
+++ /dev/null
@@ -1,77 +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 io.pivotal.geode.spark.connector.internal
-
-import io.pivotal.geode.spark.connector.{GeodeConnection, GeodeConnectionConf, GeodeConnectionManager}
-
-import scala.collection.mutable
-
-/**
- * Default implementation of GeodeConnectionFactory
- */
-class DefaultGeodeConnectionManager extends GeodeConnectionManager {
-
-  def getConnection(connConf: GeodeConnectionConf): GeodeConnection =
-    DefaultGeodeConnectionManager.getConnection(connConf)
-
-  def closeConnection(connConf: GeodeConnectionConf): Unit =
-    DefaultGeodeConnectionManager.closeConnection(connConf)
-
-}
-
-object DefaultGeodeConnectionManager  {
-
-  /** connection cache, keyed by host:port pair */
-  private[connector] val connections = mutable.Map[(String, Int), GeodeConnection]()
-
-  /**
-   * use locator host:port pair to lookup cached connection. create new connection 
-   * and add it to the cache `connections` if it does not exist.
-   */
-  def getConnection(connConf: GeodeConnectionConf)
-    (implicit factory: DefaultGeodeConnectionFactory = new DefaultGeodeConnectionFactory): GeodeConnection = {
-
-    def getCachedConnection(locators: Seq[(String, Int)]): GeodeConnection = {
-      val conns = connConf.locators.map(connections withDefaultValue null).filter(_ != null)
-      if (conns.nonEmpty) conns(0) else null
-    }
-
-    val conn1 = getCachedConnection(connConf.locators)
-    if (conn1 != null) conn1
-    else connections.synchronized {
-      val conn2 = getCachedConnection(connConf.locators)
-      if (conn2 != null) conn2
-      else {
-        val conn3 = factory.newConnection(connConf.locators, connConf.geodeProps)
-        connConf.locators.foreach(pair => connections += (pair -> conn3))
-        conn3
-      }
-    }
-  }
-
-  /**
-   * Close the connection and remove it from connection cache.
-   * Note: multiple entries may share the same connection, all those entries are removed.
-   */
-  def closeConnection(connConf: GeodeConnectionConf): Unit = {
-    val conns = connConf.locators.map(connections withDefaultValue null).filter(_ != null)
-    if (conns.nonEmpty) connections.synchronized {
-      conns(0).close()
-      connections.retain((k,v) => v != conns(0))
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/LocatorHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/LocatorHelper.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/LocatorHelper.scala
deleted file mode 100644
index 71fed52..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/LocatorHelper.scala
+++ /dev/null
@@ -1,135 +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 io.pivotal.geode.spark.connector.internal
-
-import java.net.InetSocketAddress
-import java.util.{ArrayList => JArrayList}
-
-import com.gemstone.gemfire.cache.client.internal.locator.{GetAllServersResponse, GetAllServersRequest}
-import com.gemstone.gemfire.distributed.internal.ServerLocation
-import com.gemstone.gemfire.distributed.internal.tcpserver.TcpClient
-import org.apache.spark.Logging
-
-import scala.util.{Failure, Success, Try}
-
-
-object LocatorHelper extends Logging {
-
-  /** valid locator strings are: host[port] and host:port */
-  final val LocatorPattern1 = """([\w-_]+(\.[\w-_]+)*)\[([0-9]{2,5})\]""".r
-  final val LocatorPattern2 = """([\w-_]+(\.[\w-_]+)*):([0-9]{2,5})""".r
-
-  /** convert single locator string to Try[(host, port)] */
-  def locatorStr2HostPortPair(locatorStr: String): Try[(String, Int)] =
-    locatorStr match {
-      case LocatorPattern1(host, domain, port) => Success((host, port.toInt))
-      case LocatorPattern2(host, domain, port) => Success((host, port.toInt))
-      case _ => Failure(new Exception(s"invalid locator: $locatorStr"))
-    }
-
-  /** 
-   * Parse locator strings and returns Seq of (hostname, port) pair. 
-   * Valid locator string are one or more "host[port]" and/or "host:port"
-   * separated by `,`. For example:
-   *    host1.mydomain.com[8888],host2.mydomain.com[8889] 
-   *    host1.mydomain.com:8888,host2.mydomain.com:8889 
-   */
-  def parseLocatorsString(locatorsStr: String): Seq[(String, Int)] =
-    locatorsStr.split(",").map(locatorStr2HostPortPair).map(_.get)
-
-
-  /**
-   * Return the list of live Geode servers for the given locators.
-   * @param locators locators for the given Geode cluster
-   * @param serverGroup optional server group name, default is "" (empty string)
-   */
-  def getAllGeodeServers(locators: Seq[(String, Int)], serverGroup: String = ""): Option[Seq[(String, Int)]] = {
-    var result: Option[Seq[(String, Int)]] = None
-    locators.find { case (host, port) =>
-      try {
-        val addr = new InetSocketAddress(host, port)
-        val req = new GetAllServersRequest(serverGroup)
-        val res = TcpClient.requestToServer(addr.getAddress, addr.getPort, req, 2000)
-        if (res != null) {
-          import scala.collection.JavaConverters._
-          val servers = res.asInstanceOf[GetAllServersResponse].getServers.asInstanceOf[JArrayList[ServerLocation]]
-          if (servers.size > 0)
-            result = Some(servers.asScala.map(e => (e.getHostName, e.getPort)))
-        }
-      } catch { case e: Exception => logWarning("getAllGeodeServers error", e)
-      }
-      result.isDefined
-    }
-    result
-  }
-
-  /**
-   * Pick up at most 3 preferred servers from all available servers based on
-   * host name and Spark executor id.
-   *
-   * This method is used by DefaultGeodeConnection to create ClientCache. Usually
-   * one server is enough to initialize ClientCacheFactory, but this provides two
-   * backup servers in case of the 1st server can't be connected.
-   *   
-   * @param servers all available servers in the form of (hostname, port) pairs
-   * @param hostName the host name of the Spark executor
-   * @param executorId the Spark executor Id, such as "<driver>", "0", "1", ...
-   * @return Seq[(hostname, port)] of preferred servers
-   */
-  def pickPreferredGeodeServers(
-    servers: Seq[(String, Int)], hostName: String, executorId: String): Seq[(String, Int)] = {
-
-    // pick up `length` items form the Seq starts at the `start` position.
-    //  The Seq is treated as a ring, so at most `Seq.size` items can be picked
-    def circularTake[T](seq: Seq[T], start: Int, length: Int): Seq[T] = {
-      val size = math.min(seq.size, length)
-      (start until start + size).map(x => seq(x % seq.size))
-    }
-
-    // map executor id to int: "<driver>" (or non-number string) to 0, and "n" to n + 1
-    val id = try { executorId.toInt + 1 } catch { case e: NumberFormatException => 0 }
-    
-    // algorithm: 
-    // 1. sort server list
-    // 2. split sorted server list into 3 sub-lists a, b, and c:
-    //      list-a: servers on the given host
-    //      list-b: servers that are in front of list-a on the sorted server list
-    //      list-c: servers that are behind list-a on the sorted server list
-    //    then rotate list-a based on executor id, then create new server list:
-    //      modified list-a ++ list-c ++ list-b
-    // 3. if there's no server on the given host, then create new server list
-    //    by rotating sorted server list based on executor id.
-    // 4. take up to 3 servers from the new server list
-    val sortedServers = servers.sorted
-    val firstIdx = sortedServers.indexWhere(p => p._1 == hostName)
-    val lastIdx = if (firstIdx < 0) -1 else sortedServers.lastIndexWhere(p => p._1 == hostName)
-
-    if (firstIdx < 0) { // no local server
-      circularTake(sortedServers, id, 3)
-    } else {
-      val (seq1, seq2) = sortedServers.splitAt(firstIdx)
-      val seq = if (firstIdx == lastIdx) {  // one local server
-        seq2 ++ seq1
-      } else { // multiple local server
-        val (seq3, seq4) = seq2.splitAt(lastIdx - firstIdx + 1)
-        val seq3b = if (id % seq3.size == 0) seq3 else circularTake(seq3, id, seq3.size)
-        seq3b ++ seq4 ++ seq1
-      }
-      circularTake(seq, 0, 3)
-    }
-  }  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/geodefunctions/StructStreamingResultCollector.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/geodefunctions/StructStreamingResultCollector.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/geodefunctions/StructStreamingResultCollector.scala
deleted file mode 100644
index a8666fc..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/geodefunctions/StructStreamingResultCollector.scala
+++ /dev/null
@@ -1,152 +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 io.pivotal.geode.spark.connector.internal.geodefunctions
-
-import java.util.concurrent.{TimeUnit, LinkedBlockingQueue, BlockingQueue}
-import com.gemstone.gemfire.DataSerializer
-import com.gemstone.gemfire.cache.execute.ResultCollector
-import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl
-import com.gemstone.gemfire.cache.query.types.StructType
-import com.gemstone.gemfire.distributed.DistributedMember
-import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput}
-import io.pivotal.geode.spark.connector.internal.geodefunctions.StructStreamingResultSender.
-       {TYPE_CHUNK, DATA_CHUNK, ERROR_CHUNK, SER_DATA, UNSER_DATA, BYTEARR_DATA}
-
-/**
- * StructStreamingResultCollector and StructStreamingResultSender are paired
- * to transfer result of list of `com.gemstone.gemfire.cache.query.Struct`
- * from Geode server to Spark Connector (the client of Geode server)
- * in streaming, i.e., while sender sending the result, the collector can
- * start processing the arrived result without waiting for full result to
- * become available.
- */
-class StructStreamingResultCollector(desc: String) extends ResultCollector[Array[Byte], Iterator[Array[Object]]] {
-
-  /** the constructor that provide default `desc` (description) */
-  def this() = this("StructStreamingResultCollector")
-  
-  private val queue: BlockingQueue[Array[Byte]] = new LinkedBlockingQueue[Array[Byte]]()
-  var structType: StructType = null
-
-  /** ------------------------------------------ */
-  /**  ResultCollector interface implementations */
-  /** ------------------------------------------ */
-  
-  override def getResult: Iterator[Array[Object]] = resultIterator
-
-  override def getResult(timeout: Long, unit: TimeUnit): Iterator[Array[Object]] = 
-    throw new UnsupportedOperationException()
-
-  /** addResult add non-empty byte array (chunk) to the queue */
-  override def addResult(memberID: DistributedMember, chunk: Array[Byte]): Unit = 
-    if (chunk != null && chunk.size > 1) {
-      this.queue.add(chunk)
-      // println(s"""$desc receive from $memberID: ${chunk.mkString(" ")}""")
-    }
-
-  /** endResults add special `Array.empty` to the queue as marker of end of data */
-  override def endResults(): Unit = this.queue.add(Array.empty)
-  
-  override def clearResults(): Unit = this.queue.clear()
-
-  /** ------------------------------------------ */
-  /**             Internal methods               */
-  /** ------------------------------------------ */
-
-  def getResultType: StructType = {
-    // trigger lazy resultIterator initialization if necessary
-    if (structType == null)  resultIterator.hasNext
-    structType        
-  }
-
-  /**
-   * Note: The data is sent in chunks, and each chunk contains multiple 
-   * records. So the result iterator is an iterator (I) of iterator (II),
-   * i.e., go through each chunk (iterator (I)), and for each chunk, go 
-   * through each record (iterator (II)). 
-   */
-  private lazy val resultIterator = new Iterator[Array[Object]] {
-
-    private var currentIterator: Iterator[Array[Object]] = nextIterator()
-    
-    override def hasNext: Boolean = {
-      if (!currentIterator.hasNext && currentIterator != Iterator.empty) currentIterator = nextIterator()
-      currentIterator.hasNext
-    }
-
-    /** Note: make sure call `hasNext` first to adjust `currentIterator` */
-    override def next(): Array[Object] = currentIterator.next()
-  }
-  
-  /** get the iterator for the next chunk of data */
-  private def nextIterator(): Iterator[Array[Object]] = {
-    val chunk: Array[Byte] = queue.take
-    if (chunk.isEmpty) {
-      Iterator.empty
-    } else {
-      val input = new ByteArrayDataInput()
-      input.initialize(chunk, Version.CURRENT)
-      val chunkType = input.readByte()
-      // println(s"chunk type $chunkType")
-      chunkType match {
-        case TYPE_CHUNK =>
-          if (structType == null)
-            structType = DataSerializer.readObject(input).asInstanceOf[StructTypeImpl]
-          nextIterator()
-        case DATA_CHUNK =>
-          // require(structType != null && structType.getFieldNames.length > 0)
-          if (structType == null) structType = StructStreamingResultSender.KeyValueType
-          chunkToIterator(input, structType.getFieldNames.length)
-        case ERROR_CHUNK => 
-          val error = DataSerializer.readObject(input).asInstanceOf[Exception]
-          errorPropagationIterator(error)
-        case _ => throw new RuntimeException(s"unknown chunk type: $chunkType")
-      }
-    }
-  }
-
-  /** create a iterator that propagate sender's exception */
-  private def errorPropagationIterator(ex: Exception) = new Iterator[Array[Object]] {
-    val re = new RuntimeException(ex)
-    override def hasNext: Boolean = throw re
-    override def next(): Array[Object] = throw re
-  }
-  
-  /** convert a chunk of data to an iterator */
-  private def chunkToIterator(input: ByteArrayDataInput, rowSize: Int) = new Iterator[Array[Object]] {
-    override def hasNext: Boolean = input.available() > 0
-    val tmpInput = new ByteArrayDataInput()
-    override def next(): Array[Object] = 
-      (0 until rowSize).map { ignore =>
-        val b = input.readByte()
-        b match {
-          case SER_DATA => 
-            val arr: Array[Byte] = DataSerializer.readByteArray(input)
-            tmpInput.initialize(arr, Version.CURRENT)
-            DataSerializer.readObject(tmpInput).asInstanceOf[Object]
-          case UNSER_DATA =>
-            DataSerializer.readObject(input).asInstanceOf[Object]
-          case BYTEARR_DATA =>
-            DataSerializer.readByteArray(input).asInstanceOf[Object]
-          case _ => 
-            throw new RuntimeException(s"unknown data type $b")
-        }
-      }.toArray
-  }
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParser.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParser.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParser.scala
deleted file mode 100644
index 3f6dfad..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParser.scala
+++ /dev/null
@@ -1,58 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import scala.util.parsing.combinator.RegexParsers
-
-class QueryParser extends RegexParsers {
-
-  def query: Parser[String] = opt(rep(IMPORT ~ PACKAGE)) ~> select ~> opt(distinct) ~> projection ~> from ~> regions <~ opt(where ~ filter) ^^ {
-    _.toString
-  }
-
-  val IMPORT: Parser[String] = "[Ii][Mm][Pp][Oo][Rr][Tt]".r
-
-  val select: Parser[String] = "[Ss][Ee][Ll][Ee][Cc][Tt]".r
-
-  val distinct: Parser[String] = "[Dd][Ii][Ss][Tt][Ii][Nn][Cc][Tt]".r
-
-  val from: Parser[String] = "[Ff][Rr][Oo][Mm]".r
-
-  val where: Parser[String] = "[Ww][Hh][Ee][Rr][Ee]".r
-
-  def PACKAGE: Parser[String] = """[\w.]+""".r
-
-  def projection: Parser[String] = "*" | repsep("""["\w]+[.\w"]*""".r, ",") ^^ {
-    _.toString
-  }
-
-  def regions: Parser[String] = repsep(region <~ opt(alias), ",") ^^ {
-    _.toString
-  }
-
-  def region: Parser[String] = """/[\w.]+[/[\w.]+]*""".r | """[\w]+[.\w]*""".r
-
-  def alias: Parser[String] = not(where) ~> """[\w]+""".r
-
-  def filter: Parser[String] = """[\w.]+[[\s]+[<>=.'\w]+]*""".r
-}
-
-object QueryParser extends QueryParser {
-
-  def parseOQL(expression: String) = parseAll(query, expression)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryRDD.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryRDD.scala
deleted file mode 100644
index 474aa6a..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryRDD.scala
+++ /dev/null
@@ -1,83 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodeRDDPartition, ServerSplitsPartitioner}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.{TaskContext, SparkContext, Partition}
-import scala.reflect.ClassTag
-
-/**
- * An RDD that provides the functionality that read the OQL query result
- *
- * @param sc The SparkContext this RDD is associated with
- * @param queryString The OQL query string
- * @param connConf The GeodeConnectionConf that provide the GeodeConnection
- */
-class QueryRDD[T](@transient sc: SparkContext,
-                  queryString: String,
-                  connConf: GeodeConnectionConf)
-                 (implicit ct: ClassTag[T])
-  extends RDD[T](sc, Seq.empty) {
-
-  override def getPartitions: Array[Partition] = {
-    val conn = connConf.getConnection
-    val regionPath = getRegionPathFromQuery(queryString)
-    val md = conn.getRegionMetadata(regionPath)
-    md match {
-      case Some(metadata) =>
-        if (metadata.isPartitioned) {
-          val splits = ServerSplitsPartitioner.partitions(conn, metadata, Map.empty)
-          logInfo(s"QueryRDD.getPartitions():isPartitioned=true, partitions=${splits.mkString(",")}")
-          splits
-        }
-        else {
-          logInfo(s"QueryRDD.getPartitions():isPartitioned=false")
-          Array[Partition](new GeodeRDDPartition(0, Set.empty))
-
-        }
-      case None => throw new RuntimeException(s"Region $regionPath metadata was not found.")
-    }
-  }
-
-  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
-    val buckets = split.asInstanceOf[GeodeRDDPartition].bucketSet
-    val regionPath = getRegionPathFromQuery(queryString)
-    val result = connConf.getConnection.executeQuery(regionPath, buckets, queryString)
-    result match {
-      case it: Iterator[T] =>
-        logInfo(s"QueryRDD.compute():query=$queryString, partition=$split")
-        it
-      case _ =>
-        throw new RuntimeException("Unexpected OQL result: " + result.toString)
-    }
-  }
-
-  private def getRegionPathFromQuery(queryString: String): String = {
-    val r = QueryParser.parseOQL(queryString).get
-    r match {
-      case r: String =>
-        val start = r.indexOf("/") + 1
-        var end = r.indexOf(")")
-        if (r.indexOf(".") > 0) end = math.min(r.indexOf("."), end)
-        if (r.indexOf(",") > 0) end = math.min(r.indexOf(","), end)
-        val regionPath = r.substring(start, end)
-        regionPath
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryResultCollector.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryResultCollector.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryResultCollector.scala
deleted file mode 100644
index bedc58d..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/QueryResultCollector.scala
+++ /dev/null
@@ -1,69 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import java.util.concurrent.{TimeUnit, LinkedBlockingDeque}
-
-import com.gemstone.gemfire.DataSerializer
-import com.gemstone.gemfire.cache.execute.ResultCollector
-import com.gemstone.gemfire.distributed.DistributedMember
-import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput}
-
-class QueryResultCollector extends ResultCollector[Array[Byte], Iterator[Object]]{
-
-  private val queue = new LinkedBlockingDeque[Array[Byte]]()
-
-  override def getResult = resultIterator
-
-  override def getResult(timeout: Long, unit: TimeUnit) = throw new UnsupportedOperationException
-
-  override def addResult(memberID: DistributedMember , chunk: Array[Byte]) =
-    if (chunk != null && chunk.size > 0) {
-      queue.add(chunk)
-    }
-
-  override def endResults = queue.add(Array.empty)
-
-
-  override def clearResults = queue.clear
-
-  private lazy val resultIterator = new Iterator[Object] {
-    private var currentIterator = nextIterator
-    def hasNext = {
-      if (!currentIterator.hasNext && currentIterator != Iterator.empty)
-        currentIterator = nextIterator
-      currentIterator.hasNext
-    }
-    def next = currentIterator.next
-  }
-
-  private def nextIterator: Iterator[Object] = {
-    val chunk = queue.take
-    if (chunk.isEmpty) {
-      Iterator.empty
-    }
-    else {
-      val input = new ByteArrayDataInput
-      input.initialize(chunk, Version.CURRENT)
-      new Iterator[Object] {
-        override def hasNext: Boolean = input.available() > 0
-        override def next: Object = DataSerializer.readObject(input).asInstanceOf[Object]
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RDDConverter.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RDDConverter.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RDDConverter.scala
deleted file mode 100644
index 6a1611c..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RDDConverter.scala
+++ /dev/null
@@ -1,40 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{DataFrame, SQLContext}
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.sources.{BaseRelation, TableScan}
-
-import scala.tools.nsc.backend.icode.analysis.DataFlowAnalysis
-
-case class OQLRelation[T](queryRDD: QueryRDD[T])(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan {
-
-  override def schema: StructType = new SchemaBuilder(queryRDD).toSparkSchema()
-
-  override def buildScan(): RDD[Row] = new RowBuilder(queryRDD).toRowRDD()
-
-}
-
-object RDDConverter {
-
-  def queryRDDToDataFrame[T](queryRDD: QueryRDD[T], sqlContext: SQLContext): DataFrame = {
-    sqlContext.baseRelationToDataFrame(OQLRelation(queryRDD)(sqlContext))
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RowBuilder.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RowBuilder.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RowBuilder.scala
deleted file mode 100644
index e54411c..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/RowBuilder.scala
+++ /dev/null
@@ -1,38 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import com.gemstone.gemfire.cache.query.internal.StructImpl
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-
-class RowBuilder[T](queryRDD: QueryRDD[T]) {
-
-  /**
-   * Convert QueryRDD to RDD of Row
-   * @return RDD of Rows
-   */
-  def toRowRDD(): RDD[Row] = {
-    val rowRDD = queryRDD.map(row => {
-      row match {
-        case si: StructImpl => Row.fromSeq(si.getFieldValues)
-        case obj: Object => Row.fromSeq(Seq(obj))
-      }
-    })
-    rowRDD
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/SchemaBuilder.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/SchemaBuilder.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/SchemaBuilder.scala
deleted file mode 100644
index 3ca20b7..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/SchemaBuilder.scala
+++ /dev/null
@@ -1,73 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import com.gemstone.gemfire.cache.query.internal.StructImpl
-import org.apache.spark.sql.types._
-import scala.collection.mutable.ListBuffer
-import org.apache.spark.Logging
-
-class SchemaBuilder[T](queryRDD: QueryRDD[T]) extends Logging {
-
-  val nullStructType = StructType(Nil)
-  
-  val typeMap:Map[Class[_], DataType] = Map( 
-    (classOf[java.lang.String], StringType),
-    (classOf[java.lang.Integer], IntegerType),
-    (classOf[java.lang.Short], ShortType),
-    (classOf[java.lang.Long], LongType),
-    (classOf[java.lang.Double], DoubleType),
-    (classOf[java.lang.Float], FloatType),
-    (classOf[java.lang.Boolean], BooleanType),
-    (classOf[java.lang.Byte], ByteType),
-    (classOf[java.util.Date], DateType),
-    (classOf[java.lang.Object], nullStructType)
-  )
-  
-  /**
-   * Analyse QueryRDD to get the Spark schema
-   * @return The schema represented by Spark StructType
-   */
-  def toSparkSchema(): StructType = {
-    val row = queryRDD.first()
-    val tpe = row match {
-      case r: StructImpl => constructFromStruct(r)
-      case null => StructType(StructField("col1", NullType) :: Nil)
-      case default => 
-        val value = typeMap.getOrElse(default.getClass(), nullStructType)
-        StructType(StructField("col1", value) :: Nil)
-    }
-    logInfo(s"Schema: $tpe")
-    tpe
-  }
-  
-  def constructFromStruct(r:StructImpl) = {
-    val names = r.getFieldNames
-    val values = r.getFieldValues
-    val lb = new ListBuffer[StructField]()
-    for (i <- 0 until names.length) {
-      val name = names(i)
-      val value = values(i)
-      val dataType = value match {
-        case null => NullType
-        case default => typeMap.getOrElse(default.getClass,  nullStructType)
-      }
-      lb += StructField(name, dataType)
-    }
-    StructType(lb.toSeq)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/UndefinedSerializer.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/UndefinedSerializer.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/UndefinedSerializer.scala
deleted file mode 100644
index 37dec42..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/oql/UndefinedSerializer.scala
+++ /dev/null
@@ -1,46 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import com.esotericsoftware.kryo.{Kryo, Serializer}
-import com.esotericsoftware.kryo.io.{Output, Input}
-import com.gemstone.gemfire.cache.query.QueryService
-import com.gemstone.gemfire.cache.query.internal.Undefined
-
-/**
- * This is the customized serializer to serialize QueryService.UNDEFINED,
- * i.e. com.gemstone.gemfire.cache.query.internal.Undefined, in order to
- * guarantee the singleton Undefined after its deserialization within Spark.
- */
-class UndefinedSerializer extends Serializer[Undefined] {
-
-  def write(kryo: Kryo, output: Output, u: Undefined) {
-    //Only serialize a byte for Undefined
-    output.writeByte(u.getDSFID)
-  }
-
-  def read (kryo: Kryo, input: Input, tpe: Class[Undefined]): Undefined = {
-    //Read DSFID of Undefined
-    input.readByte()
-    QueryService.UNDEFINED match {
-      case null => new Undefined
-      case _ =>
-        //Avoid calling Undefined constructor again.
-        QueryService.UNDEFINED.asInstanceOf[Undefined]
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeJoinRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeJoinRDD.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeJoinRDD.scala
deleted file mode 100644
index e9dd658..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeJoinRDD.scala
+++ /dev/null
@@ -1,67 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.GeodeConnectionConf
-import org.apache.spark.{TaskContext, Partition}
-import org.apache.spark.rdd.RDD
-import scala.collection.JavaConversions._
-
-/**
- * An `RDD[T, V]` that will represent the result of a join between `left` RDD[T]
- * and the specified Geode Region[K, V].
- */
-class GeodeJoinRDD[T, K, V] private[connector]
-  ( left: RDD[T],
-    func: T => K,
-    val regionPath: String,
-    val connConf: GeodeConnectionConf
-  ) extends RDD[(T, V)](left.context, left.dependencies) {
-
-  /** validate region existence when GeodeRDD object is created */
-  validate()
-
-  /** Validate region, and make sure it exists. */
-  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
-
-  override protected def getPartitions: Array[Partition] = left.partitions
-
-  override def compute(split: Partition, context: TaskContext): Iterator[(T, V)] = {
-    val region = connConf.getConnection.getRegionProxy[K, V](regionPath)
-    if (func == null) computeWithoutFunc(split, context, region)
-    else computeWithFunc(split, context, region)
-  }
-
-  /** T is (K, V1) since there's no map function `func` */
-  private def computeWithoutFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, V)] = {
-    val leftPairs = left.iterator(split, context).toList.asInstanceOf[List[(K, _)]]
-    val leftKeys = leftPairs.map { case (k, v) => k}.toSet
-    // Note: get all will return (key, null) for non-exist entry, so remove those entries
-    val rightPairs = region.getAll(leftKeys).filter { case (k, v) => v != null}
-    leftPairs.filter{case (k, v) => rightPairs.contains(k)}
-             .map {case (k, v) => ((k, v).asInstanceOf[T], rightPairs.get(k).get)}.toIterator
-  }
-  
-  private def computeWithFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, V)] = {
-    val leftPairs = left.iterator(split, context).toList.map(t => (t, func(t)))
-    val leftKeys = leftPairs.map { case (t, k) => k}.toSet
-    // Note: get all will return (key, null) for non-exist entry, so remove those entries
-    val rightPairs = region.getAll(leftKeys).filter { case (k, v) => v != null}
-    leftPairs.filter { case (t, k) => rightPairs.contains(k)}.map {case (t, k) => (t, rightPairs.get(k).get)}.toIterator
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeOuterJoinRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeOuterJoinRDD.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeOuterJoinRDD.scala
deleted file mode 100644
index 3d61d47..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeOuterJoinRDD.scala
+++ /dev/null
@@ -1,69 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.GeodeConnectionConf
-import org.apache.spark.{TaskContext, Partition}
-import org.apache.spark.rdd.RDD
-import scala.collection.JavaConversions._
-
-/**
- * An `RDD[ T, Option[V] ]` that represents the result of a left outer join 
- * between `left` RDD[T] and the specified Geode Region[K, V].
- */
-class GeodeOuterJoinRDD[T, K, V] private[connector]
- ( left: RDD[T],
-   func: T => K,
-   val regionPath: String,
-   val connConf: GeodeConnectionConf
-  ) extends RDD[(T, Option[V])](left.context, left.dependencies) {
-
-  /** validate region existence when GeodeRDD object is created */
-  validate()
-
-  /** Validate region, and make sure it exists. */
-  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
-
-  override protected def getPartitions: Array[Partition] = left.partitions
-
-  override def compute(split: Partition, context: TaskContext): Iterator[(T, Option[V])] = {
-    val region = connConf.getConnection.getRegionProxy[K, V](regionPath)
-    if (func == null) computeWithoutFunc(split, context, region)
-    else computeWithFunc(split, context, region)
-  }
-
-  /** T is (K1, V1), and K1 and K are the same type since there's no map function `func` */
-  private def computeWithoutFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, Option[V])] = {
-    val leftPairs = left.iterator(split, context).toList.asInstanceOf[List[(K, _)]]
-    val leftKeys = leftPairs.map { case (k, v) => k}.toSet
-    // Note: get all will return (key, null) for non-exist entry
-    val rightPairs = region.getAll(leftKeys)
-    // rightPairs is a java.util.Map, not scala map, so need to convert map.get() to Option
-    leftPairs.map{ case (k, v) => ((k, v).asInstanceOf[T], Option(rightPairs.get(k))) }.toIterator
-  }
-
-  private def computeWithFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, Option[V])] = {
-    val leftPairs = left.iterator(split, context).toList.map(t => (t, func(t)))
-    val leftKeys = leftPairs.map { case (t, k) => k}.toSet
-    // Note: get all will return (key, null) for non-exist entry
-    val rightPairs = region.getAll(leftKeys)
-    // rightPairs is a java.util.Map, not scala map, so need to convert map.get() to Option
-    leftPairs.map{ case (t, k) => (t, Option(rightPairs.get(k)))}.toIterator
-  }
-}
-



[07/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSQLContextFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSQLContextFunctions.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSQLContextFunctions.scala
new file mode 100644
index 0000000..51f1d01
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSQLContextFunctions.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.pivotal.gemfire.spark.connector
+
+import io.pivotal.gemfire.spark.connector.internal.oql.{OQLRelation, QueryRDD}
+import org.apache.spark.Logging
+import org.apache.spark.sql.{DataFrame, SQLContext}
+
+/**
+ * Provide GemFire OQL specific functions
+ */
+class GemFireSQLContextFunctions(@transient sqlContext: SQLContext) extends Serializable with Logging {
+
+  /**
+   * Expose a GemFire OQL query result as a DataFrame
+   * @param query the OQL query string.
+   */
+  def gemfireOQL(
+    query: String,
+    connConf: GemFireConnectionConf = GemFireConnectionConf(sqlContext.sparkContext.getConf)): DataFrame = {
+    logInfo(s"OQL query = $query")
+    val rdd = new QueryRDD[Object](sqlContext.sparkContext, query, connConf)
+    sqlContext.baseRelationToDataFrame(OQLRelation(rdd)(sqlContext))
+  }
+
+  private[connector] def defaultConnectionConf: GemFireConnectionConf =
+    GemFireConnectionConf(sqlContext.sparkContext.getConf)
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSparkContextFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSparkContextFunctions.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSparkContextFunctions.scala
new file mode 100644
index 0000000..5341977
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireSparkContextFunctions.scala
@@ -0,0 +1,39 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRegionRDD
+import org.apache.spark.SparkContext
+
+import scala.reflect.ClassTag
+
+/** Provides GemFire specific methods on `SparkContext` */
+class GemFireSparkContextFunctions(@transient sc: SparkContext) extends Serializable {
+
+  /**
+   * Expose a GemFire region as a GemFireRDD
+   * @param regionPath the full path of the region
+   * @param connConf the GemFireConnectionConf that can be used to access the region
+   * @param opConf use this to specify preferred partitioner
+   *        and its parameters. The implementation will use it if it's applicable
+   */
+  def gemfireRegion[K: ClassTag, V: ClassTag] (
+    regionPath: String, connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf),
+    opConf: Map[String, String] = Map.empty): GemFireRegionRDD[K, V] =
+    GemFireRegionRDD[K, V](sc, regionPath, connConf, opConf)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
new file mode 100644
index 0000000..7d147b2
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
@@ -0,0 +1,164 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal
+
+import java.net.InetAddress
+
+import com.gemstone.gemfire.cache.client.{ClientCache, ClientCacheFactory, ClientRegionShortcut}
+import com.gemstone.gemfire.cache.execute.{FunctionException, FunctionService}
+import com.gemstone.gemfire.cache.query.Query
+import com.gemstone.gemfire.cache.{Region, RegionService}
+import com.gemstone.gemfire.internal.cache.execute.InternalExecution
+import io.pivotal.gemfire.spark.connector.internal.oql.QueryResultCollector
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartition
+import org.apache.spark.{SparkEnv, Logging}
+import io.pivotal.gemfire.spark.connector.GemFireConnection
+import io.pivotal.gemfire.spark.connector.internal.gemfirefunctions._
+import java.util.{Set => JSet, List => JList }
+
+/**
+ * Default GemFireConnection implementation. The instance of this should be
+ * created by DefaultGemFireConnectionFactory
+ * @param locators pairs of host/port of locators
+ * @param gemFireProps The initial gemfire properties to be used.
+ */
+private[connector] class DefaultGemFireConnection (
+  locators: Seq[(String, Int)], gemFireProps: Map[String, String] = Map.empty) 
+  extends GemFireConnection with Logging {
+
+  private val clientCache = initClientCache()
+
+  /** Register GemFire functions to the GemFire cluster */
+  FunctionService.registerFunction(RetrieveRegionMetadataFunction.getInstance())
+  FunctionService.registerFunction(RetrieveRegionFunction.getInstance())
+
+  private def initClientCache() : ClientCache = {
+    try {
+      val ccf = getClientCacheFactory
+      ccf.create()
+    } catch {
+      case e: Exception =>
+        logError(s"""Failed to init ClientCache, locators=${locators.mkString(",")}, Error: $e""")
+        throw new RuntimeException(e)
+    }
+  }
+  
+  private def getClientCacheFactory: ClientCacheFactory = {
+    import io.pivotal.gemfire.spark.connector.map2Properties
+    val ccf = new ClientCacheFactory(gemFireProps)
+    ccf.setPoolReadTimeout(30000)
+    val servers = LocatorHelper.getAllGemFireServers(locators)
+    if (servers.isDefined && servers.get.size > 0) {
+      val sparkIp = System.getenv("SPARK_LOCAL_IP")
+      val hostName = if (sparkIp != null) InetAddress.getByName(sparkIp).getCanonicalHostName
+                     else InetAddress.getLocalHost.getCanonicalHostName
+      val executorId = SparkEnv.get.executorId      
+      val pickedServers = LocatorHelper.pickPreferredGemFireServers(servers.get, hostName, executorId)
+      logInfo(s"""Init ClientCache: severs=${pickedServers.mkString(",")}, host=$hostName executor=$executorId props=$gemFireProps""")
+      logDebug(s"""Init ClientCache: all-severs=${pickedServers.mkString(",")}""")
+      pickedServers.foreach{ case (host, port)  => ccf.addPoolServer(host, port) }
+    } else {
+      logInfo(s"""Init ClientCache: locators=${locators.mkString(",")}, props=$gemFireProps""")
+      locators.foreach { case (host, port)  => ccf.addPoolLocator(host, port) }
+    }
+    ccf
+  }
+
+  /** close the clientCache */
+  override def close(): Unit =
+    if (! clientCache.isClosed) clientCache.close()
+
+  /** ----------------------------------------- */
+  /** implementation of GemFireConnection trait */
+  /** ----------------------------------------- */
+
+  override def getQuery(queryString: String): Query =
+    clientCache.asInstanceOf[RegionService].getQueryService.newQuery(queryString)
+
+  override def validateRegion[K, V](regionPath: String): Unit = {
+    val md = getRegionMetadata[K, V](regionPath)
+    if (! md.isDefined) throw new RuntimeException(s"The region named $regionPath was not found")
+  }
+
+  def getRegionMetadata[K, V](regionPath: String): Option[RegionMetadata] = {
+    import scala.collection.JavaConversions.setAsJavaSet
+    val region = getRegionProxy[K, V](regionPath)
+    val set0: JSet[Integer] = Set[Integer](0)
+    val exec = FunctionService.onRegion(region).asInstanceOf[InternalExecution].withBucketFilter(set0)
+    exec.setWaitOnExceptionFlag(true)
+    try {
+      val collector = exec.execute(RetrieveRegionMetadataFunction.ID)
+      val r = collector.getResult.asInstanceOf[JList[RegionMetadata]]
+      logDebug(r.get(0).toString)
+      Some(r.get(0))
+    } catch {
+      case e: FunctionException => 
+        if (e.getMessage.contains(s"The region named /$regionPath was not found")) None
+        else throw e
+    }
+  }
+
+  def getRegionProxy[K, V](regionPath: String): Region[K, V] = {
+    val region1: Region[K, V] = clientCache.getRegion(regionPath).asInstanceOf[Region[K, V]]
+    if (region1 != null) region1
+    else DefaultGemFireConnection.regionLock.synchronized {
+      val region2 = clientCache.getRegion(regionPath).asInstanceOf[Region[K, V]]
+      if (region2 != null) region2
+      else clientCache.createClientRegionFactory[K, V](ClientRegionShortcut.PROXY).create(regionPath)
+    }
+  }
+
+  override def getRegionData[K, V](regionPath: String, whereClause: Option[String], split: GemFireRDDPartition): Iterator[(K, V)] = {
+    val region = getRegionProxy[K, V](regionPath)
+    val desc = s"""RDD($regionPath, "${whereClause.getOrElse("")}", ${split.index})"""
+    val args : Array[String] = Array[String](whereClause.getOrElse(""), desc)
+    val collector = new StructStreamingResultCollector(desc)
+        // RetrieveRegionResultCollector[(K, V)]
+    import scala.collection.JavaConversions.setAsJavaSet
+    val exec = FunctionService.onRegion(region).withArgs(args).withCollector(collector).asInstanceOf[InternalExecution]
+      .withBucketFilter(split.bucketSet.map(Integer.valueOf))
+    exec.setWaitOnExceptionFlag(true)
+    exec.execute(RetrieveRegionFunction.ID)
+    collector.getResult.map{objs: Array[Object] => (objs(0).asInstanceOf[K], objs(1).asInstanceOf[V])}
+  }
+
+  override def executeQuery(regionPath: String, bucketSet: Set[Int], queryString: String) = {
+    import scala.collection.JavaConversions.setAsJavaSet
+    FunctionService.registerFunction(QueryFunction.getInstance())
+    val collector = new QueryResultCollector
+    val region = getRegionProxy(regionPath)
+    val args: Array[String] = Array[String](queryString, bucketSet.toString)
+    val exec = FunctionService.onRegion(region).withCollector(collector).asInstanceOf[InternalExecution]
+      .withBucketFilter(bucketSet.map(Integer.valueOf))
+      .withArgs(args)
+    exec.execute(QueryFunction.ID)
+    collector.getResult
+  }
+}
+
+private[connector] object DefaultGemFireConnection {
+  /** a lock object only used by getRegionProxy...() */
+  private val regionLock = new Object
+}
+
+/** The purpose of this class is making unit test DefaultGemFireConnectionManager easier */
+class DefaultGemFireConnectionFactory {
+
+  def newConnection(locators: Seq[(String, Int)], gemFireProps: Map[String, String] = Map.empty) =
+    new DefaultGemFireConnection(locators, gemFireProps)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManager.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManager.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManager.scala
new file mode 100644
index 0000000..6722ca8
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManager.scala
@@ -0,0 +1,77 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal
+
+import io.pivotal.gemfire.spark.connector.{GemFireConnection, GemFireConnectionConf, GemFireConnectionManager}
+
+import scala.collection.mutable
+
+/**
+ * Default implementation of GemFireConnectionFactory
+ */
+class DefaultGemFireConnectionManager extends GemFireConnectionManager {
+
+  def getConnection(connConf: GemFireConnectionConf): GemFireConnection =
+    DefaultGemFireConnectionManager.getConnection(connConf)
+
+  def closeConnection(connConf: GemFireConnectionConf): Unit =
+    DefaultGemFireConnectionManager.closeConnection(connConf)
+
+}
+
+object DefaultGemFireConnectionManager  {
+
+  /** connection cache, keyed by host:port pair */
+  private[connector] val connections = mutable.Map[(String, Int), GemFireConnection]()
+
+  /**
+   * use locator host:port pair to lookup cached connection. create new connection 
+   * and add it to the cache `connections` if it does not exist.
+   */
+  def getConnection(connConf: GemFireConnectionConf)
+    (implicit factory: DefaultGemFireConnectionFactory = new DefaultGemFireConnectionFactory): GemFireConnection = {
+
+    def getCachedConnection(locators: Seq[(String, Int)]): GemFireConnection = {
+      val conns = connConf.locators.map(connections withDefaultValue null).filter(_ != null)
+      if (conns.nonEmpty) conns(0) else null
+    }
+
+    val conn1 = getCachedConnection(connConf.locators)
+    if (conn1 != null) conn1
+    else connections.synchronized {
+      val conn2 = getCachedConnection(connConf.locators)
+      if (conn2 != null) conn2
+      else {
+        val conn3 = factory.newConnection(connConf.locators, connConf.gemfireProps)
+        connConf.locators.foreach(pair => connections += (pair -> conn3))
+        conn3
+      }
+    }
+  }
+
+  /**
+   * Close the connection and remove it from connection cache.
+   * Note: multiple entries may share the same connection, all those entries are removed.
+   */
+  def closeConnection(connConf: GemFireConnectionConf): Unit = {
+    val conns = connConf.locators.map(connections withDefaultValue null).filter(_ != null)
+    if (conns.nonEmpty) connections.synchronized {
+      conns(0).close()
+      connections.retain((k,v) => v != conns(0))
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
new file mode 100644
index 0000000..1d72775
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
@@ -0,0 +1,135 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal
+
+import java.net.InetSocketAddress
+import java.util.{ArrayList => JArrayList}
+
+import com.gemstone.gemfire.cache.client.internal.locator.{GetAllServersResponse, GetAllServersRequest}
+import com.gemstone.gemfire.distributed.internal.ServerLocation
+import com.gemstone.gemfire.distributed.internal.tcpserver.TcpClient
+import org.apache.spark.Logging
+
+import scala.util.{Failure, Success, Try}
+
+
+object LocatorHelper extends Logging {
+
+  /** valid locator strings are: host[port] and host:port */
+  final val LocatorPattern1 = """([\w-_]+(\.[\w-_]+)*)\[([0-9]{2,5})\]""".r
+  final val LocatorPattern2 = """([\w-_]+(\.[\w-_]+)*):([0-9]{2,5})""".r
+
+  /** convert single locator string to Try[(host, port)] */
+  def locatorStr2HostPortPair(locatorStr: String): Try[(String, Int)] =
+    locatorStr match {
+      case LocatorPattern1(host, domain, port) => Success((host, port.toInt))
+      case LocatorPattern2(host, domain, port) => Success((host, port.toInt))
+      case _ => Failure(new Exception(s"invalid locator: $locatorStr"))
+    }
+
+  /** 
+   * Parse locator strings and returns Seq of (hostname, port) pair. 
+   * Valid locator string are one or more "host[port]" and/or "host:port"
+   * separated by `,`. For example:
+   *    host1.mydomain.com[8888],host2.mydomain.com[8889] 
+   *    host1.mydomain.com:8888,host2.mydomain.com:8889 
+   */
+  def parseLocatorsString(locatorsStr: String): Seq[(String, Int)] =
+    locatorsStr.split(",").map(locatorStr2HostPortPair).map(_.get)
+
+
+  /**
+   * Return the list of live GemFire servers for the given locators. 
+   * @param locators locators for the given GemFire cluster
+   * @param serverGroup optional server group name, default is "" (empty string)
+   */
+  def getAllGemFireServers(locators: Seq[(String, Int)], serverGroup: String = ""): Option[Seq[(String, Int)]] = {
+    var result: Option[Seq[(String, Int)]] = None
+    locators.find { case (host, port) =>
+      try {
+        val addr = new InetSocketAddress(host, port)
+        val req = new GetAllServersRequest(serverGroup)
+        val res = TcpClient.requestToServer(addr.getAddress, addr.getPort, req, 2000)
+        if (res != null) {
+          import scala.collection.JavaConverters._
+          val servers = res.asInstanceOf[GetAllServersResponse].getServers.asInstanceOf[JArrayList[ServerLocation]]
+          if (servers.size > 0)
+            result = Some(servers.asScala.map(e => (e.getHostName, e.getPort)))
+        }
+      } catch { case e: Exception => logWarning("getAllGemFireServers error", e)
+      }
+      result.isDefined
+    }
+    result
+  }
+
+  /**
+   * Pick up at most 3 preferred servers from all available servers based on
+   * host name and Spark executor id.
+   *
+   * This method is used by DefaultGemFireConnection to create ClientCache. Usually
+   * one server is enough to initialize ClientCacheFactory, but this provides two
+   * backup servers in case of the 1st server can't be connected.
+   *   
+   * @param servers all available servers in the form of (hostname, port) pairs
+   * @param hostName the host name of the Spark executor
+   * @param executorId the Spark executor Id, such as "<driver>", "0", "1", ...
+   * @return Seq[(hostname, port)] of preferred servers
+   */
+  def pickPreferredGemFireServers(
+    servers: Seq[(String, Int)], hostName: String, executorId: String): Seq[(String, Int)] = {
+
+    // pick up `length` items form the Seq starts at the `start` position.
+    //  The Seq is treated as a ring, so at most `Seq.size` items can be picked
+    def circularTake[T](seq: Seq[T], start: Int, length: Int): Seq[T] = {
+      val size = math.min(seq.size, length)
+      (start until start + size).map(x => seq(x % seq.size))
+    }
+
+    // map executor id to int: "<driver>" (or non-number string) to 0, and "n" to n + 1
+    val id = try { executorId.toInt + 1 } catch { case e: NumberFormatException => 0 }
+    
+    // algorithm: 
+    // 1. sort server list
+    // 2. split sorted server list into 3 sub-lists a, b, and c:
+    //      list-a: servers on the given host
+    //      list-b: servers that are in front of list-a on the sorted server list
+    //      list-c: servers that are behind list-a on the sorted server list
+    //    then rotate list-a based on executor id, then create new server list:
+    //      modified list-a ++ list-c ++ list-b
+    // 3. if there's no server on the given host, then create new server list
+    //    by rotating sorted server list based on executor id.
+    // 4. take up to 3 servers from the new server list
+    val sortedServers = servers.sorted
+    val firstIdx = sortedServers.indexWhere(p => p._1 == hostName)
+    val lastIdx = if (firstIdx < 0) -1 else sortedServers.lastIndexWhere(p => p._1 == hostName)
+
+    if (firstIdx < 0) { // no local server
+      circularTake(sortedServers, id, 3)
+    } else {
+      val (seq1, seq2) = sortedServers.splitAt(firstIdx)
+      val seq = if (firstIdx == lastIdx) {  // one local server
+        seq2 ++ seq1
+      } else { // multiple local server
+        val (seq3, seq4) = seq2.splitAt(lastIdx - firstIdx + 1)
+        val seq3b = if (id % seq3.size == 0) seq3 else circularTake(seq3, id, seq3.size)
+        seq3b ++ seq4 ++ seq1
+      }
+      circularTake(seq, 0, 3)
+    }
+  }  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultCollector.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultCollector.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultCollector.scala
new file mode 100644
index 0000000..39bc0cc
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultCollector.scala
@@ -0,0 +1,152 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions
+
+import java.util.concurrent.{TimeUnit, LinkedBlockingQueue, BlockingQueue}
+import com.gemstone.gemfire.DataSerializer
+import com.gemstone.gemfire.cache.execute.ResultCollector
+import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl
+import com.gemstone.gemfire.cache.query.types.StructType
+import com.gemstone.gemfire.distributed.DistributedMember
+import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput}
+import io.pivotal.gemfire.spark.connector.internal.gemfirefunctions.StructStreamingResultSender.
+       {TYPE_CHUNK, DATA_CHUNK, ERROR_CHUNK, SER_DATA, UNSER_DATA, BYTEARR_DATA}
+
+/**
+ * StructStreamingResultCollector and StructStreamingResultSender are paired
+ * to transfer result of list of `com.gemstone.gemfire.cache.query.Struct`
+ * from GemFire server to Spark Connector (the client of GemFire server) 
+ * in streaming, i.e., while sender sending the result, the collector can
+ * start processing the arrived result without waiting for full result to
+ * become available.
+ */
+class StructStreamingResultCollector(desc: String) extends ResultCollector[Array[Byte], Iterator[Array[Object]]] {
+
+  /** the constructor that provide default `desc` (description) */
+  def this() = this("StructStreamingResultCollector")
+  
+  private val queue: BlockingQueue[Array[Byte]] = new LinkedBlockingQueue[Array[Byte]]()
+  var structType: StructType = null
+
+  /** ------------------------------------------ */
+  /**  ResultCollector interface implementations */
+  /** ------------------------------------------ */
+  
+  override def getResult: Iterator[Array[Object]] = resultIterator
+
+  override def getResult(timeout: Long, unit: TimeUnit): Iterator[Array[Object]] = 
+    throw new UnsupportedOperationException()
+
+  /** addResult add non-empty byte array (chunk) to the queue */
+  override def addResult(memberID: DistributedMember, chunk: Array[Byte]): Unit = 
+    if (chunk != null && chunk.size > 1) {
+      this.queue.add(chunk)
+      // println(s"""$desc receive from $memberID: ${chunk.mkString(" ")}""")
+    }
+
+  /** endResults add special `Array.empty` to the queue as marker of end of data */
+  override def endResults(): Unit = this.queue.add(Array.empty)
+  
+  override def clearResults(): Unit = this.queue.clear()
+
+  /** ------------------------------------------ */
+  /**             Internal methods               */
+  /** ------------------------------------------ */
+
+  def getResultType: StructType = {
+    // trigger lazy resultIterator initialization if necessary
+    if (structType == null)  resultIterator.hasNext
+    structType        
+  }
+
+  /**
+   * Note: The data is sent in chunks, and each chunk contains multiple 
+   * records. So the result iterator is an iterator (I) of iterator (II),
+   * i.e., go through each chunk (iterator (I)), and for each chunk, go 
+   * through each record (iterator (II)). 
+   */
+  private lazy val resultIterator = new Iterator[Array[Object]] {
+
+    private var currentIterator: Iterator[Array[Object]] = nextIterator()
+    
+    override def hasNext: Boolean = {
+      if (!currentIterator.hasNext && currentIterator != Iterator.empty) currentIterator = nextIterator()
+      currentIterator.hasNext
+    }
+
+    /** Note: make sure call `hasNext` first to adjust `currentIterator` */
+    override def next(): Array[Object] = currentIterator.next()
+  }
+  
+  /** get the iterator for the next chunk of data */
+  private def nextIterator(): Iterator[Array[Object]] = {
+    val chunk: Array[Byte] = queue.take
+    if (chunk.isEmpty) {
+      Iterator.empty
+    } else {
+      val input = new ByteArrayDataInput()
+      input.initialize(chunk, Version.CURRENT)
+      val chunkType = input.readByte()
+      // println(s"chunk type $chunkType")
+      chunkType match {
+        case TYPE_CHUNK =>
+          if (structType == null)
+            structType = DataSerializer.readObject(input).asInstanceOf[StructTypeImpl]
+          nextIterator()
+        case DATA_CHUNK =>
+          // require(structType != null && structType.getFieldNames.length > 0)
+          if (structType == null) structType = StructStreamingResultSender.KeyValueType
+          chunkToIterator(input, structType.getFieldNames.length)
+        case ERROR_CHUNK => 
+          val error = DataSerializer.readObject(input).asInstanceOf[Exception]
+          errorPropagationIterator(error)
+        case _ => throw new RuntimeException(s"unknown chunk type: $chunkType")
+      }
+    }
+  }
+
+  /** create a iterator that propagate sender's exception */
+  private def errorPropagationIterator(ex: Exception) = new Iterator[Array[Object]] {
+    val re = new RuntimeException(ex)
+    override def hasNext: Boolean = throw re
+    override def next(): Array[Object] = throw re
+  }
+  
+  /** convert a chunk of data to an iterator */
+  private def chunkToIterator(input: ByteArrayDataInput, rowSize: Int) = new Iterator[Array[Object]] {
+    override def hasNext: Boolean = input.available() > 0
+    val tmpInput = new ByteArrayDataInput()
+    override def next(): Array[Object] = 
+      (0 until rowSize).map { ignore =>
+        val b = input.readByte()
+        b match {
+          case SER_DATA => 
+            val arr: Array[Byte] = DataSerializer.readByteArray(input)
+            tmpInput.initialize(arr, Version.CURRENT)
+            DataSerializer.readObject(tmpInput).asInstanceOf[Object]
+          case UNSER_DATA =>
+            DataSerializer.readObject(input).asInstanceOf[Object]
+          case BYTEARR_DATA =>
+            DataSerializer.readByteArray(input).asInstanceOf[Object]
+          case _ => 
+            throw new RuntimeException(s"unknown data type $b")
+        }
+      }.toArray
+  }
+  
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParser.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParser.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParser.scala
new file mode 100644
index 0000000..71e1823
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParser.scala
@@ -0,0 +1,58 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import scala.util.parsing.combinator.RegexParsers
+
+class QueryParser extends RegexParsers {
+
+  def query: Parser[String] = opt(rep(IMPORT ~ PACKAGE)) ~> select ~> opt(distinct) ~> projection ~> from ~> regions <~ opt(where ~ filter) ^^ {
+    _.toString
+  }
+
+  val IMPORT: Parser[String] = "[Ii][Mm][Pp][Oo][Rr][Tt]".r
+
+  val select: Parser[String] = "[Ss][Ee][Ll][Ee][Cc][Tt]".r
+
+  val distinct: Parser[String] = "[Dd][Ii][Ss][Tt][Ii][Nn][Cc][Tt]".r
+
+  val from: Parser[String] = "[Ff][Rr][Oo][Mm]".r
+
+  val where: Parser[String] = "[Ww][Hh][Ee][Rr][Ee]".r
+
+  def PACKAGE: Parser[String] = """[\w.]+""".r
+
+  def projection: Parser[String] = "*" | repsep("""["\w]+[.\w"]*""".r, ",") ^^ {
+    _.toString
+  }
+
+  def regions: Parser[String] = repsep(region <~ opt(alias), ",") ^^ {
+    _.toString
+  }
+
+  def region: Parser[String] = """/[\w.]+[/[\w.]+]*""".r | """[\w]+[.\w]*""".r
+
+  def alias: Parser[String] = not(where) ~> """[\w]+""".r
+
+  def filter: Parser[String] = """[\w.]+[[\s]+[<>=.'\w]+]*""".r
+}
+
+object QueryParser extends QueryParser {
+
+  def parseOQL(expression: String) = parseAll(query, expression)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryRDD.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryRDD.scala
new file mode 100644
index 0000000..c16a70d
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryRDD.scala
@@ -0,0 +1,83 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFireRDDPartition, ServerSplitsPartitioner}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.{TaskContext, SparkContext, Partition}
+import scala.reflect.ClassTag
+
+/**
+ * An RDD that provides the functionality that read the OQL query result
+ *
+ * @param sc The SparkContext this RDD is associated with
+ * @param queryString The OQL query string
+ * @param connConf The GemFireConnectionConf that provide the GemFireConnection
+ */
+class QueryRDD[T](@transient sc: SparkContext,
+                  queryString: String,
+                  connConf: GemFireConnectionConf)
+                 (implicit ct: ClassTag[T])
+  extends RDD[T](sc, Seq.empty) {
+
+  override def getPartitions: Array[Partition] = {
+    val conn = connConf.getConnection
+    val regionPath = getRegionPathFromQuery(queryString)
+    val md = conn.getRegionMetadata(regionPath)
+    md match {
+      case Some(metadata) =>
+        if (metadata.isPartitioned) {
+          val splits = ServerSplitsPartitioner.partitions(conn, metadata, Map.empty)
+          logInfo(s"QueryRDD.getPartitions():isPartitioned=true, partitions=${splits.mkString(",")}")
+          splits
+        }
+        else {
+          logInfo(s"QueryRDD.getPartitions():isPartitioned=false")
+          Array[Partition](new GemFireRDDPartition(0, Set.empty))
+
+        }
+      case None => throw new RuntimeException(s"Region $regionPath metadata was not found.")
+    }
+  }
+
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    val buckets = split.asInstanceOf[GemFireRDDPartition].bucketSet
+    val regionPath = getRegionPathFromQuery(queryString)
+    val result = connConf.getConnection.executeQuery(regionPath, buckets, queryString)
+    result match {
+      case it: Iterator[T] =>
+        logInfo(s"QueryRDD.compute():query=$queryString, partition=$split")
+        it
+      case _ =>
+        throw new RuntimeException("Unexpected OQL result: " + result.toString)
+    }
+  }
+
+  private def getRegionPathFromQuery(queryString: String): String = {
+    val r = QueryParser.parseOQL(queryString).get
+    r match {
+      case r: String =>
+        val start = r.indexOf("/") + 1
+        var end = r.indexOf(")")
+        if (r.indexOf(".") > 0) end = math.min(r.indexOf("."), end)
+        if (r.indexOf(",") > 0) end = math.min(r.indexOf(","), end)
+        val regionPath = r.substring(start, end)
+        regionPath
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryResultCollector.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryResultCollector.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryResultCollector.scala
new file mode 100644
index 0000000..7032e5a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryResultCollector.scala
@@ -0,0 +1,69 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import java.util.concurrent.{TimeUnit, LinkedBlockingDeque}
+
+import com.gemstone.gemfire.DataSerializer
+import com.gemstone.gemfire.cache.execute.ResultCollector
+import com.gemstone.gemfire.distributed.DistributedMember
+import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput}
+
+class QueryResultCollector extends ResultCollector[Array[Byte], Iterator[Object]]{
+
+  private val queue = new LinkedBlockingDeque[Array[Byte]]()
+
+  override def getResult = resultIterator
+
+  override def getResult(timeout: Long, unit: TimeUnit) = throw new UnsupportedOperationException
+
+  override def addResult(memberID: DistributedMember , chunk: Array[Byte]) =
+    if (chunk != null && chunk.size > 0) {
+      queue.add(chunk)
+    }
+
+  override def endResults = queue.add(Array.empty)
+
+
+  override def clearResults = queue.clear
+
+  private lazy val resultIterator = new Iterator[Object] {
+    private var currentIterator = nextIterator
+    def hasNext = {
+      if (!currentIterator.hasNext && currentIterator != Iterator.empty)
+        currentIterator = nextIterator
+      currentIterator.hasNext
+    }
+    def next = currentIterator.next
+  }
+
+  private def nextIterator: Iterator[Object] = {
+    val chunk = queue.take
+    if (chunk.isEmpty) {
+      Iterator.empty
+    }
+    else {
+      val input = new ByteArrayDataInput
+      input.initialize(chunk, Version.CURRENT)
+      new Iterator[Object] {
+        override def hasNext: Boolean = input.available() > 0
+        override def next: Object = DataSerializer.readObject(input).asInstanceOf[Object]
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RDDConverter.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RDDConverter.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RDDConverter.scala
new file mode 100644
index 0000000..894066c
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RDDConverter.scala
@@ -0,0 +1,40 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, SQLContext}
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.sources.{BaseRelation, TableScan}
+
+import scala.tools.nsc.backend.icode.analysis.DataFlowAnalysis
+
+case class OQLRelation[T](queryRDD: QueryRDD[T])(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan {
+
+  override def schema: StructType = new SchemaBuilder(queryRDD).toSparkSchema()
+
+  override def buildScan(): RDD[Row] = new RowBuilder(queryRDD).toRowRDD()
+
+}
+
+object RDDConverter {
+
+  def queryRDDToDataFrame[T](queryRDD: QueryRDD[T], sqlContext: SQLContext): DataFrame = {
+    sqlContext.baseRelationToDataFrame(OQLRelation(queryRDD)(sqlContext))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RowBuilder.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RowBuilder.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RowBuilder.scala
new file mode 100644
index 0000000..3a4fa6a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/RowBuilder.scala
@@ -0,0 +1,38 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import com.gemstone.gemfire.cache.query.internal.StructImpl
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql._
+
+class RowBuilder[T](queryRDD: QueryRDD[T]) {
+
+  /**
+   * Convert QueryRDD to RDD of Row
+   * @return RDD of Rows
+   */
+  def toRowRDD(): RDD[Row] = {
+    val rowRDD = queryRDD.map(row => {
+      row match {
+        case si: StructImpl => Row.fromSeq(si.getFieldValues)
+        case obj: Object => Row.fromSeq(Seq(obj))
+      }
+    })
+    rowRDD
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/SchemaBuilder.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/SchemaBuilder.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/SchemaBuilder.scala
new file mode 100644
index 0000000..11fff90
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/SchemaBuilder.scala
@@ -0,0 +1,73 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import com.gemstone.gemfire.cache.query.internal.StructImpl
+import org.apache.spark.sql.types._
+import scala.collection.mutable.ListBuffer
+import org.apache.spark.Logging
+
+class SchemaBuilder[T](queryRDD: QueryRDD[T]) extends Logging {
+
+  val nullStructType = StructType(Nil)
+  
+  val typeMap:Map[Class[_], DataType] = Map( 
+    (classOf[java.lang.String], StringType),
+    (classOf[java.lang.Integer], IntegerType),
+    (classOf[java.lang.Short], ShortType),
+    (classOf[java.lang.Long], LongType),
+    (classOf[java.lang.Double], DoubleType),
+    (classOf[java.lang.Float], FloatType),
+    (classOf[java.lang.Boolean], BooleanType),
+    (classOf[java.lang.Byte], ByteType),
+    (classOf[java.util.Date], DateType),
+    (classOf[java.lang.Object], nullStructType)
+  )
+  
+  /**
+   * Analyse QueryRDD to get the Spark schema
+   * @return The schema represented by Spark StructType
+   */
+  def toSparkSchema(): StructType = {
+    val row = queryRDD.first()
+    val tpe = row match {
+      case r: StructImpl => constructFromStruct(r)
+      case null => StructType(StructField("col1", NullType) :: Nil)
+      case default => 
+        val value = typeMap.getOrElse(default.getClass(), nullStructType)
+        StructType(StructField("col1", value) :: Nil)
+    }
+    logInfo(s"Schema: $tpe")
+    tpe
+  }
+  
+  def constructFromStruct(r:StructImpl) = {
+    val names = r.getFieldNames
+    val values = r.getFieldValues
+    val lb = new ListBuffer[StructField]()
+    for (i <- 0 until names.length) {
+      val name = names(i)
+      val value = values(i)
+      val dataType = value match {
+        case null => NullType
+        case default => typeMap.getOrElse(default.getClass,  nullStructType)
+      }
+      lb += StructField(name, dataType)
+    }
+    StructType(lb.toSeq)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/UndefinedSerializer.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/UndefinedSerializer.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/UndefinedSerializer.scala
new file mode 100644
index 0000000..35e8120
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/oql/UndefinedSerializer.scala
@@ -0,0 +1,46 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import com.esotericsoftware.kryo.{Kryo, Serializer}
+import com.esotericsoftware.kryo.io.{Output, Input}
+import com.gemstone.gemfire.cache.query.QueryService
+import com.gemstone.gemfire.cache.query.internal.Undefined
+
+/**
+ * This is the customized serializer to serialize QueryService.UNDEFINED,
+ * i.e. com.gemstone.gemfire.cache.query.internal.Undefined, in order to
+ * guarantee the singleton Undefined after its deserialization within Spark.
+ */
+class UndefinedSerializer extends Serializer[Undefined] {
+
+  def write(kryo: Kryo, output: Output, u: Undefined) {
+    //Only serialize a byte for Undefined
+    output.writeByte(u.getDSFID)
+  }
+
+  def read (kryo: Kryo, input: Input, tpe: Class[Undefined]): Undefined = {
+    //Read DSFID of Undefined
+    input.readByte()
+    QueryService.UNDEFINED match {
+      case null => new Undefined
+      case _ =>
+        //Avoid calling Undefined constructor again.
+        QueryService.UNDEFINED.asInstanceOf[Undefined]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireJoinRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireJoinRDD.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireJoinRDD.scala
new file mode 100644
index 0000000..cfa1537
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireJoinRDD.scala
@@ -0,0 +1,67 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf
+import org.apache.spark.{TaskContext, Partition}
+import org.apache.spark.rdd.RDD
+import scala.collection.JavaConversions._
+
+/**
+ * An `RDD[T, V]` that will represent the result of a join between `left` RDD[T]
+ * and the specified GemFire Region[K, V].
+ */
+class GemFireJoinRDD[T, K, V] private[connector]
+  ( left: RDD[T],
+    func: T => K,
+    val regionPath: String,
+    val connConf: GemFireConnectionConf
+  ) extends RDD[(T, V)](left.context, left.dependencies) {
+
+  /** validate region existence when GemFireRDD object is created */
+  validate()
+
+  /** Validate region, and make sure it exists. */
+  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
+
+  override protected def getPartitions: Array[Partition] = left.partitions
+
+  override def compute(split: Partition, context: TaskContext): Iterator[(T, V)] = {
+    val region = connConf.getConnection.getRegionProxy[K, V](regionPath)
+    if (func == null) computeWithoutFunc(split, context, region)
+    else computeWithFunc(split, context, region)
+  }
+
+  /** T is (K, V1) since there's no map function `func` */
+  private def computeWithoutFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, V)] = {
+    val leftPairs = left.iterator(split, context).toList.asInstanceOf[List[(K, _)]]
+    val leftKeys = leftPairs.map { case (k, v) => k}.toSet
+    // Note: get all will return (key, null) for non-exist entry, so remove those entries
+    val rightPairs = region.getAll(leftKeys).filter { case (k, v) => v != null}
+    leftPairs.filter{case (k, v) => rightPairs.contains(k)}
+             .map {case (k, v) => ((k, v).asInstanceOf[T], rightPairs.get(k).get)}.toIterator
+  }
+  
+  private def computeWithFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, V)] = {
+    val leftPairs = left.iterator(split, context).toList.map(t => (t, func(t)))
+    val leftKeys = leftPairs.map { case (t, k) => k}.toSet
+    // Note: get all will return (key, null) for non-exist entry, so remove those entries
+    val rightPairs = region.getAll(leftKeys).filter { case (k, v) => v != null}
+    leftPairs.filter { case (t, k) => rightPairs.contains(k)}.map {case (t, k) => (t, rightPairs.get(k).get)}.toIterator
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireOuterJoinRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireOuterJoinRDD.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireOuterJoinRDD.scala
new file mode 100644
index 0000000..ec3a512
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireOuterJoinRDD.scala
@@ -0,0 +1,69 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf
+import org.apache.spark.{TaskContext, Partition}
+import org.apache.spark.rdd.RDD
+import scala.collection.JavaConversions._
+
+/**
+ * An `RDD[ T, Option[V] ]` that represents the result of a left outer join 
+ * between `left` RDD[T] and the specified GemFire Region[K, V].
+ */
+class GemFireOuterJoinRDD[T, K, V] private[connector]
+ ( left: RDD[T],
+   func: T => K,
+   val regionPath: String,
+   val connConf: GemFireConnectionConf
+  ) extends RDD[(T, Option[V])](left.context, left.dependencies) {
+
+  /** validate region existence when GemFireRDD object is created */
+  validate()
+
+  /** Validate region, and make sure it exists. */
+  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
+
+  override protected def getPartitions: Array[Partition] = left.partitions
+
+  override def compute(split: Partition, context: TaskContext): Iterator[(T, Option[V])] = {
+    val region = connConf.getConnection.getRegionProxy[K, V](regionPath)
+    if (func == null) computeWithoutFunc(split, context, region)
+    else computeWithFunc(split, context, region)
+  }
+
+  /** T is (K1, V1), and K1 and K are the same type since there's no map function `func` */
+  private def computeWithoutFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, Option[V])] = {
+    val leftPairs = left.iterator(split, context).toList.asInstanceOf[List[(K, _)]]
+    val leftKeys = leftPairs.map { case (k, v) => k}.toSet
+    // Note: get all will return (key, null) for non-exist entry
+    val rightPairs = region.getAll(leftKeys)
+    // rightPairs is a java.util.Map, not scala map, so need to convert map.get() to Option
+    leftPairs.map{ case (k, v) => ((k, v).asInstanceOf[T], Option(rightPairs.get(k))) }.toIterator
+  }
+
+  private def computeWithFunc(split: Partition, context: TaskContext, region: Region[K, V]): Iterator[(T, Option[V])] = {
+    val leftPairs = left.iterator(split, context).toList.map(t => (t, func(t)))
+    val leftKeys = leftPairs.map { case (t, k) => k}.toSet
+    // Note: get all will return (key, null) for non-exist entry
+    val rightPairs = region.getAll(leftKeys)
+    // rightPairs is a java.util.Map, not scala map, so need to convert map.get() to Option
+    leftPairs.map{ case (t, k) => (t, Option(rightPairs.get(k)))}.toIterator
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartition.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartition.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartition.scala
new file mode 100644
index 0000000..02e1eed
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartition.scala
@@ -0,0 +1,36 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import org.apache.spark.Partition
+
+/**
+ * This serializable class represents a GemFireRDD partition. Each partition is mapped
+ * to one or more buckets of region. The GemFireRDD can materialize the data of the 
+ * partition based on all information contained here.
+ * @param partitionId partition id, a 0 based number.
+ * @param bucketSet region bucket id set for this partition. Set.empty means whole
+ *                  region (used for replicated region)
+ * @param locations preferred location for this partition                  
+ */
+case class GemFireRDDPartition (
+  partitionId: Int, bucketSet: Set[Int], locations: Seq[String] = Nil)
+  extends Partition  {
+  
+  override def index: Int = partitionId
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitioner.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitioner.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitioner.scala
new file mode 100644
index 0000000..807a979
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitioner.scala
@@ -0,0 +1,59 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import io.pivotal.gemfire.spark.connector.GemFireConnection
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata
+import org.apache.spark.{Logging, Partition}
+
+import scala.reflect.ClassTag
+
+/**
+ * A GemFireRDD partitioner is used to partition the region into multiple RDD partitions.
+ */
+trait GemFireRDDPartitioner extends Serializable {
+
+  def name: String
+  
+  /** the function that generates partitions */
+  def partitions[K: ClassTag, V: ClassTag]
+    (conn: GemFireConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition]
+}
+
+object GemFireRDDPartitioner extends Logging {
+
+  /** To add new partitioner, just add it to the following list */
+  final val partitioners: Map[String, GemFireRDDPartitioner] =
+    List(OnePartitionPartitioner, ServerSplitsPartitioner).map(e => (e.name, e)).toMap
+
+  /**
+   * Get a partitioner based on given name, a default partitioner will be returned if there's
+   * no partitioner for the given name. 
+   */
+  def apply(name: String = defaultPartitionedRegionPartitioner.name): GemFireRDDPartitioner = {
+    val p = partitioners.get(name)
+    if (p.isDefined) p.get else {
+      logWarning(s"Invalid preferred partitioner name $name.")
+      defaultPartitionedRegionPartitioner
+    }
+  }
+
+  val defaultReplicatedRegionPartitioner = OnePartitionPartitioner
+
+  val defaultPartitionedRegionPartitioner = ServerSplitsPartitioner
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitionerImpl.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitionerImpl.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitionerImpl.scala
new file mode 100644
index 0000000..72904ee
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDPartitionerImpl.scala
@@ -0,0 +1,89 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import io.pivotal.gemfire.spark.connector.GemFireConnection
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata
+import io.pivotal.gemfire.spark.connector.NumberPartitionsPerServerPropKey
+import org.apache.spark.Partition
+import scala.collection.JavaConversions._
+import scala.collection.immutable.SortedSet
+import scala.collection.mutable
+import scala.reflect.ClassTag
+
+/** This partitioner maps whole region to one GemFireRDDPartition */
+object OnePartitionPartitioner extends GemFireRDDPartitioner {
+
+  override val name = "OnePartition"
+
+  override def partitions[K: ClassTag, V: ClassTag]
+    (conn: GemFireConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition] =
+    Array[Partition](new GemFireRDDPartition(0, Set.empty))
+}
+
+/**
+  * This partitioner maps whole region to N * M GemFire RDD partitions, where M is the number of 
+  * GemFire servers that contain the data for the given region. Th default value of N is 1.
+  */
+object ServerSplitsPartitioner extends GemFireRDDPartitioner {
+
+  override val name = "ServerSplits"
+
+  override def partitions[K: ClassTag, V: ClassTag]
+  (conn: GemFireConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition] = {
+    if (md == null) throw new RuntimeException("RegionMetadata is null")
+    val n = try { env.getOrElse(NumberPartitionsPerServerPropKey, "2").toInt } catch { case e: NumberFormatException => 2 }
+    if (!md.isPartitioned || md.getServerBucketMap == null || md.getServerBucketMap.isEmpty)
+      Array[Partition](new GemFireRDDPartition(0, Set.empty))
+    else {
+      val map = mapAsScalaMap(md.getServerBucketMap)
+        .map { case (srv, set) => (srv, asScalaSet(set).map(_.toInt)) }.toList
+        .map { case (srv, set) => (srv.getHostName, set) }
+       doPartitions(map, md.getTotalBuckets, n)
+    }
+  }
+
+  /** Converts server to bucket ID set list to array of RDD partitions */
+  def doPartitions(serverBucketMap: List[(String, mutable.Set[Int])], totalBuckets: Int, n: Int)
+    : Array[Partition] = {
+
+    // method that calculates the group size for splitting "k" items into "g" groups
+    def groupSize(k: Int, g: Int): Int = scala.math.ceil(k / g.toDouble).toInt
+
+    // 1. convert list of server and bucket set pairs to a list of server and sorted bucket set pairs
+    val srvToSortedBucketSet = serverBucketMap.map { case (srv, set) => (srv, SortedSet[Int]() ++ set) }
+
+    // 2. split bucket set of each server into n splits if possible, and server to Seq(server)
+    val srvToSplitedBuckeSet = srvToSortedBucketSet.flatMap { case (host, set) =>
+      if (set.isEmpty) Nil else set.grouped(groupSize(set.size, n)).toList.map(s => (Seq(host), s)) }
+
+    // 3. calculate empty bucket IDs by removing all bucket sets of all servers from the full bucket sets
+    val emptyIDs = SortedSet[Int]() ++ ((0 until totalBuckets).toSet /: srvToSortedBucketSet) {case (s1, (k, s2)) => s1 &~ s2}
+
+    // 4. distribute empty bucket IDs to all partitions evenly.
+    //    The empty buckets do not contain data when partitions are created, but they may contain data
+    //    when RDD is materialized, so need to include those bucket IDs in the partitions.
+    val srvToFinalBucketSet = if (emptyIDs.isEmpty) srvToSplitedBuckeSet
+      else srvToSplitedBuckeSet.zipAll(
+        emptyIDs.grouped(groupSize(emptyIDs.size, srvToSplitedBuckeSet.size)).toList, (Nil, Set.empty), Set.empty).map
+          { case ((server, set1), set2) => (server, SortedSet[Int]() ++ set1 ++ set2) }
+
+    // 5. create array of partitions w/ 0-based index
+    (0 until srvToFinalBucketSet.size).toList.zip(srvToFinalBucketSet).map
+      { case (i, (srv, set)) => new GemFireRDDPartition(i, set, srv) }.toArray
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDWriter.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDWriter.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDWriter.scala
new file mode 100644
index 0000000..b6b1330
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRDDWriter.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.pivotal.gemfire.spark.connector.internal.rdd
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector._
+import org.apache.spark.{Logging, TaskContext}
+
+import scala.collection.Iterator
+import java.util.{HashMap => JMap}
+
+/** This trait provide some common code for pair and non-pair RDD writer */
+private[rdd] abstract class GemFireRDDWriterBase (opConf: Map[String, String]) extends Serializable {
+
+  val batchSize = try { opConf.getOrElse(RDDSaveBatchSizePropKey, RDDSaveBatchSizeDefault.toString).toInt}
+                  catch { case e: NumberFormatException => RDDSaveBatchSizeDefault }
+
+  def mapDump(map: Map[_, _], num: Int): String = {
+    val firstNum = map.take(num + 1)
+    if (firstNum.size > num) s"$firstNum ..." else s"$firstNum"    
+  }  
+}
+
+/**
+ * Writer object that provides write function that saves non-pair RDD partitions to GemFire.
+ * Those functions will be executed on Spark executors.
+ * @param regionPath the full path of the region where the data is written to
+ */
+class GemFireRDDWriter[T, K, V] 
+  (regionPath: String, connConf: GemFireConnectionConf, opConf: Map[String, String] = Map.empty)
+  extends GemFireRDDWriterBase(opConf) with Serializable with Logging {
+
+  def write(func: T => (K, V))(taskContext: TaskContext, data: Iterator[T]): Unit = {
+    val region: Region[K, V] = connConf.getConnection.getRegionProxy[K, V](regionPath)
+    var count = 0
+    val chunks = data.grouped(batchSize)
+    chunks.foreach { chunk =>
+      val map = chunk.foldLeft(new JMap[K, V]()){case (m, t) => val (k, v) = func(t); m.put(k, v); m}
+      region.putAll(map)
+      count += chunk.length
+    }
+    logDebug(s"$count entries (batch.size = $batchSize) are saved to region $regionPath")
+  }
+}
+
+
+/**
+ * Writer object that provides write function that saves pair RDD partitions to GemFire.
+ * Those functions will be executed on Spark executors.
+ * @param regionPath the full path of the region where the data is written to
+ */
+class GemFirePairRDDWriter[K, V]
+  (regionPath: String, connConf: GemFireConnectionConf, opConf: Map[String, String] = Map.empty)
+  extends GemFireRDDWriterBase(opConf) with Serializable with Logging {
+
+  def write(taskContext: TaskContext, data: Iterator[(K, V)]): Unit = {
+    val region: Region[K, V] = connConf.getConnection.getRegionProxy[K, V](regionPath)
+    var count = 0
+    val chunks = data.grouped(batchSize)
+    chunks.foreach { chunk =>
+      val map = chunk.foldLeft(new JMap[K, V]()){case (m, (k,v)) => m.put(k,v); m}
+      region.putAll(map)
+      count += chunk.length
+    }
+    logDebug(s"$count entries (batch.batch = $batchSize) are saved to region $regionPath")
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
new file mode 100644
index 0000000..4535917
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
@@ -0,0 +1,138 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.rdd
+
+import scala.collection.Seq
+import scala.reflect.ClassTag
+import org.apache.spark.rdd.RDD
+import org.apache.spark.{TaskContext, Partition, SparkContext}
+import io.pivotal.gemfire.spark.connector.{GemFireConnectionConf, PreferredPartitionerPropKey}
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartitioner._
+
+/**
+ * This class exposes GemFire region as a RDD.
+ * @param sc the Spark Context
+ * @param regionPath the full path of the region
+ * @param connConf the GemFireConnectionConf to access the region
+ * @param opConf the parameters for this operation, such as preferred partitioner.
+ */
+class GemFireRegionRDD[K, V] private[connector]
+  (@transient sc: SparkContext,
+   val regionPath: String,
+   val connConf: GemFireConnectionConf,
+   val opConf: Map[String, String] = Map.empty,
+   val whereClause: Option[String] = None 
+  ) (implicit ctk: ClassTag[K], ctv: ClassTag[V])
+  extends RDD[(K, V)](sc, Seq.empty) {
+
+  /** validate region existence when GemFireRDD object is created */
+  validate()
+
+  /** Validate region, and make sure it exists. */
+  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
+
+  def kClassTag = ctk
+  
+  def vClassTag = ctv
+
+  /**
+   * method `copy` is used by method `where` that creates new immutable
+   * GemFireRDD instance based this instance.
+   */
+  private def copy(
+    regionPath: String = regionPath,
+    connConf: GemFireConnectionConf = connConf,
+    opConf: Map[String, String] = opConf,
+    whereClause: Option[String] = None
+  ): GemFireRegionRDD[K, V] = {
+
+    require(sc != null,
+    """RDD transformation requires a non-null SparkContext. Unfortunately
+      |SparkContext in this GemFireRDD is null. This can happen after 
+      |GemFireRDD has been deserialized. SparkContext is not Serializable,
+      |therefore it deserializes to null. RDD transformations are not allowed
+      |inside lambdas used in other RDD transformations.""".stripMargin )
+
+    new GemFireRegionRDD[K, V](sc, regionPath, connConf, opConf, whereClause)
+  }
+
+  /** When where clause is specified, OQL query
+    * `select key, value from /<region-path>.entries where <where clause> `
+    * is used to filter the dataset.
+    */
+  def where(whereClause: Option[String]): GemFireRegionRDD[K, V] = {
+    if (whereClause.isDefined) copy(whereClause = whereClause)
+    else this
+  }
+
+  /** this version is for Java API that doesn't use scala.Option */
+  def where(whereClause: String): GemFireRegionRDD[K, V] = {
+    if (whereClause == null || whereClause.trim.isEmpty) this
+    else copy(whereClause = Option(whereClause.trim))
+  }
+
+  /**
+   * Use preferred partitioner generate partitions. `defaultReplicatedRegionPartitioner`
+   * will be used if it's a replicated region. 
+   */
+  override def getPartitions: Array[Partition] = {
+    val conn = connConf.getConnection
+    val md = conn.getRegionMetadata[K, V](regionPath)
+    md match {
+      case None => throw new RuntimeException(s"region $regionPath was not found.")
+      case Some(data) =>
+        logInfo(s"""RDD id=${this.id} region=$regionPath conn=${connConf.locators.mkString(",")}, env=$opConf""")
+        val p = if (data.isPartitioned) preferredPartitioner else defaultReplicatedRegionPartitioner
+        val splits = p.partitions[K, V](conn, data, opConf)
+        logDebug(s"""RDD id=${this.id} region=$regionPath partitions=\n  ${splits.mkString("\n  ")}""")
+        splits
+    }
+  }
+
+  /**
+   * provide preferred location(s) (host name(s)) of the given partition. 
+   * Only some partitioner implementation(s) provides this info, which is
+   * useful when Spark cluster and GemFire cluster share some hosts.
+   */
+  override def getPreferredLocations(split: Partition) =
+    split.asInstanceOf[GemFireRDDPartition].locations
+
+  /**
+   * Get preferred partitioner. return `defaultPartitionedRegionPartitioner` if none
+   * preference is specified. 
+   */
+  private def preferredPartitioner = 
+    GemFireRDDPartitioner(opConf.getOrElse(
+      PreferredPartitionerPropKey, GemFireRDDPartitioner.defaultPartitionedRegionPartitioner.name))
+
+  /** materialize a RDD partition */
+  override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
+    val partition = split.asInstanceOf[GemFireRDDPartition]
+    logDebug(s"compute RDD id=${this.id} partition $partition")
+    connConf.getConnection.getRegionData[K,V](regionPath, whereClause, partition)
+    // new InterruptibleIterator(context, split.asInstanceOf[GemFireRDDPartition[K, V]].iterator)
+  }
+}
+
+object GemFireRegionRDD {
+
+  def apply[K: ClassTag, V: ClassTag](sc: SparkContext, regionPath: String,
+    connConf: GemFireConnectionConf, opConf: Map[String, String] = Map.empty)
+    : GemFireRegionRDD[K, V] =
+    new GemFireRegionRDD[K, V](sc, regionPath, connConf, opConf)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRegionRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRegionRDD.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRegionRDD.scala
new file mode 100644
index 0000000..1a2e8f2
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRegionRDD.scala
@@ -0,0 +1,26 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi
+
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRegionRDD
+import org.apache.spark.api.java.JavaPairRDD
+
+class GemFireJavaRegionRDD[K, V](rdd: GemFireRegionRDD[K, V]) extends JavaPairRDD[K, V](rdd)(rdd.kClassTag, rdd.vClassTag) {
+  
+  def where(whereClause: String): GemFireJavaRegionRDD[K, V] = new GemFireJavaRegionRDD(rdd.where(whereClause))
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/JavaAPIHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/JavaAPIHelper.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/JavaAPIHelper.scala
new file mode 100644
index 0000000..3d10fb8
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/javaapi/JavaAPIHelper.scala
@@ -0,0 +1,53 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi
+
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
+import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream}
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+/**
+ *  A helper class to make it possible to access components written in Scala from Java code.
+ */
+private[connector] object JavaAPIHelper {
+
+  /** Returns a `ClassTag` of a given runtime class. */
+  def getClassTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
+
+  /**
+   * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef].
+   * see JavaSparkContext.fakeClassTag in Spark for more info.
+   */
+  def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
+
+  /** Converts a Java `Properties` to a Scala immutable `Map[String, String]`. */
+  def propertiesToScalaMap[K, V](props: java.util.Properties): Map[String, String] =
+    Map(props.toSeq: _*)
+
+  /** convert a JavaRDD[(K,V)] to JavaPairRDD[K,V] */
+  def toJavaPairRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] =
+    JavaPairRDD.fromJavaRDD(rdd)
+
+  /** convert a JavaDStream[(K,V)] to JavaPairDStream[K,V] */
+  def toJavaPairDStream[K, V](ds: JavaDStream[(K, V)]): JavaPairDStream[K, V] =
+    JavaPairDStream.fromJavaDStream(ds)
+
+  /** an empty Map[String, String] for default opConf **/
+  val emptyStrStrMap: Map[String, String] = Map.empty
+}


[09/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/JavaApiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/JavaApiIntegrationTest.java b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/JavaApiIntegrationTest.java
new file mode 100644
index 0000000..bb75c7a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/JavaApiIntegrationTest.java
@@ -0,0 +1,424 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector;
+
+import com.gemstone.gemfire.cache.Region;
+import io.pivotal.gemfire.spark.connector.GemFireConnection;
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf$;
+import io.pivotal.gemfire.spark.connector.internal.DefaultGemFireConnectionManager$;
+import io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaRegionRDD;
+import ittest.io.pivotal.gemfire.spark.connector.testkit.GemFireCluster$;
+import ittest.io.pivotal.gemfire.spark.connector.testkit.IOUtils;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.scalatest.junit.JUnitSuite;
+import io.pivotal.gemfire.spark.connector.package$;
+import scala.Tuple2;
+import scala.Option;
+import scala.Some;
+
+import java.util.*;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.RDDSaveBatchSizePropKey;
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.javaFunctions;
+import static org.junit.Assert.*;
+
+public class JavaApiIntegrationTest extends JUnitSuite {
+
+  static JavaSparkContext jsc = null;
+  static GemFireConnectionConf connConf = null;
+  
+  static int numServers = 2;
+  static int numObjects = 1000;
+  static String regionPath = "pr_str_int_region";
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // start gemfire cluster, and spark context
+    Properties settings = new Properties();
+    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml");
+    settings.setProperty("num-of-servers", Integer.toString(numServers));
+    int locatorPort = GemFireCluster$.MODULE$.start(settings);
+
+    // start spark context in local mode
+    Properties props = new Properties();
+    props.put("log4j.logger.org.apache.spark", "INFO");
+    props.put("log4j.logger.io.pivotal.gemfire.spark.connector","DEBUG");
+    IOUtils.configTestLog4j("ERROR", props);
+    SparkConf conf = new SparkConf()
+            .setAppName("RetrieveRegionIntegrationTest")
+            .setMaster("local[2]")
+            .set(package$.MODULE$.GemFireLocatorPropKey(), "localhost:"+ locatorPort);
+    // sc = new SparkContext(conf);
+    jsc = new JavaSparkContext(conf);
+    connConf = GemFireConnectionConf.apply(jsc.getConf());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    // stop connection, spark context, and gemfire cluster
+    DefaultGemFireConnectionManager$.MODULE$.closeConnection(GemFireConnectionConf$.MODULE$.apply(jsc.getConf()));
+    jsc.stop();
+    GemFireCluster$.MODULE$.stop();
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   utility methods
+  // --------------------------------------------------------------------------------------------
+
+  private <K,V> void matchMapAndPairList(Map<K,V> map, List<Tuple2<K,V>> list) {
+    assertTrue("size mismatch \nmap: " + map.toString() + "\nlist: " + list.toString(), map.size() == list.size());
+    for (Tuple2<K, V> p : list) {
+      assertTrue("value mismatch: k=" + p._1() + " v1=" + p._2() + " v2=" + map.get(p._1()),
+                 p._2().equals(map.get(p._1())));
+    }
+  }
+
+  private Region<String, Integer> prepareStrIntRegion(String regionPath, int start, int stop) {
+    HashMap<String, Integer> entriesMap = new HashMap<>();
+    for (int i = start; i < stop; i ++) {
+      entriesMap.put("k_" + i, i);
+    }
+
+    GemFireConnection conn = connConf.getConnection();
+    Region<String, Integer> region = conn.getRegionProxy(regionPath);
+    region.removeAll(region.keySetOnServer());
+    region.putAll(entriesMap);
+    return region;
+  }
+
+  private JavaPairRDD<String, Integer> prepareStrIntJavaPairRDD(int start, int stop) {
+    List<Tuple2<String, Integer>> data = new ArrayList<>();
+    for (int i = start; i < stop; i ++) {
+      data.add(new Tuple2<>("k_" + i, i));
+    }
+    return jsc.parallelizePairs(data);
+  }
+
+  private JavaPairRDD<Integer, Integer> prepareIntIntJavaPairRDD(int start, int stop) {
+    List<Tuple2<Integer, Integer>> data = new ArrayList<>();
+    for (int i = start; i < stop; i ++) {
+      data.add(new Tuple2<>(i, i * 2));
+    }
+    return jsc.parallelizePairs(data);
+  }
+
+  private JavaRDD<Integer> prepareIntJavaRDD(int start, int stop) {
+    List<Integer> data = new ArrayList<>();
+    for (int i = start; i < stop; i ++) {
+      data.add(i);
+    }
+    return jsc.parallelize(data);
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaRDD.saveToGemfire 
+  // --------------------------------------------------------------------------------------------
+
+  static class IntToStrIntPairFunction implements PairFunction<Integer, String, Integer> {
+    @Override public Tuple2<String, Integer> call(Integer x) throws Exception {
+      return new Tuple2<>("k_" + x, x);
+    }
+  }
+
+  @Test
+  public void testRDDSaveToGemfireWithDefaultConnConfAndOpConf() throws Exception {
+    verifyRDDSaveToGemfire(true, true);
+  }
+
+  @Test
+  public void testRDDSaveToGemfireWithDefaultConnConf() throws Exception {
+    verifyRDDSaveToGemfire(true, false);
+  }
+  
+  @Test
+  public void testRDDSaveToGemfireWithConnConfAndOpConf() throws Exception {
+    verifyRDDSaveToGemfire(false, true);
+  }
+
+  @Test
+  public void testRDDSaveToGemfireWithConnConf() throws Exception {
+    verifyRDDSaveToGemfire(false, false);
+  }
+  
+  public void verifyRDDSaveToGemfire(boolean useDefaultConnConf, boolean useOpConf) throws Exception {
+    Region<String, Integer> region = prepareStrIntRegion(regionPath, 0, 0);  // remove all entries
+    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(0, numObjects);
+
+    PairFunction<Integer, String, Integer> func = new IntToStrIntPairFunction();
+    Properties opConf = new Properties();
+    opConf.put(RDDSaveBatchSizePropKey, "200");
+
+    if (useDefaultConnConf) {
+      if (useOpConf)
+        javaFunctions(rdd1).saveToGemfire(regionPath, func, opConf);
+      else
+        javaFunctions(rdd1).saveToGemfire(regionPath, func);
+    } else {
+      if (useOpConf)
+        javaFunctions(rdd1).saveToGemfire(regionPath, func, connConf, opConf);
+      else
+        javaFunctions(rdd1).saveToGemfire(regionPath, func, connConf);
+    }
+    
+    Set<String> keys = region.keySetOnServer();
+    Map<String, Integer> map = region.getAll(keys);
+
+    List<Tuple2<String, Integer>> expectedList = new ArrayList<>();
+
+    for (int i = 0; i < numObjects; i ++) {
+      expectedList.add(new Tuple2<>("k_" + i, i));
+    }
+    matchMapAndPairList(map, expectedList);
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaPairRDD.saveToGemfire
+  // --------------------------------------------------------------------------------------------
+
+  @Test
+  public void testPairRDDSaveToGemfireWithDefaultConnConfAndOpConf() throws Exception {
+    verifyPairRDDSaveToGemfire(true, true);
+  }
+
+  @Test
+  public void testPairRDDSaveToGemfireWithDefaultConnConf() throws Exception {
+    verifyPairRDDSaveToGemfire(true, false);
+  }
+  
+  @Test
+  public void testPairRDDSaveToGemfireWithConnConfAndOpConf() throws Exception {
+    verifyPairRDDSaveToGemfire(false, true);
+  }
+
+  @Test
+  public void testPairRDDSaveToGemfireWithConnConf() throws Exception {
+    verifyPairRDDSaveToGemfire(false, false);
+  }
+  
+  public void verifyPairRDDSaveToGemfire(boolean useDefaultConnConf, boolean useOpConf) throws Exception {
+    Region<String, Integer> region = prepareStrIntRegion(regionPath, 0, 0);  // remove all entries
+    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(0, numObjects);
+    Properties opConf = new Properties();
+    opConf.put(RDDSaveBatchSizePropKey, "200");
+
+    if (useDefaultConnConf) {
+      if (useOpConf)
+        javaFunctions(rdd1).saveToGemfire(regionPath, opConf);
+      else
+        javaFunctions(rdd1).saveToGemfire(regionPath);
+    } else {
+      if (useOpConf)
+        javaFunctions(rdd1).saveToGemfire(regionPath, connConf, opConf);
+      else
+        javaFunctions(rdd1).saveToGemfire(regionPath, connConf);
+    }
+
+    Set<String> keys = region.keySetOnServer();
+    Map<String, Integer> map = region.getAll(keys);
+
+    List<Tuple2<String, Integer>> expectedList = new ArrayList<>();
+    for (int i = 0; i < numObjects; i ++) {
+      expectedList.add(new Tuple2<>("k_" + i, i));
+    }
+    matchMapAndPairList(map, expectedList);
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaSparkContext.gemfireRegion and where clause
+  // --------------------------------------------------------------------------------------------
+
+  @Test
+  public void testJavaSparkContextGemfireRegion() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);  // remove all entries
+    Properties emptyProps = new Properties();
+    GemFireJavaRegionRDD<String, Integer> rdd1 = javaFunctions(jsc).gemfireRegion(regionPath);
+    GemFireJavaRegionRDD<String, Integer> rdd2 = javaFunctions(jsc).gemfireRegion(regionPath, emptyProps);
+    GemFireJavaRegionRDD<String, Integer> rdd3 = javaFunctions(jsc).gemfireRegion(regionPath, connConf);
+    GemFireJavaRegionRDD<String, Integer> rdd4 = javaFunctions(jsc).gemfireRegion(regionPath, connConf, emptyProps);
+    GemFireJavaRegionRDD<String, Integer> rdd5 = rdd1.where("value.intValue() < 50");
+
+    HashMap<String, Integer> expectedMap = new HashMap<>();
+    for (int i = 0; i < numObjects; i ++) {
+      expectedMap.put("k_" + i, i);
+    }
+
+    matchMapAndPairList(expectedMap, rdd1.collect());
+    matchMapAndPairList(expectedMap, rdd2.collect());
+    matchMapAndPairList(expectedMap, rdd3.collect());
+    matchMapAndPairList(expectedMap, rdd4.collect());
+
+    HashMap<String, Integer> expectedMap2 = new HashMap<>();
+    for (int i = 0; i < 50; i ++) {
+      expectedMap2.put("k_" + i, i);
+    }
+
+    matchMapAndPairList(expectedMap2, rdd5.collect());
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaPairRDD.joinGemfireRegion
+  // --------------------------------------------------------------------------------------------
+
+  @Test
+  public void testPairRDDJoinWithSameKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(-5, 10);
+
+    JavaPairRDD<Tuple2<String, Integer>, Integer> rdd2a = javaFunctions(rdd1).joinGemfireRegion(regionPath);
+    JavaPairRDD<Tuple2<String, Integer>, Integer> rdd2b = javaFunctions(rdd1).joinGemfireRegion(regionPath, connConf);
+    // System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Tuple2<String, Integer>, Integer> expectedMap = new HashMap<>();
+    for (int i = 0; i < 10; i ++) {
+      expectedMap.put(new Tuple2<>("k_" + i, i), i);
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+  static class IntIntPairToStrKeyFunction implements Function<Tuple2<Integer, Integer>, String> {
+    @Override public String call(Tuple2<Integer, Integer> pair) throws Exception {
+      return "k_" + pair._1();
+    }
+  }
+
+  @Test
+  public void testPairRDDJoinWithDiffKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaPairRDD<Integer, Integer> rdd1 = prepareIntIntJavaPairRDD(-5, 10);
+    Function<Tuple2<Integer, Integer>, String> func = new IntIntPairToStrKeyFunction();
+
+    JavaPairRDD<Tuple2<Integer, Integer>, Integer> rdd2a = javaFunctions(rdd1).joinGemfireRegion(regionPath, func);
+    JavaPairRDD<Tuple2<Integer, Integer>, Integer> rdd2b = javaFunctions(rdd1).joinGemfireRegion(regionPath, func, connConf);
+    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Tuple2<Integer, Integer>, Integer> expectedMap = new HashMap<>();
+    for (int i = 0; i < 10; i ++) {
+      expectedMap.put(new Tuple2<>(i, i * 2), i);
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaPairRDD.outerJoinGemfireRegion
+  // --------------------------------------------------------------------------------------------
+
+  @Test
+  public void testPairRDDOuterJoinWithSameKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(-5, 10);
+
+    JavaPairRDD<Tuple2<String, Integer>, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath);
+    JavaPairRDD<Tuple2<String, Integer>, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath, connConf);
+    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Tuple2<String, Integer>, Option<Integer>> expectedMap = new HashMap<>();
+    for (int i = -5; i < 10; i ++) {
+      if (i < 0)
+        expectedMap.put(new Tuple2<>("k_" + i, i), Option.apply((Integer) null));
+      else
+        expectedMap.put(new Tuple2<>("k_" + i, i), Some.apply(i));
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+  @Test
+  public void testPairRDDOuterJoinWithDiffKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaPairRDD<Integer, Integer> rdd1 = prepareIntIntJavaPairRDD(-5, 10);
+    Function<Tuple2<Integer, Integer>, String> func = new IntIntPairToStrKeyFunction();
+
+    JavaPairRDD<Tuple2<Integer, Integer>, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath, func);
+    JavaPairRDD<Tuple2<Integer, Integer>, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath, func, connConf);
+    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Tuple2<Integer, Integer>, Option<Integer>> expectedMap = new HashMap<>();
+    for (int i = -5; i < 10; i ++) {
+      if (i < 0)
+        expectedMap.put(new Tuple2<>(i, i * 2), Option.apply((Integer) null));
+      else
+        expectedMap.put(new Tuple2<>(i, i * 2), Some.apply(i));
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaRDD.joinGemfireRegion
+  // --------------------------------------------------------------------------------------------
+
+  static class IntToStrKeyFunction implements Function<Integer, String> {
+    @Override public String call(Integer x) throws Exception {
+      return "k_" + x;
+    }
+  }
+
+  @Test
+  public void testRDDJoinWithSameKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(-5, 10);
+
+    Function<Integer, String> func = new IntToStrKeyFunction();
+    JavaPairRDD<Integer, Integer> rdd2a = javaFunctions(rdd1).joinGemfireRegion(regionPath, func);
+    JavaPairRDD<Integer, Integer> rdd2b = javaFunctions(rdd1).joinGemfireRegion(regionPath, func, connConf);
+    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Integer, Integer> expectedMap = new HashMap<>();
+    for (int i = 0; i < 10; i ++) {
+      expectedMap.put(i, i);
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+  // --------------------------------------------------------------------------------------------
+  //   JavaRDD.outerJoinGemfireRegion
+  // --------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRDDOuterJoinWithSameKeyType() throws Exception {
+    prepareStrIntRegion(regionPath, 0, numObjects);
+    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(-5, 10);
+
+    Function<Integer, String> func = new IntToStrKeyFunction();
+    JavaPairRDD<Integer, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath, func);
+    JavaPairRDD<Integer, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGemfireRegion(regionPath, func, connConf);
+    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
+
+    HashMap<Integer, Option<Integer>> expectedMap = new HashMap<>();
+    for (int i = -5; i < 10; i ++) {
+      if (i < 0)
+        expectedMap.put(i, Option.apply((Integer) null));
+      else
+        expectedMap.put(i, Some.apply(i));
+    }
+    matchMapAndPairList(expectedMap, rdd2a.collect());
+    matchMapAndPairList(expectedMap, rdd2b.collect());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Portfolio.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Portfolio.java b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Portfolio.java
new file mode 100644
index 0000000..5fa03c6
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Portfolio.java
@@ -0,0 +1,109 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector;
+
+import java.io.Serializable;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Properties;
+import com.gemstone.gemfire.cache.Declarable;
+
+/**
+ * A stock portfolio that consists of multiple {@link Position} objects that
+ * represent shares of stock (a "security").  Instances of
+ * <code>Portfolio</code> can be stored in a GemFire <code>Region</code> and
+ * their contents can be queried using the GemFire query service.
+ * </p>
+ * This class is <code>Serializable</code> because we want it to be distributed
+ * to multiple members of a distributed system.  Because this class is
+ * <code>Declarable</code>, we can describe instances of it in a GemFire
+ * <code>cache.xml</code> file.
+ * </p>
+ *
+ */
+public class Portfolio implements Declarable, Serializable {
+
+  private static final long serialVersionUID = 9097335119586059309L;
+
+  private int id;  /* id is used as the entry key and is stored in the entry */
+  private String type;
+  private Map<String,Position> positions = new LinkedHashMap<String,Position>();
+  private String status;
+
+  public Portfolio(Properties props) {
+    init(props);
+  }
+
+  @Override
+  public void init(Properties props) {
+    this.id = Integer.parseInt(props.getProperty("id"));
+    this.type = props.getProperty("type", "type1");
+    this.status = props.getProperty("status", "active");
+
+    // get the positions. These are stored in the properties object
+    // as Positions, not String, so use Hashtable protocol to get at them.
+    // the keys are named "positionN", where N is an integer.
+    for (Map.Entry<Object, Object> entry: props.entrySet()) {
+      String key = (String)entry.getKey();
+      if (key.startsWith("position")) {
+        Position pos = (Position)entry.getValue();
+        this.positions.put(pos.getSecId(), pos);
+      }
+    }
+  }
+
+  public void setType(String t) {this.type = t; }
+
+  public String getStatus(){
+    return status;
+  }
+
+  public int getId(){
+    return this.id;
+  }
+
+  public Map<String,Position> getPositions(){
+    return this.positions;
+  }
+
+  public String getType() {
+    return this.type;
+  }
+
+  public boolean isActive(){
+    return status.equals("active");
+  }
+
+  @Override
+  public String toString(){
+    StringBuilder buf = new StringBuilder();
+    buf.append("\n\tPortfolio [id=" + this.id + " status=" + this.status);
+    buf.append(" type=" + this.type);
+    boolean firstTime = true;
+    for (Map.Entry<String, Position> entry: positions.entrySet()) {
+      if (!firstTime) {
+        buf.append(", ");
+      }
+      buf.append("\n\t\t");
+      buf.append(entry.getKey() + ":" + entry.getValue());
+      firstTime = false;
+    }
+    buf.append("]");
+    return buf.toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Position.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Position.java b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Position.java
new file mode 100644
index 0000000..b8e8be9
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Position.java
@@ -0,0 +1,73 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector;
+
+import java.io.Serializable;
+import java.util.Properties;
+import com.gemstone.gemfire.cache.Declarable;
+
+/**
+ * Represents a number of shares of a stock ("security") held in a {@link
+ * Portfolio}.
+ * </p>
+ * This class is <code>Serializable</code> because we want it to be distributed
+ * to multiple members of a distributed system.  Because this class is
+ * <code>Declarable</code>, we can describe instances of it in a GemFire
+ * <code>cache.xml</code> file.
+ * </p>
+ *
+ */
+public class Position implements Declarable, Serializable {
+
+  private static final long serialVersionUID = -8229531542107983344L;
+
+  private String secId;
+  private double qty;
+  private double mktValue;
+
+  public Position(Properties props) {
+    init(props);
+  }
+
+  @Override
+  public void init(Properties props) {
+    this.secId = props.getProperty("secId");
+    this.qty = Double.parseDouble(props.getProperty("qty"));
+    this.mktValue = Double.parseDouble(props.getProperty("mktValue"));
+  }
+
+  public String getSecId(){
+    return this.secId;
+  }
+
+  public double getQty(){
+    return this.qty;
+  }
+
+  public double getMktValue() {
+    return this.mktValue;
+  }
+
+  @Override
+  public String toString(){
+    return new StringBuilder()
+            .append("Position [secId=").append(secId)
+            .append(" qty=").append(this.qty)
+            .append(" mktValue=").append(mktValue).append("]").toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml b/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
new file mode 100644
index 0000000..79893d6
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-regions.xml
@@ -0,0 +1,49 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<!DOCTYPE cache PUBLIC
+  "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
+  "http://www.gemstone.com/dtd/cache6_5.dtd" >
+
+<cache>
+  <!-- test region for OQL test -->
+  <region name="obj_obj_region" refid="PARTITION_REDUNDANT" />
+
+  <region name="obj_obj_rep_region" refid="REPLICATE" />
+
+  <region name="str_int_region" refid="PARTITION_REDUNDANT">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.Integer</value-constraint>
+    </region-attributes>
+  </region>
+
+  <region name="str_str_region" refid="PARTITION_REDUNDANT">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.String</value-constraint>
+    </region-attributes>
+  </region>
+
+  <region name="str_str_rep_region" refid="REPLICATE">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.String</value-constraint>
+    </region-attributes>
+  </region>
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml b/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
new file mode 100644
index 0000000..3023959
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/resources/test-retrieve-regions.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<!DOCTYPE cache PUBLIC
+  "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
+  "http://www.gemstone.com/dtd/cache6_5.dtd" >
+
+<cache>
+  <!-- combinations of key, value types with region types -->
+  <region name="pr_r_obj_obj_region" refid="PARTITION_REDUNDANT" />
+  <region name="pr_obj_obj_region" refid="PARTITION" />
+  <region name="rr_obj_obj_region" refid="REPLICATE" />
+  <region name="rr_p_obj_obj_region" refid="REPLICATE_PERSISTENT" />
+
+  <region name="pr_r_str_int_region" refid="PARTITION_REDUNDANT">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.Integer</value-constraint>
+    </region-attributes>
+  </region>
+  
+  <region name="pr_str_int_region" refid="PARTITION">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.Integer</value-constraint>
+    </region-attributes>
+  </region>
+
+  <region name="rr_str_int_region" refid="REPLICATE">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.Integer</value-constraint>
+    </region-attributes>
+  </region>
+  
+  <region name="rr_p_str_int_region" refid="REPLICATE_PERSISTENT">
+    <region-attributes>
+      <key-constraint>java.lang.String</key-constraint>
+      <value-constraint>java.lang.Integer</value-constraint>
+    </region-attributes>
+  </region>
+</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/BasicIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/BasicIntegrationTest.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/BasicIntegrationTest.scala
new file mode 100644
index 0000000..10c7eaf
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/BasicIntegrationTest.scala
@@ -0,0 +1,598 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector
+
+import java.util.Properties
+import com.gemstone.gemfire.cache.query.QueryService
+import com.gemstone.gemfire.cache.query.internal.StructImpl
+import io.pivotal.gemfire.spark.connector._
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.internal.{RegionMetadata, DefaultGemFireConnectionManager}
+import io.pivotal.gemfire.spark.connector.internal.oql.{RDDConverter, QueryRDD}
+import ittest.io.pivotal.gemfire.spark.connector.testkit.GemFireCluster
+import ittest.io.pivotal.gemfire.spark.connector.testkit.IOUtils
+import org.apache.spark.streaming.{Seconds, StreamingContext, TestInputDStream}
+import org.apache.spark.{SparkContext, SparkConf}
+import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
+import scala.collection.JavaConversions
+import scala.reflect.ClassTag
+
+case class Number(str: String, len: Int)
+
+class BasicIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GemFireCluster {
+
+  var sc: SparkContext = null
+
+  override def beforeAll() {
+    // start gemfire cluster, and spark context
+    val settings = new Properties()
+    settings.setProperty("cache-xml-file", "src/it/resources/test-regions.xml")
+    settings.setProperty("num-of-servers", "2")
+    val locatorPort = GemFireCluster.start(settings)
+
+    // start spark context in local mode
+    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
+                            "log4j.logger.io.pivotal.gemfire.spark.connector" -> "DEBUG")
+    val conf = new SparkConf()
+      .setAppName("BasicIntegrationTest")
+      .setMaster("local[2]")
+      .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+      .set(GemFireLocatorPropKey, s"localhost[$locatorPort]")
+      .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+      .set("spark.kryo.registrator", "io.pivotal.gemfire.spark.connector.GemFireKryoRegistrator")
+
+    sc = new SparkContext(conf)
+  }
+
+  override def afterAll() {
+    // stop connection, spark context, and gemfire cluster
+    DefaultGemFireConnectionManager.closeConnection(GemFireConnectionConf(sc.getConf))
+    sc.stop()
+    GemFireCluster.stop()
+  }
+
+  //Convert Map[Object, Object] to java.util.Properties
+  private def map2Props(map: Map[Object, Object]): java.util.Properties =
+    (new java.util.Properties /: map) {case (props, (k,v)) => props.put(k,v); props}
+
+  // ===========================================================
+  //       DefaultGemFireConnection functional tests
+  // ===========================================================
+
+  test("DefaultGemFireConnection.validateRegion()") {
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+
+    // normal exist-region
+    var regionPath: String = "str_str_region"
+    conn.validateRegion[String, String](regionPath)
+
+    // non-exist region
+    regionPath = "non_exist_region"
+    try {
+      conn.validateRegion[String, String](regionPath)
+      fail("validateRegion failed to catch non-exist region error")
+    } catch {
+      case e: RuntimeException => 
+        if (! e.getMessage.contains(s"The region named $regionPath was not found"))
+          fail("validateRegion gives wrong exception on non-exist region", e)
+      case e: Throwable => 
+        fail("validateRegion gives wrong exception on non-exist region", e)
+    }
+
+    // Note: currently, can't catch type mismatch error
+    conn.validateRegion[String, Integer]("str_str_region")
+  }
+
+  test("DefaultGemFireConnection.getRegionMetadata()") {
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+
+    // exist region
+    validateRegionMetadata(conn, "obj_obj_region", true, 113, null, null, false)
+    validateRegionMetadata(conn, "str_int_region", true, 113, "java.lang.String", "java.lang.Integer", false)
+    validateRegionMetadata(conn, "str_str_rep_region", false, 0, "java.lang.String", "java.lang.String", true)
+
+    // non-exist region
+    assert(! conn.getRegionMetadata("no_exist_region").isDefined)
+  }
+    
+  def validateRegionMetadata(
+    conn: GemFireConnection, regionPath: String, partitioned: Boolean, buckets: Int,
+    keyType: String, valueType: String, emptyMap: Boolean): Unit = {
+
+    val mdOption = conn.getRegionMetadata(regionPath)
+    val md = mdOption.get
+   
+    assert(md.getRegionPath == s"/$regionPath")
+    assert(md.isPartitioned == partitioned)
+    assert(md.getKeyTypeName == keyType)
+    assert(md.getValueTypeName == valueType)
+    assert(md.getTotalBuckets == buckets)
+    if (emptyMap) assert(md.getServerBucketMap == null) 
+    else assert(md.getServerBucketMap != null)
+  }
+
+  test("DefaultGemFireConnection.getRegionProxy()") {
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+
+    val region1 = conn.getRegionProxy[String, String]("str_str_region")
+    region1.put("1", "One")
+    assert(region1.get("1") == "One")
+    region1.remove("1")
+    assert(region1.get("1") == null)
+    
+    // getRegionProxy doesn't fail when region doesn't exist
+    val region2 = conn.getRegionProxy[String, String]("non_exist_region")
+    try {
+      region2.put("1", "One")
+      fail("getRegionProxy failed to catch non-exist region error")
+    } catch {
+      case e: Exception =>
+        if (e.getCause == null || ! e.getCause.getMessage.contains(s"Region named /non_exist_region was not found")) {
+          e.printStackTrace()
+          fail("validateRegion gives wrong exception on non-exist region", e)
+        }
+    }
+  }
+  
+  // Note: DefaultGemFireConnecton.getQuery() and getRegionData() are covered by 
+  //       RetrieveRegionIntegrationTest.scala and following OQL tests.
+  
+  // ===========================================================
+  //                OQL functional tests
+  // ===========================================================
+  
+  private def initRegion(regionName: String): Unit = {
+
+    //Populate some data in the region
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+
+    //This will call the implicit conversion map2Properties in connector package object, since it is Map[String, String]
+    var position1 = new Position(Map("secId" -> "SUN", "qty" -> "34000", "mktValue" -> "24.42"))
+    var position2 = new Position(Map("secId" -> "IBM", "qty" -> "8765", "mktValue" -> "34.29"))
+    val portfolio1 = new Portfolio(map2Props(Map("id" ->"1", "type" -> "type1", "status" -> "active",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("1", portfolio1)
+
+    position1 = new Position(Map("secId" -> "YHOO", "qty" -> "9834", "mktValue" -> "12.925"))
+    position2 = new Position(Map("secId" -> "GOOG", "qty" -> "12176", "mktValue" -> "21.972"))
+    val portfolio2 = new Portfolio(map2Props(Map("id" -> "2", "type" -> "type2", "status" -> "inactive",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("2", portfolio2)
+
+    position1 = new Position(Map("secId" -> "MSFT", "qty" -> "98327", "mktValue" -> "23.32"))
+    position2 = new Position(Map("secId" -> "AOL", "qty" -> "978", "mktValue" -> "40.373"))
+    val portfolio3 = new Portfolio(map2Props(Map("id" -> "3", "type" -> "type3", "status" -> "active",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("3", portfolio3)
+
+    position1 = new Position(Map("secId" -> "APPL", "qty" -> "67", "mktValue" -> "67.356572"))
+    position2 = new Position(Map("secId" -> "ORCL", "qty" -> "376", "mktValue" -> "101.34"))
+    val portfolio4 = new Portfolio(map2Props(Map("id" -> "4", "type" -> "type1", "status" -> "inactive",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("4", portfolio4)
+
+    position1 = new Position(Map("secId" -> "SAP", "qty" -> "90", "mktValue" -> "67.356572"))
+    position2 = new Position(Map("secId" -> "DELL", "qty" -> "376", "mktValue" -> "101.34"))
+    val portfolio5 = new Portfolio(map2Props(Map("id" -> "5", "type" -> "type2", "status" -> "active",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("5", portfolio5)
+
+    position1 = new Position(Map("secId" -> "RHAT", "qty" -> "90", "mktValue" -> "67.356572"))
+    position2 = new Position(Map("secId" -> "NOVL", "qty" -> "376", "mktValue" -> "101.34"))
+    val portfolio6 = new Portfolio(map2Props(Map("id" -> "6", "type" -> "type3", "status" -> "inactive",
+      "position1" -> position1, "position2" -> position2)))
+    rgn.put("6", portfolio6)
+
+    position1 = new Position(Map("secId" -> "MSFT", "qty" -> "98327", "mktValue" -> "23.32"))
+    position2 = new Position(Map("secId" -> "AOL", "qty" -> "978", "mktValue" -> "40.373"))
+    val portfolio7 = new Portfolio(map2Props(Map("id" -> "7", "type" -> "type4", "status" -> "active",
+      "position1" -> position1, "position2" -> position2)))
+    //Not using null, due to intermittent query failure on column containing null, likely a Spark SQL bug
+    //portfolio7.setType(null)
+    rgn.put("7", portfolio7)
+  }
+
+  private def getQueryRDD[T: ClassTag](
+    query: String, connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)): QueryRDD[T] =
+      new QueryRDD[T](sc, query, connConf)
+
+  test("Run GemFire OQL query and convert the returned QueryRDD to DataFrame: Partitioned Region") {
+    simpleQuery("obj_obj_region")
+  }
+
+  test("Run GemFire OQL query and convert the returned QueryRDD to DataFrame: Replicated Region") {
+    simpleQuery("obj_obj_rep_region")
+  }
+
+  private def simpleQuery(regionName: String) {
+    //Populate some data in the region
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, String] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> "one", "2" -> "two", "3" -> "three")))
+
+    //Create QueryRDD using OQL
+    val OQLResult: QueryRDD[String] = getQueryRDD[String](s"select * from /$regionName")
+
+    //verify the QueryRDD
+    val oqlRS: Array[String] = OQLResult.collect()
+    oqlRS should have length 3
+    oqlRS should contain theSameElementsAs List("one", "two", "three")
+
+    //Convert QueryRDD to DataFrame
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    // this is used to implicitly convert an RDD to a DataFrame.
+    import sqlContext.implicits._
+    val dataFrame = OQLResult.map(x => Number(x, x.length)).toDF()
+    //Register dataFrame as a table of two columns of type String and Int respectively
+    dataFrame.registerTempTable("numberTable")
+
+    //Issue SQL query against the table
+    val SQLResult = sqlContext.sql("SELECT * FROM numberTable")
+    //Verify the SQL query result, r(0) mean column 0
+    val sqlRS: Array[Any] = SQLResult.map(r => r(0)).collect()
+    sqlRS should have length 3
+    sqlRS should contain theSameElementsAs List("one", "two", "three")
+
+    //Convert QueryRDD to DataFrame using RDDConverter
+    val dataFrame2 = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
+    //Register dataFrame2 as a table of two columns of type String and Int respectively
+    dataFrame2.registerTempTable("numberTable2")
+
+    //Issue SQL query against the table
+    val SQLResult2 = sqlContext.sql("SELECT * FROM numberTable2")
+    //Verify the SQL query result, r(0) mean column 0
+    val sqlRS2: Array[Any] = SQLResult2.map(r => r(0)).collect()
+    sqlRS2 should have length 3
+    sqlRS2 should contain theSameElementsAs List("one", "two", "three")
+
+    //Remove the region entries, because other tests might use the same region as well
+    List("1", "2", "3").foreach(rgn.remove)
+  }
+
+  test("Run GemFire OQL query and directly return DataFrame: Partitioned Region") {
+    simpleQueryDataFrame("obj_obj_region")
+  }
+
+  test("Run GemFire OQL query and directly return DataFrame: Replicated Region") {
+    simpleQueryDataFrame("obj_obj_rep_region")
+  }
+
+  private def simpleQueryDataFrame(regionName: String) {
+    //Populate some data in the region
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+    val rgn: Region[String, String] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> "one", "2" -> "two", "3" -> "three")))
+
+    //Create DataFrame using GemFire OQL
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"select * from /$regionName")
+    dataFrame.registerTempTable("numberTable")
+
+    //Issue SQL query against the table
+    val SQLResult = sqlContext.sql("SELECT * FROM numberTable")
+    //Verify the SQL query result, r(0) mean column 0
+    val sqlRS: Array[Any] = SQLResult.map(r => r(0)).collect()
+    sqlRS should have length 3
+    sqlRS should contain theSameElementsAs List("one", "two", "three")
+
+    //Remove the region entries, because other tests might use the same region as well
+    List("1", "2", "3").foreach(rgn.remove)
+  }
+
+  test("GemFire OQL query with UDT: Partitioned Region") {
+    queryUDT("obj_obj_region")
+  }
+
+  test("GemFire OQL query with UDT: Replicated Region") {
+    queryUDT("obj_obj_rep_region")
+  }
+
+  private def queryUDT(regionName: String) {
+
+    //Populate some data in the region
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+    val e1: Employee = new Employee("hello", 123)
+    val e2: Employee = new Employee("world", 456)
+    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> e1, "2" -> e2)))
+
+    //Create QueryRDD using OQL
+    val OQLResult: QueryRDD[Object] = getQueryRDD(s"select name, age from /$regionName")
+
+    //verify the QueryRDD
+    val oqlRS: Array[Object] = OQLResult.collect()
+    oqlRS should have length 2
+    oqlRS.map(e => e.asInstanceOf[StructImpl].getFieldValues.apply(1)) should contain theSameElementsAs List(123, 456)
+
+    //Convert QueryRDD to DataFrame
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+    //Convert QueryRDD to DataFrame using RDDConverter
+    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
+    dataFrame.registerTempTable("employee")
+    val SQLResult = sqlContext.sql("SELECT * FROM employee")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.map(r => r(0)).collect()
+    sqlRS should have length 2
+    sqlRS should contain theSameElementsAs List("hello", "world")
+
+    List("1", "2").foreach(rgn.remove)
+  }
+
+  test("GemFire OQL query with UDT and directly return DataFrame: Partitioned Region") {
+    queryUDTDataFrame("obj_obj_region")
+  }
+
+  test("GemFire OQL query with UDT and directly return DataFrame: Replicated Region") {
+    queryUDTDataFrame("obj_obj_rep_region")
+  }
+
+  private def queryUDTDataFrame(regionName: String) {
+    //Populate some data in the region
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+    val e1: Employee = new Employee("hello", 123)
+    val e2: Employee = new Employee("world", 456)
+    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> e1, "2" -> e2)))
+
+    //Create DataFrame using GemFire OQL
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"select name, age from /$regionName")
+
+    dataFrame.registerTempTable("employee")
+    val SQLResult = sqlContext.sql("SELECT * FROM employee")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.map(r => r(0)).collect()
+    sqlRS should have length 2
+    sqlRS should contain theSameElementsAs List("hello", "world")
+
+    List("1", "2").foreach(rgn.remove)
+  }
+
+  test("GemFire OQL query with more complex UDT: Partitioned Region") {
+    complexUDT("obj_obj_region")
+  }
+
+  test("GemFire OQL query with more complex UDT: Replicated Region") {
+    complexUDT("obj_obj_rep_region")
+  }
+
+  private def complexUDT(regionName: String) {
+
+    initRegion(regionName)
+
+    //Create QueryRDD using OQL
+    val OQLResult: QueryRDD[Object] = getQueryRDD(s"SELECT DISTINCT * FROM /$regionName WHERE status = 'active'")
+
+    //verify the QueryRDD
+    val oqlRS: Array[Int] = OQLResult.collect().map(r => r.asInstanceOf[Portfolio].getId)
+    oqlRS should contain theSameElementsAs List(1, 3, 5, 7)
+
+    //Convert QueryRDD to DataFrame
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+    //Convert QueryRDD to DataFrame using RDDConverter
+    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
+
+    dataFrame.registerTempTable("Portfolio")
+
+    val SQLResult = sqlContext.sql("SELECT * FROM Portfolio")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.collect().map(r => r(0).asInstanceOf[Portfolio].getType)
+    sqlRS should contain theSameElementsAs List("type1", "type2", "type3", "type4")
+  }
+
+  test("GemFire OQL query with more complex UDT and directly return DataFrame: Partitioned Region") {
+    complexUDTDataFrame("obj_obj_region")
+  }
+
+  test("GemFire OQL query with more complex UDT and directly return DataFrame: Replicated Region") {
+    complexUDTDataFrame("obj_obj_rep_region")
+  }
+
+  private def complexUDTDataFrame(regionName: String) {
+
+    initRegion(regionName)
+
+    //Create DataFrame using GemFire OQL
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"SELECT DISTINCT * FROM /$regionName WHERE status = 'active'")
+    dataFrame.registerTempTable("Portfolio")
+
+    val SQLResult = sqlContext.sql("SELECT * FROM Portfolio")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.collect().map(r => r(0).asInstanceOf[Portfolio].getType)
+    sqlRS should contain theSameElementsAs List("type1", "type2", "type3", "type4")
+  }
+
+  test("GemFire OQL query with more complex UDT with Projection: Partitioned Region") {
+    queryComplexUDTProjection("obj_obj_region")
+  }
+
+  test("GemFire OQL query with more complex UDT with Projection: Replicated Region") {
+    queryComplexUDTProjection("obj_obj_rep_region")
+  }
+
+  private def queryComplexUDTProjection(regionName: String) {
+
+    initRegion(regionName)
+
+    //Create QueryRDD using OQL
+    val OQLResult: QueryRDD[Object] = getQueryRDD[Object](s"""SELECT id, "type", positions, status FROM /$regionName WHERE status = 'active'""")
+
+    //verify the QueryRDD
+    val oqlRS: Array[Int] = OQLResult.collect().map(si => si.asInstanceOf[StructImpl].getFieldValues.apply(0).asInstanceOf[Int])
+    oqlRS should contain theSameElementsAs List(1, 3, 5, 7)
+
+    //Convert QueryRDD to DataFrame
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+    //Convert QueryRDD to DataFrame using RDDConverter
+    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
+
+    dataFrame.registerTempTable("Portfolio")
+
+    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.collect().map(r => r(0))
+    sqlRS should contain theSameElementsAs List(3)
+  }
+
+  test("GemFire OQL query with more complex UDT with Projection and directly return DataFrame: Partitioned Region") {
+    queryComplexUDTProjectionDataFrame("obj_obj_region")
+  }
+
+  test("GemFire OQL query with more complex UDT with Projection and directly return DataFrame: Replicated Region") {
+    queryComplexUDTProjectionDataFrame("obj_obj_rep_region")
+  }
+
+  private def queryComplexUDTProjectionDataFrame(regionName: String) {
+
+    initRegion(regionName)
+
+    //Create DataFrame using GemFire OQL
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"""SELECT id, "type", positions, status FROM /$regionName WHERE status = 'active'""")
+    dataFrame.registerTempTable("Portfolio")
+
+    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.collect().map(r => r(0))
+    sqlRS should contain theSameElementsAs List(3)
+  }
+
+  test("GemFire OQL query with more complex UDT with nested Projection and directly return DataFrame: Partitioned Region") {
+    queryComplexUDTNestProjectionDataFrame("obj_obj_region")
+  }
+
+  test("GemFire OQL query with more complex UDT with nested Projection and directly return DataFrame: Replicated Region") {
+    queryComplexUDTNestProjectionDataFrame("obj_obj_rep_region")
+  }
+
+  private def queryComplexUDTNestProjectionDataFrame(regionName: String) {
+
+    initRegion(regionName)
+
+    //Create DataFrame using GemFire OQL
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"""SELECT r.id, r."type", r.positions, r.status FROM /$regionName r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'""")
+    dataFrame.registerTempTable("Portfolio")
+
+    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
+
+    //Verify the SQL query result
+    val sqlRS = SQLResult.collect().map(r => r(0))
+    sqlRS should contain theSameElementsAs List(3)
+  }
+
+  test("Undefined instance deserialization: Partitioned Region") {
+    undefinedInstanceDeserialization("obj_obj_region")
+  }
+
+  test("Undefined instance deserialization: Replicated Region") {
+    undefinedInstanceDeserialization("obj_obj_rep_region")
+  }
+
+  private def undefinedInstanceDeserialization(regionName: String) {
+
+    val conn = GemFireConnectionConf(sc.getConf).getConnection
+    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+
+    //Put some new data
+    rgn.put("1", "one")
+
+    //Query some non-existent columns, which should return UNDEFINED
+    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+    val dataFrame = sqlContext.gemfireOQL(s"SELECT col100, col200 FROM /$regionName")
+    val col1 = dataFrame.first().apply(0)
+    val col2 = dataFrame.first().apply(1)
+    assert(col1 == QueryService.UNDEFINED)
+    assert(col2 == QueryService.UNDEFINED)
+    //Verify that col1 and col2 refer to the same Undefined object
+    assert(col1.asInstanceOf[AnyRef] eq col2.asInstanceOf[AnyRef])
+  }
+
+  test("RDD.saveToGemFire") {
+    val regionName = "str_str_region"
+    // generate: Vector((1,11), (2,22), (3,33), (4,44), (5,55), (6,66))
+    val data = (1 to 6).map(_.toString).map(e=> (e, e*2))
+    val rdd = sc.parallelize(data)
+    rdd.saveToGemfire(regionName)
+
+    // verify
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val region: Region[String, String] = connConf.getConnection.getRegionProxy(regionName)
+    println("region key set on server: " + region.keySetOnServer())
+    assert((1 to 6).map(_.toString).toSet == JavaConversions.asScalaSet(region.keySetOnServer()))
+    (1 to 6).map(_.toString).foreach(e => assert(e*2 == region.get(e)))
+  }
+
+  // ===========================================================
+  //        DStream.saveToGemfire() functional tests
+  // ===========================================================
+
+  test("Basic DStream test") {
+    import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListener}
+    import io.pivotal.gemfire.spark.connector.streaming._
+    import org.apache.spark.streaming.ManualClockHelper
+
+    class TestStreamListener extends StreamingListener {
+      var count = 0
+      override def onBatchCompleted(batch: StreamingListenerBatchCompleted) = count += 1
+    }
+
+    def batchDuration = Seconds(1)
+    val ssc = new StreamingContext(sc, batchDuration)
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    val dstream = new TestInputDStream(ssc, input, 2)
+    dstream.saveToGemfire[String, Int]("str_int_region", (e: Int) => (e.toString, e))
+    try {
+      val listener = new TestStreamListener
+      ssc.addStreamingListener(listener)
+      ssc.start()
+      ManualClockHelper.addToTime(ssc, batchDuration.milliseconds * input.length)
+      while (listener.count < input.length) ssc.awaitTerminationOrTimeout(50)
+    } catch {
+      case e: Exception => e.printStackTrace(); throw e
+//    } finally {
+//      ssc.stop()
+    }
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val region: Region[String, Int] = conn.getRegionProxy("str_int_region")
+
+    // verify gemfire region contents
+    println("region key set on server: " + region.keySetOnServer())
+    assert((1 to 12).map(_.toString).toSet == JavaConversions.asScalaSet(region.keySetOnServer()))
+    (1 to 12).foreach(e => assert(e == region.get(e.toString)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RDDJoinRegionIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RDDJoinRegionIntegrationTest.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RDDJoinRegionIntegrationTest.scala
new file mode 100644
index 0000000..c286491
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RDDJoinRegionIntegrationTest.scala
@@ -0,0 +1,300 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector
+
+import java.util.Properties
+
+import io.pivotal.gemfire.spark.connector._
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.internal.DefaultGemFireConnectionManager
+import ittest.io.pivotal.gemfire.spark.connector.testkit.GemFireCluster
+import ittest.io.pivotal.gemfire.spark.connector.testkit.IOUtils
+import org.apache.spark.{SparkContext, SparkConf}
+import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
+import java.util.{HashMap => JHashMap}
+
+class RDDJoinRegionIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GemFireCluster {
+
+  var sc: SparkContext = null
+  val numServers = 3
+  val numObjects = 1000
+
+  override def beforeAll() {
+    // start gemfire cluster, and spark context
+    val settings = new Properties()
+    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml")
+    settings.setProperty("num-of-servers", numServers.toString)
+    val locatorPort = GemFireCluster.start(settings)
+
+    // start spark context in local mode
+    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
+      "log4j.logger.io.pivotal.gemfire.spark.connector" -> "DEBUG")
+    val conf = new SparkConf()
+      .setAppName("RDDJoinRegionIntegrationTest")
+      .setMaster("local[2]")
+      .set(GemFireLocatorPropKey, s"localhost[$locatorPort]")
+    sc = new SparkContext(conf)
+  }
+
+  override def afterAll() {
+    // stop connection, spark context, and gemfire cluster
+    DefaultGemFireConnectionManager.closeConnection(GemFireConnectionConf(sc.getConf))
+    sc.stop()
+    GemFireCluster.stop()
+  }
+
+//  def matchMaps[K,V](map1:Map[K,V], map2:Map[K,V]) = {
+//    assert(map1.size == map2.size)
+//    map1.foreach(e => {
+//      assert(map2.contains(e._1))
+//      assert (e._2 == map2.get(e._1).get)
+//    })
+//  }
+  
+  // -------------------------------------------------------------------------------------------- 
+  // PairRDD.joinGemfireRegion[K2 <: K, V2](regionPath, connConf): GemFireJoinRDD[(K, V), K, V2]  
+  // -------------------------------------------------------------------------------------------- 
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K, V2], replicated region", JoinTest) {
+    verifyPairRDDJoinRegionWithSameKeyType("rr_str_int_region")
+  }
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K, V2], partitioned region", JoinTest) {
+    verifyPairRDDJoinRegionWithSameKeyType("pr_str_int_region")
+  }
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K, V2], partitioned redundant region", JoinTest) {
+    verifyPairRDDJoinRegionWithSameKeyType("pr_r_str_int_region")
+  }
+
+  def verifyPairRDDJoinRegionWithSameKeyType(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => ("k_" + x, x*2))
+    val rdd = sc.parallelize(data)
+
+    val rdd2 = rdd.joinGemfireRegion[String, Int](regionPath, connConf)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (0 until 50).map(i => ((s"k_$i", i*2), i)).toMap
+    // matchMaps[(String, Int), Int](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+
+  // ------------------------------------------------------------------------------------------------------
+  // PairRDD.joinGemfireRegion[K2, V2](regionPath, ((K, V)) => K2, connConf): GemFireJoinRDD[(K, V), K2, V2]
+  // -------------------------------------------------------------------------------------------------------
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K2, V2], replicated region", JoinTest) {
+    verifyPairRDDJoinRegionWithDiffKeyType("rr_str_int_region")
+  }
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K2, V2], partitioned region", JoinTest) {
+    verifyPairRDDJoinRegionWithDiffKeyType("pr_str_int_region")
+  }
+
+  test("PairRDD.joinGemFireRegion: RDD[K, V] with Region[K2, V2], partitioned redundant region", JoinTest) {
+    verifyPairRDDJoinRegionWithDiffKeyType("pr_r_str_int_region")
+  }
+
+  def verifyPairRDDJoinRegionWithDiffKeyType(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => (x, x*2))
+    val rdd = sc.parallelize(data)
+
+    val func :((Int, Int)) => String = pair => s"k_${pair._1}"
+
+    val rdd2 = rdd.joinGemfireRegion[String, Int](regionPath, func /*, connConf*/)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (0 until 50).map(i => ((i, i*2), i)).toMap
+    // matchMaps[(Int, Int), Int](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+
+  // ------------------------------------------------------------------------------------------------ 
+  // PairRDD.outerJoinGemfireRegion[K2 <: K, V2](regionPath, connConf): GemFireJoinRDD[(K, V), K, V2]  
+  // ------------------------------------------------------------------------------------------------ 
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K, V2], replicated region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithSameKeyType("rr_str_int_region")
+  }
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K, V2], partitioned region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithSameKeyType("pr_str_int_region")
+  }
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K, V2], partitioned redundant region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithSameKeyType("pr_r_str_int_region")
+  }
+
+  def verifyPairRDDOuterJoinRegionWithSameKeyType(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => ("k_" + x, x*2))
+    val rdd = sc.parallelize(data)
+
+    val rdd2 = rdd.outerJoinGemfireRegion[String, Int](regionPath /*, connConf*/)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (-5 until 50).map {
+      i => if (i < 0) ((s"k_$i", i * 2), None)
+      else ((s"k_$i", i*2), Some(i))}.toMap
+    // matchMaps[(String, Int), Option[Int]](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+
+  // ------------------------------------------------------------------------------------------------------
+  // PairRDD.joinGemfireRegion[K2, V2](regionPath, ((K, V)) => K2, connConf): GemFireJoinRDD[(K, V), K2, V2]
+  // -------------------------------------------------------------------------------------------------------
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K2, V2], replicated region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithDiffKeyType("rr_str_int_region")
+  }
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K2, V2], partitioned region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithDiffKeyType("pr_str_int_region")
+  }
+
+  test("PairRDD.outerJoinGemFireRegion: RDD[K, V] with Region[K2, V2], partitioned redundant region", OuterJoinTest) {
+    verifyPairRDDOuterJoinRegionWithDiffKeyType("pr_r_str_int_region")
+  }
+
+  def verifyPairRDDOuterJoinRegionWithDiffKeyType(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => (x, x*2))
+    val rdd = sc.parallelize(data)
+
+    val func :((Int, Int)) => String = pair => s"k_${pair._1}"
+
+    val rdd2 = rdd.outerJoinGemfireRegion[String, Int](regionPath, func, connConf)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (-5 until 50).map {
+      i => if (i < 0) ((i, i * 2), None)
+      else ((i, i*2), Some(i))}.toMap
+    // matchMaps[(Int, Int), Option[Int]](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+
+  // --------------------------------------------------------------------------------------------
+  // RDD.joinGemfireRegion[K, V](regionPath, T => K,  connConf): GemFireJoinRDD[T, K, V]
+  // --------------------------------------------------------------------------------------------
+
+  test("RDD.joinGemFireRegion: RDD[T] with Region[K, V], replicated region", JoinTest) {
+    verifyRDDJoinRegion("rr_str_int_region")
+  }
+
+  test("RDD.joinGemFireRegion: RDD[T] with Region[K, V], partitioned region", JoinTest) {
+    verifyRDDJoinRegion("pr_str_int_region")
+  }
+
+  test("RDD.joinGemFireRegion: RDD[T] with Region[K, V], partitioned redundant region", JoinTest) {
+    verifyRDDJoinRegion("pr_r_str_int_region")
+  }
+
+  def verifyRDDJoinRegion(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => s"k_$x")
+    val rdd = sc.parallelize(data)
+
+    val rdd2 = rdd.joinGemfireRegion[String, Int](regionPath, x => x, connConf)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (0 until 50).map(i => (s"k_$i", i)).toMap
+    // matchMaps[String, Int](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+
+  // --------------------------------------------------------------------------------------------
+  // RDD.outerJoinGemfireRegion[K, V](regionPath, T => K, connConf): GemFireJoinRDD[T, K, V]
+  // --------------------------------------------------------------------------------------------
+
+  test("RDD.outerJoinGemFireRegion: RDD[T] with Region[K, V], replicated region", OnlyTest) {
+    verifyRDDOuterJoinRegion("rr_str_int_region")
+  }
+
+  test("RDD.outerJoinGemFireRegion: RDD[T] with Region[K, V], partitioned region", OnlyTest) {
+    verifyRDDOuterJoinRegion("pr_str_int_region")
+  }
+
+  test("RDD.outerJoinGemFireRegion: RDD[T] with Region[K, V], partitioned redundant region", OnlyTest) {
+    verifyRDDOuterJoinRegion("pr_r_str_int_region")
+  }
+
+  def verifyRDDOuterJoinRegion(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val data = (-5 until 50).map(x => s"k_$x")
+    val rdd = sc.parallelize(data)
+
+    val rdd2 = rdd.outerJoinGemfireRegion[String, Int](regionPath, x => x /*, connConf */)
+    val rdd2Content = rdd2.collect()
+
+    val expectedMap = (-5 until 50).map {
+      i => if (i < 0) (s"k_$i", None)
+           else (s"k_$i", Some(i))}.toMap
+    // matchMaps[String, Option[Int]](expectedMap, rdd2Content.toMap)
+    assert(expectedMap == rdd2Content.toMap)
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RetrieveRegionIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RetrieveRegionIntegrationTest.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RetrieveRegionIntegrationTest.scala
new file mode 100644
index 0000000..0ab8110
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/RetrieveRegionIntegrationTest.scala
@@ -0,0 +1,253 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector
+
+import java.util.Properties
+
+import io.pivotal.gemfire.spark.connector._
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.internal.DefaultGemFireConnectionManager
+import ittest.io.pivotal.gemfire.spark.connector.testkit.GemFireCluster
+import ittest.io.pivotal.gemfire.spark.connector.testkit.IOUtils
+import org.apache.spark.{SparkContext, SparkConf}
+import org.scalatest.{Tag, BeforeAndAfterAll, FunSuite, Matchers}
+import java.util.{HashMap => JHashMap}
+
+
+class RetrieveRegionIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GemFireCluster {
+
+  var sc: SparkContext = null
+  val numServers = 4
+  val numObjects = 1000
+
+  override def beforeAll() {
+    // start gemfire cluster, and spark context
+    val settings = new Properties()
+    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml")
+    settings.setProperty("num-of-servers", numServers.toString)
+    val locatorPort = GemFireCluster.start(settings)
+
+    // start spark context in local mode
+    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
+                            "log4j.logger.io.pivotal.gemfire.spark.connector" -> "DEBUG")
+    val conf = new SparkConf()
+      .setAppName("RetrieveRegionIntegrationTest")
+      .setMaster("local[2]")
+      .set(GemFireLocatorPropKey, s"localhost[$locatorPort]")
+    sc = new SparkContext(conf)
+  }
+
+  override def afterAll() {
+    // stop connection, spark context, and gemfire cluster
+    DefaultGemFireConnectionManager.closeConnection(GemFireConnectionConf(sc.getConf))
+    sc.stop()
+    GemFireCluster.stop()
+  }
+  
+  def executeTest[K,V](regionName:String, numObjects:Int, entriesMap:java.util.Map[K,V]) = {
+    //Populate some data in the region
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[K, V] = conn.getRegionProxy(regionName)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+    verifyRetrieveRegion[K,V](regionName, entriesMap)
+  }
+    
+  def verifyRetrieveRegion[K,V](regionName:String, entriesMap:java.util.Map[K,V])  = {
+    val rdd = sc.gemfireRegion(regionName)
+    val collectedObjs = rdd.collect()
+    collectedObjs should have length entriesMap.size
+    import scala.collection.JavaConverters._
+    matchMaps[K,V](entriesMap.asScala.toMap, collectedObjs.toMap)
+  }
+ 
+  def matchMaps[K,V](map1:Map[K,V], map2:Map[K,V]) = {
+    assert(map1.size == map2.size)
+    map1.foreach(e => {
+      assert(map2.contains(e._1))
+      assert (e._2 == map2.get(e._1).get)
+      }
+    )
+  }
+  
+  //Retrieve region for Partitioned Region where some nodes are empty (empty iterator)
+  //This test has to run first...the rest of the tests always use the same num objects
+  test("Retrieve Region for PR where some nodes are empty (Empty Iterator)") {
+    val numObjects = numServers - 1
+    val entriesMap:JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
+    executeTest[String, Int]("rr_str_int_region", numObjects, entriesMap)
+  }
+
+  //Test for retrieving from region containing string key and int value
+  def verifyRetrieveStringStringRegion(regionName:String) = {
+    val entriesMap:JHashMap[String, String] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, "value_" + i))
+    executeTest[String, String](regionName, numObjects, entriesMap)
+  }
+
+  test("Retrieve Region with replicate redundant string string") {
+    verifyRetrieveStringStringRegion("rr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned string string") {
+    verifyRetrieveStringStringRegion("pr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned redundant string string") {
+    verifyRetrieveStringStringRegion("pr_r_obj_obj_region")
+  }
+  
+
+  //Test for retrieving from region containing string key and string value
+  def verifyRetrieveStringIntRegion(regionName:String) = {
+    val entriesMap:JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
+    executeTest[String, Int](regionName, numObjects, entriesMap)
+  }
+
+  test("Retrieve Region with replicate string int region") {
+    verifyRetrieveStringIntRegion("rr_str_int_region")
+  }
+
+  test("Retrieve Region with partitioned string int region") {
+    verifyRetrieveStringIntRegion("pr_str_int_region")
+  }
+
+  test("Retrieve Region with partitioned redundant string int region") {
+    verifyRetrieveStringIntRegion("pr_r_str_int_region")
+  }
+
+  //Tests for retrieving from region containing string key and object value
+  def verifyRetrieveStringObjectRegion(regionName:String) = {
+    val entriesMap:JHashMap[String, Object] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, new Employee("ename" + i, i)))
+    executeTest[String, Object](regionName, numObjects, entriesMap)
+  }
+
+  test("Retrieve Region with replicate string obj") {
+    verifyRetrieveStringObjectRegion("rr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned string obj") {
+    verifyRetrieveStringObjectRegion("pr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned redundant string obj") {
+    verifyRetrieveStringObjectRegion("pr_r_obj_obj_region")
+  }
+
+  //Test for retrieving from region containing string key and map value
+  def verifyRetrieveStringMapRegion(regionName:String) = {
+    val entriesMap:JHashMap[String,JHashMap[String,String]] = new JHashMap()
+    (0 until numObjects).map(i => {
+      val hashMap:JHashMap[String, String] = new JHashMap()
+      hashMap.put("mapKey:" + i, "mapValue:" + i)
+      entriesMap.put("key_" + i, hashMap)
+    })
+    executeTest(regionName, numObjects, entriesMap)
+  }
+
+  test("Retrieve Region with replicate string map region") {
+    verifyRetrieveStringMapRegion("rr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned string map region") {
+    verifyRetrieveStringMapRegion("pr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partitioned redundant string map region") {
+    verifyRetrieveStringMapRegion("pr_r_obj_obj_region")
+  }
+  
+  //Test and helpers specific for retrieving from region containing string key and byte[] value
+  def executeTestWithByteArrayValues[K](regionName:String, numObjects:Int, entriesMap:java.util.Map[K,Array[Byte]]) = {
+    //Populate some data in the region
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[K, Array[Byte]] = conn.getRegionProxy(regionName)
+    rgn.putAll(entriesMap)
+    verifyRetrieveRegionWithByteArrayValues[K](regionName, entriesMap)
+  }
+  
+  def verifyRetrieveRegionWithByteArrayValues[K](regionName:String, entriesMap:java.util.Map[K,Array[Byte]])  = {
+    val rdd = sc.gemfireRegion(regionName)
+    val collectedObjs = rdd.collect()
+    collectedObjs should have length entriesMap.size
+    import scala.collection.JavaConverters._
+    matchByteArrayMaps[K](entriesMap.asScala.toMap, collectedObjs.toMap)
+  }
+  
+  def matchByteArrayMaps[K](map1:Map[K,Array[Byte]], map2:Map[K,Array[Byte]]) = {
+    map1.foreach(e => {
+      assert(map2.contains(e._1))
+      assert (java.util.Arrays.equals(e._2, map2.get(e._1).get))
+      }
+    )
+    assert(map1.size == map2.size)
+
+  }
+  
+  def verifyRetrieveStringByteArrayRegion(regionName:String) = {
+    val entriesMap:JHashMap[String, Array[Byte]] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, Array[Byte](192.toByte, 168.toByte, 0, i.toByte)))
+    executeTestWithByteArrayValues[String](regionName, numObjects, entriesMap)
+  }
+      
+  test("Retrieve Region with replicate region string byte[] region") {
+    verifyRetrieveStringByteArrayRegion("rr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partition region string byte[] region") {
+    verifyRetrieveStringByteArrayRegion("pr_obj_obj_region")
+  }
+
+  test("Retrieve Region with partition redundant region string byte[] region") {
+    verifyRetrieveStringByteArrayRegion("pr_r_obj_obj_region")
+  }
+
+  test("Retrieve Region with where clause on partitioned redundant region", FilterTest) {
+    verifyRetrieveRegionWithWhereClause("pr_r_str_int_region")
+  }
+
+  test("Retrieve Region with where clause on partitioned region", FilterTest) {
+    verifyRetrieveRegionWithWhereClause("pr_str_int_region")
+  }
+
+  test("Retrieve Region with where clause on replicated region", FilterTest) {
+    verifyRetrieveRegionWithWhereClause("rr_str_int_region")
+  }
+
+  def verifyRetrieveRegionWithWhereClause(regionPath: String): Unit = {
+    val entriesMap: JHashMap[String, Int] = new JHashMap()
+    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
+
+    val connConf: GemFireConnectionConf = GemFireConnectionConf(sc.getConf)
+    val conn = connConf.getConnection
+    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
+    rgn.removeAll(rgn.keySetOnServer())
+    rgn.putAll(entriesMap)
+
+    val rdd = sc.gemfireRegion(regionPath).where("value.intValue() < 50")
+    val expectedMap = (0 until 50).map(i => (s"key_$i", i)).toMap
+    val collectedObjs = rdd.collect()
+    // collectedObjs should have length expectedMap.size
+    matchMaps[String, Int](expectedMap, collectedObjs.toMap)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/package.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/package.scala
new file mode 100644
index 0000000..298dc4a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/package.scala
@@ -0,0 +1,29 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark
+
+import org.scalatest.Tag
+
+package object connector {
+
+  object OnlyTest extends Tag("OnlyTest")
+  object FetchDataTest extends Tag("FetchDateTest")
+  object FilterTest extends Tag("FilterTest")
+  object JoinTest extends Tag("JoinTest")
+  object OuterJoinTest extends Tag("OuterJoinTest")  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireCluster.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireCluster.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireCluster.scala
new file mode 100644
index 0000000..d8e07f5
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireCluster.scala
@@ -0,0 +1,47 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector.testkit
+
+import java.util.Properties
+
+trait GemFireCluster {
+  def startGemFireCluster(settings: Properties): Int = {
+    println("=== GemFireCluster start()")
+    GemFireCluster.start(settings)
+  }
+}
+
+object GemFireCluster {
+  private var gemfire: Option[GemFireRunner] = None
+
+  def start(settings: Properties): Int = {
+    gemfire.map(_.stopGemFireCluster()) // Clean up any old running GemFire instances
+    val runner = new GemFireRunner(settings)
+    gemfire = Some(runner)
+    runner.getLocatorPort
+  }
+
+  def stop(): Unit = {
+    println("=== GemFireCluster shutdown: " + gemfire.toString)
+    gemfire match {
+      case None => println("Nothing to shutdown.")
+      case Some(runner) => runner.stopGemFireCluster()
+    }
+    gemfire = None
+    println("=== GemFireCluster shutdown finished.")
+  }
+}


[10/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

This reverts commit 760c6e225a0269bcc80161bb0545db22a130e9b7.
The renaming didn't move the files and instead blew away the geode-functions directory


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

Branch: refs/heads/develop
Commit: ddee87fea9e3ada0f2e05cd50c1fb0d3a9ebba68
Parents: f12ece5
Author: Jason Huynh <hu...@gmail.com>
Authored: Wed Apr 20 10:29:42 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Wed Apr 20 10:31:30 2016 -0700

----------------------------------------------------------------------
 geode-spark-connector/doc/10_demos.md           |  12 +-
 geode-spark-connector/doc/1_building.md         |   6 +-
 geode-spark-connector/doc/2_quick.md            |  26 +-
 geode-spark-connector/doc/3_connecting.md       |  38 +-
 geode-spark-connector/doc/4_loading.md          |  30 +-
 geode-spark-connector/doc/5_rdd_join.md         |  26 +-
 geode-spark-connector/doc/6_save_rdd.md         |  20 +-
 geode-spark-connector/doc/7_save_dstream.md     |  10 +-
 geode-spark-connector/doc/8_oql.md              |   8 +-
 geode-spark-connector/doc/9_java_api.md         |  36 +-
 .../connector/internal/RegionMetadata.java      |  93 +++
 .../gemfirefunctions/QueryFunction.java         |  99 +++
 .../RetrieveRegionFunction.java                 | 208 +++++++
 .../RetrieveRegionMetadataFunction.java         | 118 ++++
 .../StructStreamingResultSender.java            | 219 +++++++
 .../gemfire/spark/connector/Employee.java       |  54 ++
 .../spark/connector/JavaApiIntegrationTest.java | 424 +++++++++++++
 .../gemfire/spark/connector/Portfolio.java      | 109 ++++
 .../gemfire/spark/connector/Position.java       |  73 +++
 .../src/it/resources/test-regions.xml           |  49 ++
 .../src/it/resources/test-retrieve-regions.xml  |  57 ++
 .../spark/connector/BasicIntegrationTest.scala  | 598 +++++++++++++++++++
 .../RDDJoinRegionIntegrationTest.scala          | 300 ++++++++++
 .../RetrieveRegionIntegrationTest.scala         | 253 ++++++++
 .../gemfire/spark/connector/package.scala       |  29 +
 .../connector/testkit/GemFireCluster.scala      |  47 ++
 .../spark/connector/testkit/GemFireRunner.scala | 148 +++++
 .../spark/connector/testkit/IOUtils.scala       |  94 +++
 .../spark/streaming/ManualClockHelper.scala     |  28 +
 .../spark/streaming/TestInputDStream.scala      |  44 ++
 .../javaapi/GemFireJavaDStreamFunctions.java    |  86 +++
 .../GemFireJavaPairDStreamFunctions.java        |  77 +++
 .../javaapi/GemFireJavaPairRDDFunctions.java    | 238 ++++++++
 .../javaapi/GemFireJavaRDDFunctions.java        | 178 ++++++
 .../javaapi/GemFireJavaSQLContextFunctions.java |  49 ++
 .../GemFireJavaSparkContextFunctions.java       |  87 +++
 .../connector/javaapi/GemFireJavaUtil.java      | 122 ++++
 .../spark/connector/GemFireConnection.scala     |  67 +++
 .../spark/connector/GemFireConnectionConf.scala |  73 +++
 .../connector/GemFireConnectionManager.scala    |  31 +
 .../connector/GemFireFunctionDeployer.scala     |  81 +++
 .../connector/GemFireKryoRegistrator.scala      |  29 +
 .../connector/GemFirePairRDDFunctions.scala     | 140 +++++
 .../spark/connector/GemFireRDDFunctions.scala   | 120 ++++
 .../connector/GemFireSQLContextFunctions.scala  |  42 ++
 .../GemFireSparkContextFunctions.scala          |  39 ++
 .../internal/DefaultGemFireConnection.scala     | 164 +++++
 .../DefaultGemFireConnectionManager.scala       |  77 +++
 .../connector/internal/LocatorHelper.scala      | 135 +++++
 .../StructStreamingResultCollector.scala        | 152 +++++
 .../connector/internal/oql/QueryParser.scala    |  58 ++
 .../spark/connector/internal/oql/QueryRDD.scala |  83 +++
 .../internal/oql/QueryResultCollector.scala     |  69 +++
 .../connector/internal/oql/RDDConverter.scala   |  40 ++
 .../connector/internal/oql/RowBuilder.scala     |  38 ++
 .../connector/internal/oql/SchemaBuilder.scala  |  73 +++
 .../internal/oql/UndefinedSerializer.scala      |  46 ++
 .../connector/internal/rdd/GemFireJoinRDD.scala |  67 +++
 .../internal/rdd/GemFireOuterJoinRDD.scala      |  69 +++
 .../internal/rdd/GemFireRDDPartition.scala      |  36 ++
 .../internal/rdd/GemFireRDDPartitioner.scala    |  59 ++
 .../rdd/GemFireRDDPartitionerImpl.scala         |  89 +++
 .../internal/rdd/GemFireRDDWriter.scala         |  82 +++
 .../internal/rdd/GemFireRegionRDD.scala         | 138 +++++
 .../javaapi/GemFireJavaRegionRDD.scala          |  26 +
 .../spark/connector/javaapi/JavaAPIHelper.scala |  53 ++
 .../gemfire/spark/connector/package.scala       |  69 +++
 .../streaming/GemFireDStreamFunctions.scala     |  89 +++
 .../spark/connector/streaming/package.scala     |  32 +
 .../gemfire/spark/connector/JavaAPITest.java    | 163 +++++
 .../connector/GemFireFunctionDeployerTest.scala |  58 ++
 .../DefaultGemFireConnectionManagerTest.scala   |  82 +++
 ...tStreamingResultSenderAndCollectorTest.scala | 254 ++++++++
 .../internal/oql/QueryParserTest.scala          |  83 +++
 .../connector/ConnectorImplicitsTest.scala      |  50 ++
 .../connector/GemFireConnectionConfTest.scala   | 100 ++++
 .../connector/GemFireDStreamFunctionsTest.scala |  79 +++
 .../connector/GemFireRDDFunctionsTest.scala     | 139 +++++
 .../spark/connector/LocatorHelperTest.scala     | 168 ++++++
 .../rdd/GemFireRDDPartitionerTest.scala         | 190 ++++++
 .../connector/rdd/GemFireRegionRDDTest.scala    | 117 ++++
 .../basic-demos/src/main/java/demo/Emp.java     |  95 +++
 .../src/main/java/demo/OQLJavaDemo.java         |  59 ++
 .../src/main/java/demo/PairRDDSaveJavaDemo.java |  86 +++
 .../src/main/java/demo/RDDSaveJavaDemo.java     |  85 +++
 .../src/main/java/demo/RegionToRDDJavaDemo.java |  57 ++
 .../src/main/scala/demo/NetworkWordCount.scala  |  75 +++
 .../pivotal/geode/spark/connector/Employee.java |  54 --
 .../spark/connector/JavaApiIntegrationTest.java | 424 -------------
 .../geode/spark/connector/Portfolio.java        | 109 ----
 .../pivotal/geode/spark/connector/Position.java |  73 ---
 .../src/it/resources/test-regions.xml           |  49 --
 .../src/it/resources/test-retrieve-regions.xml  |  57 --
 .../spark/connector/BasicIntegrationTest.scala  | 598 -------------------
 .../RDDJoinRegionIntegrationTest.scala          | 300 ----------
 .../RetrieveRegionIntegrationTest.scala         | 253 --------
 .../pivotal/geode/spark/connector/package.scala |  29 -
 .../spark/connector/testkit/GeodeCluster.scala  |  47 --
 .../spark/connector/testkit/GeodeRunner.scala   | 148 -----
 .../geode/spark/connector/testkit/IOUtils.scala |  94 ---
 .../spark/streaming/ManualClockHelper.scala     |  28 -
 .../spark/streaming/TestInputDStream.scala      |  44 --
 .../javaapi/GeodeJavaDStreamFunctions.java      |  86 ---
 .../javaapi/GeodeJavaPairDStreamFunctions.java  |  77 ---
 .../javaapi/GeodeJavaPairRDDFunctions.java      | 238 --------
 .../javaapi/GeodeJavaRDDFunctions.java          | 178 ------
 .../javaapi/GeodeJavaSQLContextFunctions.java   |  49 --
 .../javaapi/GeodeJavaSparkContextFunctions.java |  87 ---
 .../spark/connector/javaapi/GeodeJavaUtil.java  | 122 ----
 .../geode/spark/connector/GeodeConnection.scala |  67 ---
 .../spark/connector/GeodeConnectionConf.scala   |  73 ---
 .../connector/GeodeConnectionManager.scala      |  31 -
 .../spark/connector/GeodeFunctionDeployer.scala |  81 ---
 .../spark/connector/GeodeKryoRegistrator.scala  |  29 -
 .../spark/connector/GeodePairRDDFunctions.scala | 140 -----
 .../spark/connector/GeodeRDDFunctions.scala     | 120 ----
 .../connector/GeodeSQLContextFunctions.scala    |  42 --
 .../connector/GeodeSparkContextFunctions.scala  |  39 --
 .../internal/DefaultGeodeConnection.scala       | 164 -----
 .../DefaultGeodeConnectionManager.scala         |  77 ---
 .../connector/internal/LocatorHelper.scala      | 135 -----
 .../StructStreamingResultCollector.scala        | 152 -----
 .../connector/internal/oql/QueryParser.scala    |  58 --
 .../spark/connector/internal/oql/QueryRDD.scala |  83 ---
 .../internal/oql/QueryResultCollector.scala     |  69 ---
 .../connector/internal/oql/RDDConverter.scala   |  40 --
 .../connector/internal/oql/RowBuilder.scala     |  38 --
 .../connector/internal/oql/SchemaBuilder.scala  |  73 ---
 .../internal/oql/UndefinedSerializer.scala      |  46 --
 .../connector/internal/rdd/GeodeJoinRDD.scala   |  67 ---
 .../internal/rdd/GeodeOuterJoinRDD.scala        |  69 ---
 .../internal/rdd/GeodeRDDPartition.scala        |  36 --
 .../internal/rdd/GeodeRDDPartitioner.scala      |  59 --
 .../internal/rdd/GeodeRDDPartitionerImpl.scala  |  89 ---
 .../connector/internal/rdd/GeodeRDDWriter.scala |  82 ---
 .../connector/internal/rdd/GeodeRegionRDD.scala | 138 -----
 .../connector/javaapi/GeodeJavaRegionRDD.scala  |  26 -
 .../spark/connector/javaapi/JavaAPIHelper.scala |  53 --
 .../pivotal/geode/spark/connector/package.scala |  69 ---
 .../streaming/GeodeDStreamFunctions.scala       |  89 ---
 .../spark/connector/streaming/package.scala     |  32 -
 .../geode/spark/connector/JavaAPITest.java      | 163 -----
 .../connector/GeodeFunctionDeployerTest.scala   |  58 --
 .../DefaultGemFireConnectionManagerTest.scala   |  82 ---
 ...tStreamingResultSenderAndCollectorTest.scala | 254 --------
 .../internal/oql/QueryParserTest.scala          |  83 ---
 .../connector/ConnectorImplicitsTest.scala      |  50 --
 .../connector/GeodeConnectionConfTest.scala     | 100 ----
 .../connector/GeodeDStreamFunctionsTest.scala   |  79 ---
 .../spark/connector/GeodeRDDFunctionsTest.scala | 139 -----
 .../spark/connector/LocatorHelperTest.scala     | 168 ------
 .../connector/rdd/GeodeRDDPartitionerTest.scala | 190 ------
 .../connector/rdd/GeodeRegionRDDTest.scala      | 117 ----
 .../basic-demos/src/main/java/demo/Emp.java     |  95 ---
 .../src/main/java/demo/OQLJavaDemo.java         |  59 --
 .../src/main/java/demo/PairRDDSaveJavaDemo.java |  86 ---
 .../src/main/java/demo/RDDSaveJavaDemo.java     |  85 ---
 .../src/main/java/demo/RegionToRDDJavaDemo.java |  57 --
 .../src/main/scala/demo/NetworkWordCount.scala  |  75 ---
 .../project/Dependencies.scala                  |   8 +-
 .../project/GemFireSparkBuild.scala             |  76 +++
 .../project/GeodeSparkBuild.scala               |  76 ---
 geode-spark-connector/project/Settings.scala    |   4 +-
 163 files changed, 8375 insertions(+), 7638 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/10_demos.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/10_demos.md b/geode-spark-connector/doc/10_demos.md
index 4b4777a..30da687 100644
--- a/geode-spark-connector/doc/10_demos.md
+++ b/geode-spark-connector/doc/10_demos.md
@@ -40,21 +40,21 @@ gfsh> create region --name=str_int_region --type=PARTITION --key-constraint=java
 
 And deploy Geode functions required by the Spark Geode Connector:
 ```
-gfsh> deploy --jar=<path to connector project>/geode-functions/target/scala-2.10/geode-functions_2.10-0.5.0.jar
+gfsh> deploy --jar=<path to connector project>/gemfire-functions/target/scala-2.10/gemfire-functions_2.10-0.5.0.jar
 ```
 
 ### Run simple demos
 This section describes how to run `RDDSaveJavaDemo.java`, 
 `PairRDDSaveJavaDemo.java` and `RegionToRDDJavaDemo.java`:
 ```
-export SPARK_CLASSPATH=$CONNECTOR/geode-spark-connector/target/scala-2.10/geode-spark-connector_2.10-0.5.0.jar:$GEODE/lib/server-dependencies.jar
+export SPARK_CLASSPATH=$CONNECTOR/gemfire-spark-connector/target/scala-2.10/gemfire-spark-connector_2.10-0.5.0.jar:$GEODE/lib/server-dependencies.jar
 
 cd <spark 1.3 dir>
-bin/spark-submit --master=local[2] --class demo.RDDSaveJavaDemo $CONNECTOR/geode-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
+bin/spark-submit --master=local[2] --class demo.RDDSaveJavaDemo $CONNECTOR/gemfire-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
 
-bin/spark-submit --master=local[2] --class demo.PairRDDSaveJavaDemo $CONNECTOR/geode-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
+bin/spark-submit --master=local[2] --class demo.PairRDDSaveJavaDemo $CONNECTOR/gemfire-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
 
-bin/spark-submit --master=local[2] --class demo.RegionToRDDJavaDemo $CONNECTOR/geode-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
+bin/spark-submit --master=local[2] --class demo.RegionToRDDJavaDemo $CONNECTOR/gemfire-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar locatorHost[port]
 ```
 
 ### Run stateful network word count
@@ -67,7 +67,7 @@ $ nc -lk 9999
 
 **Terminal-2**, start word count Spark app: 
 ```
-bin/spark-submit --master=local[2] demo.NetworkWordCount $CONNECTOR/geode-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar localhost 9999 locatorHost:port`
+bin/spark-submit --master=local[2] demo.NetworkWordCount $CONNECTOR/gemfire-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar localhost 9999 locatorHost:port`
 ```
 
 Switch to Terminal-1, type some words, and hit `enter` or `return` key, then check word count at **Terminal-3**, which has `gfsh` connected to the Geode cluster:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/1_building.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/1_building.md b/geode-spark-connector/doc/1_building.md
index 47b17db..9b4d69e 100644
--- a/geode-spark-connector/doc/1_building.md
+++ b/geode-spark-connector/doc/1_building.md
@@ -17,9 +17,9 @@ sbt clean package
 ```
 
 The following jar files will be created:
- - `geode-spark-connector/target/scala-2.10/geode-spark-connector_2.10-0.5.0.jar`
- - `geode-functions/target/scala-2.10/geode-functions_2.10-0.5.0.jar`
- - `geode-spark-demos/target/scala-2.10/geode-spark-demos_2.10-0.5.0.jar `
+ - `gemfire-spark-connector/target/scala-2.10/gemfire-spark-connector_2.10-0.5.0.jar`
+ - `gemfire-functions/target/scala-2.10/gemfire-functions_2.10-0.5.0.jar`
+ - `gemfire-spark-demos/target/scala-2.10/gemfire-spark-demos_2.10-0.5.0.jar `
 
 ### Testing
 Commands to run unit and integration tests:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/2_quick.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/2_quick.md b/geode-spark-connector/doc/2_quick.md
index 01f3c06..ec331c3 100644
--- a/geode-spark-connector/doc/2_quick.md
+++ b/geode-spark-connector/doc/2_quick.md
@@ -15,7 +15,7 @@ is a good starting point.
 You need 2 terminals to follow along, one for Geode shell `gfsh`, and one for Spark shell. Set up Jdk 1.7 on both of them.
 
 ### Geode `gfsh` terminal
-In this terminal, start Geode cluster, deploy Spark Geode Connector's geode-function jar, and create demo regions.
+In this terminal, start Geode cluster, deploy Spark Geode Connector's gemfire-function jar, and create demo regions.
 
 Set up environment variables:
 ```
@@ -41,9 +41,9 @@ gfsh>create region --name=str_str_region --type=PARTITION --key-constraint=java.
 gfsh>create region --name=int_str_region --type=PARTITION --key-constraint=java.lang.Integer --value-constraint=java.lang.String
 ```
 
-Deploy Spark Geode Connector's geode-function jar (`geode-functions_2.10-0.5.0.jar`):
+Deploy Spark Geode Connector's gemfire-function jar (`gemfire-functions_2.10-0.5.0.jar`):
 ```
-gfsh>deploy --jar=<path to connector project>/geode-functions/target/scala-2.10/geode-functions_2.10-0.5.0.jar
+gfsh>deploy --jar=<path to connector project>/gemfire-functions/target/scala-2.10/gemfire-functions_2.10-0.5.0.jar
 ```
 
 ### Spark shell terminal
@@ -52,7 +52,7 @@ In this terminal, setup Spark environment, and start Spark shell.
 Set Geode locator property in Spark configuration: add 
 following to `<spark-dir>/conf/spark-defaults.conf`:
 ```
-spark.geode.locators=localhost[55221]
+spark.gemfire.locators=localhost[55221]
 ```
 Note:
  - if the file doesn't exist, create one. 
@@ -69,20 +69,20 @@ under the same directory to `log4j.properties` and update the file.
 
 Start spark-shell:
 ```
-bin/spark-shell --master local[*] --jars $CONNECTOR/geode-spark-connector/target/scala-2.10/geode-spark-connector_2.10-0.5.0.jar,$GEODE/lib/server-dependencies.jar
+bin/spark-shell --master local[*] --jars $CONNECTOR/gemfire-spark-connector/target/scala-2.10/gemfire-spark-connector_2.10-0.5.0.jar,$GEODE/lib/server-dependencies.jar
 ```
 
 Check Geode locator property in the Spark shell:
 ```
-scala> sc.getConf.get("spark.geode.locators")
+scala> sc.getConf.get("spark.gemfire.locators")
 res0: String = localhost[55221]
 ```
 
 In order to enable Geode specific functions, you need to import 
-`io.pivotal.geode.spark.connector._`
+`io.pivotal.gemfire.spark.connector._`
 ```
-scala> import io.pivotal.geode.spark.connector._
-import io.pivotal.geode.spark.connector._
+scala> import io.pivotal.gemfire.spark.connector._
+import io.pivotal.gemfire.spark.connector._
 ```
 
 ### Save Pair RDD to Geode
@@ -153,16 +153,16 @@ NEXT_STEP_NAME : END
 ### Expose Geode Region As RDD
 The same API is used to expose both replicated and partitioned region as RDDs. 
 ```
-scala> val rdd = sc.geodeRegion[String, String]("str_str_region")
-rdd: io.pivotal.geode.spark.connector.rdd.GemFireRDD[String,String] = GemFireRDD[2] at RDD at GemFireRDD.scala:19
+scala> val rdd = sc.gemfireRegion[String, String]("str_str_region")
+rdd: io.pivotal.gemfire.spark.connector.rdd.GemFireRDD[String,String] = GemFireRDD[2] at RDD at GemFireRDD.scala:19
 
 scala> rdd.foreach(println)
 (1,one)
 (3,three)
 (2,two)
 
-scala> val rdd2 = sc.geodeRegion[Int, String]("int_str_region")
-rdd2: io.pivotal.geode.spark.connector.rdd.GemFireRDD[Int,String] = GemFireRDD[3] at RDD at GemFireRDD.scala:19
+scala> val rdd2 = sc.gemfireRegion[Int, String]("int_str_region")
+rdd2: io.pivotal.gemfire.spark.connector.rdd.GemFireRDD[Int,String] = GemFireRDD[3] at RDD at GemFireRDD.scala:19
 
 scala> rdd2.foreach(println)
 (2,ab)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/3_connecting.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/3_connecting.md b/geode-spark-connector/doc/3_connecting.md
index 1a4dadd..8972be9 100644
--- a/geode-spark-connector/doc/3_connecting.md
+++ b/geode-spark-connector/doc/3_connecting.md
@@ -2,48 +2,48 @@
 
 There are two ways to connect Spark to Geode:
  - Specify Geode connection properties via `SparkConf`.
- - Specify Geode connection properties via `GeodeConnectionConf`.
+ - Specify Geode connection properties via `GemFireConnectionConf`.
 
 ### Specify Geode connection properties via `SparkConf`
-The only required Geode connection property is `spark.geode.locators`. 
+The only required Geode connection property is `spark.gemfire.locators`. 
 This can be specified in `<spark dir>/conf/spark-defaults.conf` or in Spark 
 application code. In the following examples, we assume you want to provide
 3 extra properties: `security-client-auth-init`, `security-username`, and 
-`security-password`, note that they are prefixed with `spark.geode.`.
+`security-password`, note that they are prefixed with `spark.gemfire.`.
  
 In `<spark dir>/conf/spark-defaults.com`
 ```
-spark.geode.locators=192.168.1.47[10334]
-spark.geode.security-client-auth-init=com.gemstone.geode.security.templates.UserPasswordAuthInit.create
-spark.geode.security-username=scott
-spark.geode.security-password=tiger
+spark.gemfire.locators=192.168.1.47[10334]
+spark.gemfire.security-client-auth-init=com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create
+spark.gemfire.security-username=scott
+spark.gemfire.security-password=tiger
 ```
  
 Or in the Spark application code:
 ```
-import io.pivotal.geode.spark.connector._
+import io.pivotal.gemfire.spark.connector._
 val sparkConf = new SparkConf()
-  .set(GeodeLocatorPropKey, "192.168.1.47[10334]")
-  .set("spark.geode.security-client-auth-init", "com.gemstone.geode.security.templates.UserPasswordAuthInit.create")
-  .set("spark.geode.security-username", "scott")
-  .set("spark.geode.security-password", "tiger")
+  .set(GemFireLocatorPropKey, "192.168.1.47[10334]")
+  .set("spark.gemfire.security-client-auth-init", "com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create")
+  .set("spark.gemfire.security-username", "scott")
+  .set("spark.gemfire.security-password", "tiger")
 ```
 
-After this, you can use all connector APIs without providing `GeodeConnectionConf`.
+After this, you can use all connector APIs without providing `GemfireConnectionConf`.
  
-### Specify Geode connection properties via `GeodeConnectionConf`
-Here's the code that creates `GeodeConnectionConf` with the same set of 
+### Specify Geode connection properties via `GemFireConnectionConf`
+Here's the code that creates `GemFireConnectionConf` with the same set of 
 properties as the examples above:
 ```
-val props = Map("security-client-auth-init" -> "com.gemstone.geode.security.templates.UserPasswordAuthInit.create",
+val props = Map("security-client-auth-init" -> "com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create",
                 "security-username" -> "scott",
                 "security-password" -> "tiger")
-val connConf = GeodeConnectionConf("192.168.1.47[10334]", props)
+val connConf = GemFireConnectionConf("192.168.1.47[10334]", props)
 ``` 
 
-Please note that those properties are **not** prefixed with `spark.geode.`.
+Please note that those properties are **not** prefixed with `spark.gemfire.`.
 
-After this, you can use all connector APIs that require `GeodeConnectionConf`.
+After this, you can use all connector APIs that require `GemFireConnectionConf`.
 
 ### Notes about locators
  - You can specify locator in two formats: `host[port]` or `host:port`. For

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/4_loading.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/4_loading.md b/geode-spark-connector/doc/4_loading.md
index d6789dd..b67a96e 100644
--- a/geode-spark-connector/doc/4_loading.md
+++ b/geode-spark-connector/doc/4_loading.md
@@ -1,16 +1,16 @@
 ## Loading Data from Geode
 
 To expose full data set of a Geode region as a Spark
-RDD, call `geodeRegion` method on the SparkContext object.
+RDD, call `gemfireRegion` method on the SparkContext object.
 
 ```
-val rdd = sc.geodeRegion("region path")
+val rdd = sc.gemfireRegion("region path")
 ```
 
-Or with specific `GeodeConectionConf` object instance (see 
-[Connecting to  Geode](3_connecting.md) for how to create GeodeConectionConf):
+Or with specific `GemfireConectionConf` object instance (see 
+[Connecting to  Geode](3_connecting.md) for how to create GemfireConectionConf):
 ```
-val rdd = sc.geodeRegion("region path", connConf)
+val rdd = sc.gemfireRegion("region path", connConf)
 ```
 
 ## Geode RDD Partitions
@@ -22,24 +22,24 @@ upon multiple servers, and may have duplicates for high
 availability.
 
 Since replicated region has its full dataset available on every
-server, there is only one RDD partition for a `GeodeRegionRDD` that 
+server, there is only one RDD partition for a `GemFireRegionRDD` that 
 represents a replicated region.
 
-For a `GeodeRegionRDD` that represents a partitioned region, there are 
+For a `GemFireRegionRDD` that represents a partitioned region, there are 
 many potential  ways to create RDD partitions. So far, we have 
 implemented ServerSplitsPartitioner, which will split the bucket set
 on each Geode server into two RDD partitions by default.
 The number of splits is configurable, the following shows how to set 
 three partitions per Geode server:
 ```
-import io.pivotal.geode.spark.connector._
+import io.pivotal.gemfire.spark.connector._
 
 val opConf = Map(PreferredPartitionerPropKey -> ServerSplitsPartitionerName,
                  NumberPartitionsPerServerPropKey -> "3")
 
-val rdd1 = sc.geodeRegion[String, Int]("str_int_region", opConf = opConf)
+val rdd1 = sc.gemfireRegion[String, Int]("str_int_region", opConf = opConf)
 // or
-val rdd2 = sc.geodeRegion[String, Int]("str_int_region", connConf, opConf)  
+val rdd2 = sc.gemfireRegion[String, Int]("str_int_region", connConf, opConf)  
 ```
 
 
@@ -48,7 +48,7 @@ Server-side filtering allow exposing partial dataset of a Geode region
 as a RDD, this reduces the amount of data transferred from Geode to 
 Spark to speed up processing.
 ```
-val rdd = sc.geodeRegion("<region path>").where("<where clause>")
+val rdd = sc.gemfireRegion("<region path>").where("<where clause>")
 ```
 
 The above call is translated to OQL query `select key, value from /<region path>.entries where <where clause>`, then 
@@ -59,7 +59,7 @@ In the following demo, javabean class `Emp` is used, it has 5 attributes: `id`,
 In order to make `Emp` class available on Geode servers, we need to deploy a jar file that contains `Emp` class, 
 now build the `emp.jar`,  deploy it and create region `emps` in `gfsh`:
 ```
-zip $CONNECTOR/geode-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar \
+zip $CONNECTOR/gemfire-spark-demos/basic-demos/target/scala-2.10/basic-demos_2.10-0.5.0.jar \
   -i "demo/Emp.class" --out $CONNECTOR/emp.jar
   
 gfsh
@@ -73,7 +73,7 @@ only contains `Emp.class`.
 Now in Spark shell, generate some random `Emp` records, and save them to region `emps` (remember to add `emp.jar` to 
 Spark shell classpath before starting Spark shell):
 ```
-import io.pivotal.geode.spark.connector._
+import io.pivotal.gemfire.spark.connector._
 import scala.util.Random
 import demo.Emp
 
@@ -84,12 +84,12 @@ def rpick(xs: List[String]): String = xs(Random.nextInt(xs.size))
 
 val d1 = (1 to 20).map(x => new Emp(x, rpick(lnames), rpick(fnames), 20+Random.nextInt(41), rpick(locs))).toArray
 val rdd1 = sc.parallelize(d1) 
-rdd1.saveToGeode("emps", e => (e.getId, e))
+rdd1.saveToGemfire("emps", e => (e.getId, e))
 ```
 
 Now create a RDD that contains all employees whose age is less than 40, and display its contents:
 ```
-val rdd1s = sc.geodeRegion("emps").where("value.getAge() < 40")
+val rdd1s = sc.gemfireRegion("emps").where("value.getAge() < 40")
 
 rdd1s.foreach(println)
 (5,Emp(5, Taylor, Robert, 32, FL))

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/5_rdd_join.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/5_rdd_join.md b/geode-spark-connector/doc/5_rdd_join.md
index 5c0d6c8..81be061 100644
--- a/geode-spark-connector/doc/5_rdd_join.md
+++ b/geode-spark-connector/doc/5_rdd_join.md
@@ -2,7 +2,7 @@
 
 The Spark Geode Connector suports using any RDD as a source
 of a join and outer join with a Geode region through APIs
-`joinGeodeRegion[K, V]` and `outerJoinGeodeRegion[K, V]`. 
+`joinGemfireRegion[K, V]` and `outerJoinGemfireRegion[K, V]`. 
 Those two APIs execute a single `region.getAll` call for every 
 partition of the source RDD, so no unnecessary data will be requested
 or transferred. This means a join or outer join between any RDD and
@@ -20,8 +20,8 @@ following examples.
 ### RDD[(K, V1)] join and outer join Region[K, V2]
 
 In this case, the source RDD is a pair RDD,  and it has the same key
-type as the Region. Use API `rdd.joinGeodeRegion[K, V2](regionPath)` and 
-`rdd.outerJoinGeodeRegion[K, V2](regionPath)` do the join and outer
+type as the Region. Use API `rdd.joinGemfireRegion[K, V2](regionPath)` and 
+`rdd.outerJoinGemfireRegion[K, V2](regionPath)` do the join and outer
 join. 
 
 Prepare a source RDD `rdd2`:
@@ -49,7 +49,7 @@ rdd2.foreach(println)
 
 Join RDD `rdd2` with region `emps`, and print out the result:
 ```
-val rdd2j = rdd2.joinGeodeRegion[Int, Emp]("emps")
+val rdd2j = rdd2.joinGemfireRegion[Int, Emp]("emps")
 
 rdd2j.foreach(println)
 ((11,message-11),Emp(11, Taylor, Emma, 44, CA))
@@ -69,7 +69,7 @@ entries have those key values.
 
 Outer join RDD `rdd2` with region `emps`, and print out the result:
 ```
-val rdd2o = rdd2.outerJoinGeodeRegion[Int, Emp]("emps")
+val rdd2o = rdd2.outerJoinGemfireRegion[Int, Emp]("emps")
 
 rdd2o.foreach(println)
 ((18,message-18),Some(Emp(18, Powell, Alice, 58, FL)))
@@ -95,8 +95,8 @@ since there's no region entries have those key values.
 ### RDD[(K1, V1)] join and outer join Region[K2, V2]
 
 In this case, the source RDD is still a pair RDD,  but it has different
-key type. Use API `rdd.joinGeodeRegion[K2, V2](regionPath, func)` and 
-`rdd.outerJoinGeodeRegion[K2, V2](regionPath, func)` do the join and 
+key type. Use API `rdd.joinGemfireRegion[K2, V2](regionPath, func)` and 
+`rdd.outerJoinGemfireRegion[K2, V2](regionPath, func)` do the join and 
 outer join, where `func` is the function to generate key from (k, v)
 pair, the element of source RDD, to join with Geode region.
 
@@ -125,7 +125,7 @@ rdd3.foreach(println)
 
 Join RDD `rdd3` (RDD[(String, Int)] with region `emps` (Region[Int, Emp]), and print out the result:
 ```
-val rdd3j = rdd3.joinGeodeRegion[Int, Emp]("emps", pair => pair._2)
+val rdd3j = rdd3.joinGemfireRegion[Int, Emp]("emps", pair => pair._2)
 
 rdd3j.foreach(println)
 ((message-18,18),Emp(18, Powell, Alice, 58, FL))
@@ -144,7 +144,7 @@ RDD and join key.
 
 Outer join RDD `rdd3` with region `emps`, and print out the result:
 ```
-val rdd3o = rdd3.outerJoinGeodeRegion[Int, Emp]("emps", pair => pair._2)
+val rdd3o = rdd3.outerJoinGemfireRegion[Int, Emp]("emps", pair => pair._2)
 
 rdd3o.foreach(println)
 ((message-18,18),Some(Emp(18, Powell, Alice, 58, FL)))
@@ -166,8 +166,8 @@ rdd3o.foreach(println)
 
 ### RDD[T] join and outer join Region[K, V]
 
-Use API `rdd.joinGeodeRegion[K, V](regionPath, func)` and 
-`rdd.outerJoinGeodeRegion[K, V](regionPath, func)` do the join
+Use API `rdd.joinGemfireRegion[K, V](regionPath, func)` and 
+`rdd.outerJoinGemfireRegion[K, V](regionPath, func)` do the join
 and outer join, where `func` is the function to generate key from
 `t`, the element of source RDD, to join with Geode region.
 
@@ -196,7 +196,7 @@ rdd4.foreach(println)
 
 Join RDD `d4` with region `emps`, and print out the result:
 ```
-val rdd4j = rdd4.joinGeodeRegion[Int, Emp]("emps", x => x/2)
+val rdd4j = rdd4.joinGemfireRegion[Int, Emp]("emps", x => x/2)
 
 rdd4j.foreach(println)
 (22,Emp(11, Taylor, Emma, 44, CA))
@@ -213,7 +213,7 @@ rdd4j.foreach(println)
 
 Outer join RDD `d4` with region `emps`, and print out the result:
 ```
-val rdd4o = rdd4.outerJoinGeodeRegion[Int, Emp]("emps", x => x/2)
+val rdd4o = rdd4.outerJoinGemfireRegion[Int, Emp]("emps", x => x/2)
 
 rdd4o.foreach(println)
 (36,Some(Emp(18, Powell, Alice, 58, FL)))

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/6_save_rdd.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/6_save_rdd.md b/geode-spark-connector/doc/6_save_rdd.md
index 73fd8b0..004ef62 100644
--- a/geode-spark-connector/doc/6_save_rdd.md
+++ b/geode-spark-connector/doc/6_save_rdd.md
@@ -7,7 +7,7 @@ It is possible to save any RDD to a Geode region. The requirements are:
  - the target region exists.
 
 To save an RDD to an existing Geode region, import 
-`io.pivotal.geode.spark.connector._` and call the `saveToGeode` 
+`io.pivotal.gemfire.spark.connector._` and call the `saveToGemfire` 
 method on RDD.
 
 ### Save RDD[(K, V)] to Geode
@@ -15,13 +15,13 @@ For pair RDD, i.e., RDD[(K, V)], the pair is treated as key/value pair.
 ```
 val data = Array(("1","one"),("2","two"),("3","three"))
 val rdd = sc.parallelize(data)
-rdd.saveToGeode("str_str_region")
+rdd.saveToGemfire("str_str_region")
 ```
 
-If you create GeodeConnectionConf as described in 
+If you create GemFireConnectionConf as described in 
 [Connecting to Geode](3_connecting.md), the last statement becomes:
 ```
-rdd.saveToGeode("str_str_region", connConf)
+rdd.saveToGemFire("str_str_region", connConf)
 ```
 
 You can verify the region contents:
@@ -52,27 +52,27 @@ then the pair is save to Geode.
 ```
 val data2 = Array("a","ab","abc")
 val rdd2 = sc.parallelize(data2)
-rdd2.saveToGeode("str_int_region", e => (e, e.length))
-// or use GeodeConnectionConf object directly
-// rdd2.saveToGeode("rgnb", e => (e, e.length), connConf)
+rdd2.saveToGemfire("str_int_region", e => (e, e.length))
+// or use GemFireConnectionConf object directly
+// rdd2.saveToGemfire("rgnb", e => (e, e.length), connConf)
 ```
 
 ### `rdd.save.batch.size` 
 
-The connector invokes Geode API `putAll()` to save the data. To make
+The connector invokes GemFire API `putAll()` to save the data. To make
 `putAll()` more efficient, the connector invokes putAll() for every 
 10,000 entries by default. This batch size can be changed with optional
 parameter `opConf`. The following shows how to do it:
 
 ```
   // in Scala
-  rdd.saveToGeode(regionPath, opConf = Map(RDDSaveBatchSizePropKey -> "5000"))
+  rdd.saveToGemfire(regionPath, opConf = Map(RDDSaveBatchSizePropKey -> "5000"))
 
   // in Java
   Properties opConf = new Properties();
   opConf.put(RDDSaveBatchSizePropKey, "5000");
   ...
-  javaFunctions(rdd).saveToGeode(regionPath, opConf); 
+  javaFunctions(rdd).saveToGemfire(regionPath, opConf); 
    
   // note: RDDSaveBatchSizePropKey = "rdd.save.batch.size" 
 ```

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/7_save_dstream.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/7_save_dstream.md b/geode-spark-connector/doc/7_save_dstream.md
index a0019c6..ecc793b 100644
--- a/geode-spark-connector/doc/7_save_dstream.md
+++ b/geode-spark-connector/doc/7_save_dstream.md
@@ -48,16 +48,16 @@ ssc.awaitTermination() // Wait for the computation to terminate
 
 #### Spark Streaming With Geode
 Now let's save the running word count to Geode region `str_int_region`, which 
-simply replace print() with saveToGeode():
+simply replace print() with saveToGemfire():
 
 ```
-import io.pivotal.geode.spark.connector.streaming._
-runningCounts.saveToGeode("str_int_region")
+import io.pivotal.gemfire.spark.connector.streaming._
+runningCounts.saveToGemfire("str_int_region")
 ```
 
-You can use the version of saveToGeode that has the parameter `GeodeConnectionConf`:
+You can use the version of saveToGemfire that has the parameter `GemFireConnectionConf`:
 ```
-runningCounts.saveToGeode("str_int_region", connConf)
+runningCounts.saveToGemfire("str_int_region", connConf)
 ```
 
 See [Spark Streaming Programming Guide]

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/8_oql.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/8_oql.md b/geode-spark-connector/doc/8_oql.md
index ef32ef6..f409698 100644
--- a/geode-spark-connector/doc/8_oql.md
+++ b/geode-spark-connector/doc/8_oql.md
@@ -11,7 +11,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc)
 
 Create a `DataFrame` using OQL:
 ```
-val dataFrame = sqlContext.geodeOQL("SELECT * FROM /CustomerRegion WHERE status = 'active'")
+val dataFrame = sqlContext.gemfireOQL("SELECT * FROM /CustomerRegion WHERE status = 'active'")
 ```
 
 You can repartition the `DataFrame` using `DataFrame.repartition()` if needed. 
@@ -33,9 +33,9 @@ If KryoSerializer is preferred, as described in [Spark Documentation]
 val conf = new SparkConf()
   .setAppName("MySparkApp")
   .setMaster("local[*]")
-  .set(GeodeLocatorPropKey, "localhost[55221]")
+  .set(GemFireLocatorPropKey, "localhost[55221]")
   .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-  .set("spark.kryo.registrator", "io.pivotal.geode.spark.connector.GeodeKryoRegistrator")
+  .set("spark.kryo.registrator", "io.pivotal.gemfire.spark.connector.GemFireKryoRegistrator")
 ```
 
 and register the classes (optional)
@@ -46,7 +46,7 @@ conf.registerKryoClasses(Array(classOf[MyClass1], classOf[MyClass2]))
 Use the following options to start Spark shell:
 ```
  --conf spark.serializer=org.apache.spark.serializer.KryoSerializer
- --conf spark.kryo.registrator=io.pivotal.geode.spark.connector.GeodeKryoRegistrator
+ --conf spark.kryo.registrator=io.pivotal.gemfire.spark.connector.GemFireKryoRegistrator
 ```
 
 ## References

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/doc/9_java_api.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/9_java_api.md b/geode-spark-connector/doc/9_java_api.md
index 6fbc636..b9ac91e 100644
--- a/geode-spark-connector/doc/9_java_api.md
+++ b/geode-spark-connector/doc/9_java_api.md
@@ -6,25 +6,25 @@ understand how the Spark Geode Connector works.
 
 ### Prerequisites
 The best way to use the Spark Geode Connector Java API is to statically
-import all of the methods in `GeodeJavaUtil`. This utility class is
+import all of the methods in `GemFireJavaUtil`. This utility class is
 the main entry point for Spark Geode Connector Java API.
 ```
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
 ```
 
 Create JavaSparkContext (don't forget about the static import):
 ```
 SparkConf conf = new SparkConf();
-conf.set(GeodeLocatorPropKey, "192.168.1.47[10334]")
+conf.set(GemFireLocatorPropKey, "192.168.1.47[10334]")
 JavaSparkContext jsc = new JavaSparkContext(conf);
 ```
 
 ### Accessing Geode region in Java
-Geode region is exposed as `GeodeJavaRegionRDD<K,V>`(subclass of
+Geode region is exposed as `GemFireJavaRegionRDD<K,V>`(subclass of
 `JavaPairRDD<K, V>`):
 ```
-GeodeJavaRegionRDD<Int, Emp> rdd1 = javaFunctions(jsc).geodeRegion("emps")
-GeodeJavaRegionRDD<Int, Emp> rdd2 = rdd1.where("value.getAge() < 40");
+GemFireJavaRegionRDD<Int, Emp> rdd1 = javaFunctions(jsc).gemfireRegion("emps")
+GemFireJavaRegionRDD<Int, Emp> rdd2 = rdd1.where("value.getAge() < 40");
 ```
 
 ### RDD Join and Outer Join
@@ -39,10 +39,10 @@ static class MyKeyFunction implements Function<Tuple2<String, Integer>, Integer>
 MyKeyFunction func = new MyKeyFunction();
 
 JavaPairRDD<Tuple2<String, Integer>, Emp> rdd3j =
-  javaFunction(rdd3).joinGeodeRegion("emps", func);
+  javaFunction(rdd3).joinGemfireRegion("emps", func);
 
 JavaPairRDD<Tuple2<String, Integer>, Option<Emp>> rdd3o = 
-  javaFunction(rdd3).outerJoinGeodeRegion("emps", func);
+  javaFunction(rdd3).outerJoinGemfireRegion("emps", func);
 
 ```
 
@@ -57,11 +57,11 @@ data.add(new Tuple2<>("9", "nine"));
 // create JavaPairRDD
 JavaPairRDD<String, String> rdd1 = jsc.parallelizePairs(data);
 // save to Geode
-javaFunctions(rdd1).saveToGeode("str_str_region");
+javaFunctions(rdd1).saveToGemfire("str_str_region");
 ```
 
 In order to save `JavaRDD<Tuple2<K,V>>`, it needs to be converted to 
-`JavaPairRDD<K,V>` via static method `toJavaPairRDD` from `GeodeJavaUtil`:
+`JavaPairRDD<K,V>` via static method `toJavaPairRDD` from `GemFireJavaUtil`:
 ```
 List<Tuple2<String, String>> data2 = new ArrayList<Tuple2<String, String>>();
 data2.add(new Tuple2<>("11", "eleven"));
@@ -71,7 +71,7 @@ data2.add(new Tuple2<>("13", "thirteen"));
 // create JavaRDD<Tuple2<K,V>>
 JavaRDD<Tuple2<String, String>> rdd2 =  jsc.parallelize(data2);
 // save to Geode
-javaFunctions(toJavaPairRDD(rdd2)).saveToGeode("str_str_region");
+javaFunctions(toJavaPairRDD(rdd2)).saveToGemfire("str_str_region");
 ``` 
 
 ### Saving JavaRDD to Geode
@@ -100,7 +100,7 @@ data.add("ab");
 data.add("abc");
 JavaRDD<String> jrdd =  sc.parallelize(data);
     
-javaFunctions(rdd).saveToGeode("str_int_region", pairFunc);
+javaFunctions(rdd).saveToGemfire("str_int_region", pairFunc);
 ```
 
 ### Saving JavaPairDStream and JavaDStream
@@ -108,21 +108,21 @@ Saving JavaPairDStream and JavaDStream is similar to saving JavaPairRDD
 jand JavaRDD:
 ```
 JavaPairDStream<String, String> ds1 = ...
-javaFunctions(ds1).saveToGeode("str_str_region");
+javaFunctions(ds1).saveToGemFire("str_str_region");
 
 JavaDStream<String> ds2 = ...
-javaFunctions(ds2).saveToGeode("str_int_region", pairFunc);
+javaFunctions(ds2).saveToGemFire("str_int_region", pairFunc);
 ```
 
 ### Using Geode OQL
 
-There are two geodeOQL Java APIs, with and without GeodeConnectionConf.
-Here is an example without GeodeConnectionConf, it will use default 
-GeodeConnectionConf internally.
+There are two gemfireOQL Java APIs, with and without GemFireConnectionConf.
+Here is an example without GemFireConnectionConf, it will use default 
+GemFireConnectionConf internally.
 ```
 // assume there's jsc: JavaSparkContext
 SQLContext sqlContext = new org.apache.spark.sql.SQLContext(jsc);
-DataFrame df = javaFunctions(sqlContext).geodeOQL("select * from /str_str_region");
+DataFrame df = javaFunctions(sqlContext).gemfireOQL("select * from /str_str_region");
 df.show();
 ```
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/RegionMetadata.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/RegionMetadata.java b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/RegionMetadata.java
new file mode 100644
index 0000000..fde6204
--- /dev/null
+++ b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/RegionMetadata.java
@@ -0,0 +1,93 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal;
+
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.io.Serializable;
+
+/**
+ * This class contains all info required by GemFire RDD partitioner to create partitions.
+ */
+public class RegionMetadata implements Serializable {
+
+  private String  regionPath;
+  private boolean isPartitioned;
+  private int     totalBuckets;
+  private HashMap<ServerLocation, HashSet<Integer>> serverBucketMap;
+  private String  keyTypeName;
+  private String  valueTypeName;
+
+  /**
+   * Default constructor.
+   * @param regionPath the full path of the given region
+   * @param isPartitioned true for partitioned region, false otherwise
+   * @param totalBuckets number of total buckets for partitioned region, ignored otherwise
+   * @param serverBucketMap gemfire server (host:port pair) to bucket set map
+   * @param keyTypeName region key class name
+   * @param valueTypeName region value class name                    
+   */
+  public RegionMetadata(String regionPath, boolean isPartitioned, int totalBuckets, HashMap<ServerLocation, HashSet<Integer>> serverBucketMap,
+                        String keyTypeName, String valueTypeName) {
+    this.regionPath = regionPath;
+    this.isPartitioned = isPartitioned;
+    this.totalBuckets = totalBuckets;
+    this.serverBucketMap = serverBucketMap;
+    this.keyTypeName = keyTypeName;
+    this.valueTypeName = valueTypeName;
+  }
+
+  public RegionMetadata(String regionPath, boolean isPartitioned, int totalBuckets, HashMap<ServerLocation, HashSet<Integer>> serverBucketMap) {
+    this(regionPath, isPartitioned, totalBuckets, serverBucketMap, null, null);
+  }
+
+  public String getRegionPath() {
+    return regionPath;
+  }
+
+  public boolean isPartitioned() {
+    return isPartitioned;
+  }
+
+  public int getTotalBuckets() {
+    return totalBuckets;
+  }
+  
+  public HashMap<ServerLocation, HashSet<Integer>> getServerBucketMap() {
+    return serverBucketMap;
+  }
+
+  public String getKeyTypeName() {
+    return keyTypeName;
+  }
+
+  public String getValueTypeName() {
+    return valueTypeName;
+  }
+
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("RegionMetadata(region=").append(regionPath)
+       .append("(").append(keyTypeName).append(", ").append(valueTypeName).append(")")
+       .append(", partitioned=").append(isPartitioned).append(", #buckets=").append(totalBuckets)
+       .append(", map=").append(serverBucketMap).append(")");
+    return buf.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/QueryFunction.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/QueryFunction.java b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/QueryFunction.java
new file mode 100644
index 0000000..862bc9f
--- /dev/null
+++ b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/QueryFunction.java
@@ -0,0 +1,99 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.execute.*;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
+import com.gemstone.gemfire.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
+import java.util.Iterator;
+
+public class QueryFunction implements Function {
+
+  private static final long serialVersionUID = 4866641340803692882L;
+
+  public final static String ID = "gemfire-spark-query-function";
+
+  private final static QueryFunction instance = new QueryFunction();
+
+  private static final Logger logger = LogService.getLogger();
+
+  private static final int CHUNK_SIZE = 1024;
+
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+  public static QueryFunction getInstance() {
+    return instance;
+  }
+
+  @Override
+  public boolean optimizeForWrite() {
+    return true;
+  }
+
+  @Override
+  public boolean isHA() {
+    return true;
+  }
+
+  @Override
+  public boolean hasResult() {
+    return true;
+  }
+
+  @Override
+  public void execute(FunctionContext context) {
+    try {
+      String[] args = (String[]) context.getArguments();
+      String queryString = args[0];
+      String bucketSet = args[1];
+      InternalRegionFunctionContext irfc = (InternalRegionFunctionContext) context;
+      LocalRegion localRegion = (LocalRegion) irfc.getDataSet();
+      boolean partitioned = localRegion.getDataPolicy().withPartitioning();
+      Query query = CacheFactory.getAnyInstance().getQueryService().newQuery(queryString);
+      Object result = partitioned ? query.execute((InternalRegionFunctionContext) context) : query.execute();
+      ResultSender<Object> sender = context.getResultSender();
+      HeapDataOutputStream buf = new HeapDataOutputStream(CHUNK_SIZE, null);
+      Iterator<Object> iter = ((SelectResults) result).asList().iterator();
+      while (iter.hasNext()) {
+        Object row = iter.next();
+        DataSerializer.writeObject(row, buf);
+        if (buf.size() > CHUNK_SIZE) {
+          sender.sendResult(buf.toByteArray());
+          logger.debug("OQL query=" + queryString + " bucket set=" + bucketSet + " sendResult(), data size=" + buf.size());
+          buf.reset();
+        }
+      }
+      sender.lastResult(buf.toByteArray());
+      logger.debug("OQL query=" + queryString + " bucket set=" + bucketSet + " lastResult(), data size=" + buf.size());
+      buf.reset();
+    }
+    catch(Exception e) {
+      throw new FunctionException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionFunction.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionFunction.java b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionFunction.java
new file mode 100644
index 0000000..d5a69f3
--- /dev/null
+++ b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionFunction.java
@@ -0,0 +1,208 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions;
+
+import java.util.Iterator;
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.Struct;
+import com.gemstone.gemfire.internal.cache.*;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
+import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
+import com.gemstone.gemfire.internal.cache.execute.InternalResultSender;
+import com.gemstone.gemfire.internal.cache.partitioned.PREntriesIterator;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+/**
+ * GemFire function that is used by `SparkContext.gemfireRegion(regionPath, whereClause)`
+ * to retrieve region data set for the given bucket set as a RDD partition 
+ **/
+public class RetrieveRegionFunction implements Function {
+
+  public final static String ID = "spark-gemfire-retrieve-region";
+  private static final Logger logger = LogService.getLogger();
+  private static final RetrieveRegionFunction instance = new RetrieveRegionFunction();
+
+  public RetrieveRegionFunction() {
+  }
+
+  /** ------------------------------------------ */
+  /**     interface Function implementation      */
+  /** ------------------------------------------ */
+
+  public static RetrieveRegionFunction getInstance() {
+    return instance;
+  }
+
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+  @Override
+  public boolean hasResult() {
+    return true;
+  }
+
+  @Override
+  public boolean optimizeForWrite() {
+    return true;
+  }
+
+  @Override
+  public boolean isHA() {
+    return true;
+  }
+
+  @Override
+  public void execute(FunctionContext context) {
+    String[] args = (String[]) context.getArguments();
+    String where = args[0];
+    String taskDesc = args[1];
+    InternalRegionFunctionContext irfc = (InternalRegionFunctionContext) context;
+    LocalRegion localRegion = (LocalRegion) irfc.getDataSet();
+    boolean partitioned = localRegion.getDataPolicy().withPartitioning();
+    if (where.trim().isEmpty())
+      retrieveFullRegion(irfc, partitioned, taskDesc);
+    else
+      retrieveRegionWithWhereClause(irfc, localRegion, partitioned, where, taskDesc);
+  }
+
+  /** ------------------------------------------ */
+  /**    Retrieve region data with where clause  */
+  /** ------------------------------------------ */
+
+  private void retrieveRegionWithWhereClause(
+    InternalRegionFunctionContext context, LocalRegion localRegion, boolean partitioned, String where, String desc) {
+    String regionPath = localRegion.getFullPath();
+    String qstr = "select key, value from " + regionPath + ".entries where " + where;
+    logger.info(desc + ": " + qstr);
+
+    try {
+      Cache cache = CacheFactory.getAnyInstance();
+      QueryService queryService = cache.getQueryService();
+      Query query = queryService.newQuery(qstr);
+      SelectResults<Struct> results =
+        (SelectResults<Struct>) (partitioned ?  query.execute(context) : query.execute());
+
+      Iterator<Object[]> entries = getStructIteratorWrapper(results.asList().iterator());
+      InternalResultSender irs = (InternalResultSender) context.getResultSender();
+      StructStreamingResultSender sender = new StructStreamingResultSender(irs, null, entries, desc);
+      sender.send();
+    } catch (Exception e) {
+      throw new FunctionException(e);
+    }
+  }
+
+  private Iterator<Object[]> getStructIteratorWrapper(Iterator<Struct> entries) {
+    return new WrapperIterator<Struct, Iterator<Struct>>(entries) {
+      @Override public Object[] next() {
+        return  delegate.next().getFieldValues();
+      }
+    };
+  }
+
+  /** ------------------------------------------ */
+  /**         Retrieve full region data          */
+  /** ------------------------------------------ */
+
+  private void retrieveFullRegion(InternalRegionFunctionContext context, boolean partitioned, String desc) {
+    Iterator<Object[]> entries;
+    if (partitioned) {
+      PREntriesIterator<Region.Entry> iter = (PREntriesIterator<Region.Entry>)
+              ((LocalDataSet) PartitionRegionHelper.getLocalDataForContext(context)).entrySet().iterator();
+      // entries = getPREntryIterator(iter);
+      entries = getSimpleEntryIterator(iter);
+    } else {
+      LocalRegion owner = (LocalRegion) context.getDataSet();
+      Iterator<Region.Entry> iter = (Iterator<Region.Entry>) owner.entrySet().iterator();
+      // entries = getRREntryIterator(iter, owner);
+      entries = getSimpleEntryIterator(iter);
+    }
+    InternalResultSender irs = (InternalResultSender) context.getResultSender();
+    StructStreamingResultSender sender = new StructStreamingResultSender(irs, null, entries, desc);
+    sender.send();
+  }
+
+//  /** An iterator for partitioned region that uses internal API to get serialized value */
+//  private Iterator<Object[]> getPREntryIterator(PREntriesIterator<Region.Entry> iterator) {
+//    return new WrapperIterator<Region.Entry, PREntriesIterator<Region.Entry>>(iterator) {
+//      @Override public Object[] next() {
+//        Region.Entry entry = delegate.next();
+//        int bucketId = delegate.getBucketId();
+//        KeyInfo keyInfo = new KeyInfo(entry.getKey(), null, bucketId);
+//        // owner needs to be the bucket region not the enclosing partition region
+//        LocalRegion owner = ((PartitionedRegion) entry.getRegion()).getDataStore().getLocalBucketById(bucketId);
+//        Object value = owner.getDeserializedValue(keyInfo, false, true, true, null, false);
+//        return new Object[] {keyInfo.getKey(), value};
+//      }
+//    };
+//  }
+//
+//  /** An iterator for replicated region that uses internal API to get serialized value */
+//  private Iterator<Object[]> getRREntryIterator(Iterator<Region.Entry> iterator, LocalRegion region) {
+//    final LocalRegion owner = region;
+//    return new WrapperIterator<Region.Entry, Iterator<Region.Entry>>(iterator) {
+//      @Override public Object[] next() {
+//        Region.Entry entry =  delegate.next();
+//        KeyInfo keyInfo = new KeyInfo(entry.getKey(), null, null);
+//        Object value = owner.getDeserializedValue(keyInfo, false, true, true, null, false);
+//        return new Object[] {keyInfo.getKey(), value};
+//      }
+//    };
+//  }
+
+  // todo. compare performance of regular and simple iterator
+  /** An general iterator for both partitioned and replicated region that returns un-serialized value */
+  private Iterator<Object[]> getSimpleEntryIterator(Iterator<Region.Entry> iterator) {
+    return new WrapperIterator<Region.Entry, Iterator<Region.Entry>>(iterator) {
+      @Override public Object[] next() {
+        Region.Entry entry = delegate.next();
+        return new Object[] {entry.getKey(), entry.getValue()};
+      }
+    };
+  }
+
+  /** ------------------------------------------ */
+  /**        abstract wrapper iterator           */
+  /** ------------------------------------------ */
+
+  /** An abstract wrapper iterator to reduce duplicated code of anonymous iterators */
+  abstract class WrapperIterator<T, S extends Iterator<T>> implements Iterator<Object[]> {
+
+    final S delegate;
+
+    protected WrapperIterator(S delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override public boolean hasNext() {
+      return delegate.hasNext();
+    }
+
+    @Override public void remove() { }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionMetadataFunction.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionMetadataFunction.java b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionMetadataFunction.java
new file mode 100644
index 0000000..de62cc1
--- /dev/null
+++ b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/RetrieveRegionMetadataFunction.java
@@ -0,0 +1,118 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions;
+
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.cache.BucketServerLocation66;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This GemFire function retrieve region metadata
+ */
+public class RetrieveRegionMetadataFunction implements Function {
+
+  public final static String ID = "gemfire-spark-retrieve-region-metadata";
+  
+  private static final RetrieveRegionMetadataFunction instance = new RetrieveRegionMetadataFunction();
+
+  public RetrieveRegionMetadataFunction() {
+  }
+
+  public static RetrieveRegionMetadataFunction getInstance() {
+    return instance;    
+  }
+  
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+  @Override
+  public boolean optimizeForWrite() {
+    return false;
+  }
+
+  @Override
+  public boolean isHA() {
+    return true;
+  }
+
+  @Override
+  public boolean hasResult() {
+    return true;
+  }
+
+  @Override
+  public void execute(FunctionContext context) {
+    LocalRegion region = (LocalRegion) ((InternalRegionFunctionContext) context).getDataSet();
+    String regionPath = region.getFullPath();
+    boolean isPartitioned = region.getDataPolicy().withPartitioning();
+    String kTypeName = getTypeClassName(region.getAttributes().getKeyConstraint());
+    String vTypeName = getTypeClassName(region.getAttributes().getValueConstraint());
+
+    RegionMetadata metadata;
+    if (! isPartitioned) {
+      metadata = new RegionMetadata(regionPath, false, 0, null, kTypeName, vTypeName);
+    } else {
+      PartitionedRegion pregion = (PartitionedRegion) region;
+      int totalBuckets = pregion.getAttributes().getPartitionAttributes().getTotalNumBuckets();
+      Map<Integer, List<BucketServerLocation66>> bucketMap = pregion.getRegionAdvisor().getAllClientBucketProfiles();
+      HashMap<ServerLocation, HashSet<Integer>> serverMap = bucketServerMap2ServerBucketSetMap(bucketMap);
+      metadata = new RegionMetadata(regionPath, true, totalBuckets, serverMap, kTypeName, vTypeName);
+    }
+    
+    ResultSender<RegionMetadata> sender = context.getResultSender();
+    sender.lastResult(metadata);
+  }
+  
+  private String getTypeClassName(Class clazz) {
+    return clazz == null ? null : clazz.getCanonicalName();
+  }
+  
+  /** convert bucket to server map to server to bucket set map */
+  private HashMap<ServerLocation, HashSet<Integer>>
+    bucketServerMap2ServerBucketSetMap(Map<Integer, List<BucketServerLocation66>> map) {
+    HashMap<ServerLocation, HashSet<Integer>> serverBucketMap = new HashMap<>();
+    for (Integer id : map.keySet()) {
+      List<BucketServerLocation66> locations = map.get(id);
+      for (BucketServerLocation66 location : locations) {
+        ServerLocation server = new ServerLocation(location.getHostName(), location.getPort());
+        if (location.isPrimary()) {
+          HashSet<Integer> set = serverBucketMap.get(server);
+          if (set == null) {
+            set = new HashSet<>();
+            serverBucketMap.put(server, set);
+          }
+          set.add(id);
+          break;
+        }
+      }
+    }
+    return serverBucketMap;    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSender.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSender.java b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSender.java
new file mode 100644
index 0000000..793de6a
--- /dev/null
+++ b/geode-spark-connector/gemfire-functions/src/main/java/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSender.java
@@ -0,0 +1,219 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.query.internal.types.ObjectTypeImpl;
+import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
+import com.gemstone.gemfire.cache.query.types.ObjectType;
+import com.gemstone.gemfire.cache.query.types.StructType;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.CachedDeserializable;
+import com.gemstone.gemfire.internal.logging.LogService;
+import org.apache.logging.log4j.Logger;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+/**
+ * StructStreamingResultSender and StructStreamingResultCollector  are paired
+ * to transfer result of list of `com.gemstone.gemfire.cache.query.Struct`
+ * from GemFire server to Spark Connector (the client of GemFire server)
+ * in streaming, i.e., while sender sending the result, the collector can
+ * start processing the arrived result without waiting for full result to
+ * become available.
+ */
+public class StructStreamingResultSender {
+
+  public static final byte TYPE_CHUNK   = 0x30;
+  public static final byte DATA_CHUNK   = 0x31;
+  public static final byte ERROR_CHUNK  = 0x32;
+  public static final byte SER_DATA     = 0x41;
+  public static final byte UNSER_DATA   = 0x42;
+  public static final byte BYTEARR_DATA = 0x43;
+
+  private static ObjectTypeImpl ObjField = new ObjectTypeImpl(java.lang.Object.class);
+  public static StructTypeImpl KeyValueType = new StructTypeImpl(new String[]{"key", "value"}, new ObjectType[]{ObjField, ObjField});
+
+  private static final Logger logger = LogService.getLogger();
+  private static final int CHUNK_SIZE = 4096;
+  
+  // Note: The type of ResultSender returned from GemFire FunctionContext is
+  //  always ResultSender<Object>, so can't use ResultSender<byte[]> here
+  private final ResultSender<Object> sender;
+  private final StructType structType;
+  private final Iterator<Object[]> rows;
+  private String desc;
+  private boolean closed = false;
+
+  /**
+   * the Constructor 
+   * @param sender the base ResultSender that send data in byte array
+   * @param type the StructType of result record
+   * @param rows the iterator of the collection of results
+   * @param desc description of this result (used for logging)           
+   */
+  public StructStreamingResultSender(
+    ResultSender<Object> sender, StructType type, Iterator<Object[]> rows, String desc) {
+    if (sender == null || rows == null)
+      throw new NullPointerException("sender=" + sender + ", rows=" + rows);
+    this.sender = sender;
+    this.structType = type;
+    this.rows = rows;
+    this.desc = desc;
+  }
+
+  /** the Constructor with default `desc` */
+  public StructStreamingResultSender(
+          ResultSender<Object> sender, StructType type, Iterator<Object[]> rows) {
+    this(sender, type, rows, "StructStreamingResultSender");
+  }
+  
+  /**
+   * Send the result in chunks. There are 3 types of chunk: TYPE, DATA, and ERROR.
+   * TYPE chunk for sending struct type info, DATA chunk for sending data, and
+   * ERROR chunk for sending exception. There are at most 1 TYPE chunk (omitted
+   * for `KeyValueType`) and 1 ERROR chunk (if there's error), but usually
+   * there are multiple DATA chunks. Each DATA chunk contains multiple rows
+   * of data. The chunk size is determined by the const `CHUNK_SIZE`. If an
+   * exception is thrown, it is serialized and sent as the last chunk  of the 
+   * result (in the form of ERROR chunk).
+   */
+  public void send() {
+    if (closed) throw new RuntimeException("sender is closed.");
+
+    HeapDataOutputStream buf = new HeapDataOutputStream(CHUNK_SIZE + 2048, null);
+    String dataType = null;
+    int typeSize = 0;
+    int rowCount = 0;
+    int dataSize = 0;            
+    try {
+      if (rows.hasNext()) {
+        // Note: only send type info if there's data with it
+        typeSize = sendType(buf);
+        buf.writeByte(DATA_CHUNK);
+        int rowSize = structType == null ? 2 : structType.getFieldNames().length;
+        while (rows.hasNext()) {
+          rowCount ++;
+          Object[] row = rows.next();          
+          if (rowCount < 2) dataType = entryDataType(row);
+          if (rowSize != row.length) 
+            throw new IOException(rowToString("Expect "  + rowSize + " columns, but got ", row));
+          serializeRowToBuffer(row, buf);
+          if (buf.size() > CHUNK_SIZE) {
+            dataSize += sendBufferredData(buf, false);
+            buf.writeByte(DATA_CHUNK);
+          }
+        }
+      }
+      // send last piece of data or empty byte array
+      dataSize += sendBufferredData(buf, true);
+      logger.info(desc + ": " + rowCount + " rows, type=" + dataType + ", type.size=" +
+                  typeSize + ", data.size=" + dataSize + ", row.avg.size=" +
+                  (rowCount == 0 ? "NaN" : String.format("%.1f", ((float) dataSize)/rowCount)));
+    } catch (IOException | RuntimeException e) {
+      sendException(buf, e);
+    } finally {
+      closed = true;
+    }
+  }
+
+  private String rowToString(String rowDesc, Object[] row) {
+    StringBuilder buf = new StringBuilder();
+    buf.append(rowDesc).append("(");
+    for (int i = 0; i < row.length; i++) buf.append(i ==0 ? "" : " ,").append(row[i]);
+    return buf.append(")") .toString();    
+  }
+
+  private String entryDataType(Object[] row) {
+    StringBuilder buf = new StringBuilder();
+    buf.append("(");
+    for (int i = 0; i < row.length; i++) {
+      if (i != 0) buf.append(", ");
+      buf.append(row[i].getClass().getCanonicalName());
+    }
+    return buf.append(")").toString();
+  }
+  
+  private void serializeRowToBuffer(Object[] row, HeapDataOutputStream buf) throws IOException {
+    for (Object data : row) {
+      if (data instanceof CachedDeserializable) {
+        buf.writeByte(SER_DATA);
+        DataSerializer.writeByteArray(((CachedDeserializable) data).getSerializedValue(), buf);
+      } else if (data instanceof byte[]) {
+        buf.writeByte(BYTEARR_DATA);
+        DataSerializer.writeByteArray((byte[]) data, buf);
+      } else {
+        buf.writeByte(UNSER_DATA);
+        DataSerializer.writeObject(data, buf);
+      }
+    }
+  }
+  
+  /** return the size of type data */
+  private int sendType(HeapDataOutputStream buf) throws IOException {
+    // logger.info(desc + " struct type: " + structType);
+    if (structType != null) {
+      buf.writeByte(TYPE_CHUNK);
+      DataSerializer.writeObject(structType, buf);
+      return sendBufferredData(buf, false);      
+    } else {
+      return 0;  // default KeyValue type, no type info send
+    }
+  }
+  
+  private int sendBufferredData(HeapDataOutputStream buf, boolean isLast) throws IOException {
+    if (isLast) sender.lastResult(buf.toByteArray());
+    else sender.sendResult(buf.toByteArray());
+    // logData(buf.toByteArray(), desc);
+    int s = buf.size();
+    buf.reset();
+    return s;    
+  }
+
+  /** Send the exception as the last chunk of the result. */
+  private void sendException(HeapDataOutputStream buf, Exception e) {
+    // Note: if exception happens during the serialization, the `buf` may contain
+    // partial serialized data, which may cause de-serialization hang or error.
+    // Therefore, always empty the buffer before sending the exception
+    if (buf.size() > 0) buf.reset();
+    
+    try {
+      buf.writeByte(ERROR_CHUNK);
+      DataSerializer.writeObject(e, buf);
+    } catch (IOException ioe) {
+      logger.error("StructStreamingResultSender failed to send the result:", e);
+      logger.error("StructStreamingResultSender failed to serialize the exception:", ioe);
+      buf.reset();
+    }
+    // Note: send empty chunk as the last result if serialization of exception 
+    // failed, and the error is logged on the GemFire server side.
+    sender.lastResult(buf.toByteArray());
+    // logData(buf.toByteArray(), desc);
+  }
+
+//  private void logData(byte[] data, String desc) {
+//    StringBuilder buf = new StringBuilder();
+//    buf.append(desc);
+//    for (byte b : data) {
+//      buf.append(" ").append(b);
+//    }
+//    logger.info(buf.toString());
+//  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Employee.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Employee.java b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Employee.java
new file mode 100644
index 0000000..6cb571f
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/java/ittest/io/pivotal/gemfire/spark/connector/Employee.java
@@ -0,0 +1,54 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector;
+
+import java.io.Serializable;
+
+public class Employee implements Serializable {
+
+  private String name;
+
+  private int age;
+
+  public Employee(String n, int a) {
+    name = n;
+    age = a;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public int getAge() {
+    return age;
+  }
+
+  public String toString() {
+    return new StringBuilder().append("Employee[name=").append(name).
+            append(", age=").append(age).
+            append("]").toString();
+  }
+  
+  public boolean equals(Object o) {
+    if (o instanceof Employee) {
+      return ((Employee) o).name.equals(name) && ((Employee) o).age == age;
+    }
+    return false;
+  }
+
+}
+


[02/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartition.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartition.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartition.scala
deleted file mode 100644
index 24fe72e..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartition.scala
+++ /dev/null
@@ -1,36 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import org.apache.spark.Partition
-
-/**
- * This serializable class represents a GeodeRDD partition. Each partition is mapped
- * to one or more buckets of region. The GeodeRDD can materialize the data of the 
- * partition based on all information contained here.
- * @param partitionId partition id, a 0 based number.
- * @param bucketSet region bucket id set for this partition. Set.empty means whole
- *                  region (used for replicated region)
- * @param locations preferred location for this partition                  
- */
-case class GeodeRDDPartition (
-  partitionId: Int, bucketSet: Set[Int], locations: Seq[String] = Nil)
-  extends Partition  {
-  
-  override def index: Int = partitionId
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitioner.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitioner.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitioner.scala
deleted file mode 100644
index d960cab..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitioner.scala
+++ /dev/null
@@ -1,59 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import io.pivotal.geode.spark.connector.GeodeConnection
-import io.pivotal.geode.spark.connector.internal.RegionMetadata
-import org.apache.spark.{Logging, Partition}
-
-import scala.reflect.ClassTag
-
-/**
- * A GeodeRDD partitioner is used to partition the region into multiple RDD partitions.
- */
-trait GeodeRDDPartitioner extends Serializable {
-
-  def name: String
-  
-  /** the function that generates partitions */
-  def partitions[K: ClassTag, V: ClassTag]
-    (conn: GeodeConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition]
-}
-
-object GeodeRDDPartitioner extends Logging {
-
-  /** To add new partitioner, just add it to the following list */
-  final val partitioners: Map[String, GeodeRDDPartitioner] =
-    List(OnePartitionPartitioner, ServerSplitsPartitioner).map(e => (e.name, e)).toMap
-
-  /**
-   * Get a partitioner based on given name, a default partitioner will be returned if there's
-   * no partitioner for the given name. 
-   */
-  def apply(name: String = defaultPartitionedRegionPartitioner.name): GeodeRDDPartitioner = {
-    val p = partitioners.get(name)
-    if (p.isDefined) p.get else {
-      logWarning(s"Invalid preferred partitioner name $name.")
-      defaultPartitionedRegionPartitioner
-    }
-  }
-
-  val defaultReplicatedRegionPartitioner = OnePartitionPartitioner
-
-  val defaultPartitionedRegionPartitioner = ServerSplitsPartitioner
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitionerImpl.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitionerImpl.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitionerImpl.scala
deleted file mode 100644
index 4606114..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDPartitionerImpl.scala
+++ /dev/null
@@ -1,89 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import io.pivotal.geode.spark.connector.GeodeConnection
-import io.pivotal.geode.spark.connector.internal.RegionMetadata
-import io.pivotal.geode.spark.connector.NumberPartitionsPerServerPropKey
-import org.apache.spark.Partition
-import scala.collection.JavaConversions._
-import scala.collection.immutable.SortedSet
-import scala.collection.mutable
-import scala.reflect.ClassTag
-
-/** This partitioner maps whole region to one GeodeRDDPartition */
-object OnePartitionPartitioner extends GeodeRDDPartitioner {
-
-  override val name = "OnePartition"
-
-  override def partitions[K: ClassTag, V: ClassTag]
-    (conn: GeodeConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition] =
-    Array[Partition](new GeodeRDDPartition(0, Set.empty))
-}
-
-/**
-  * This partitioner maps whole region to N * M Geode RDD partitions, where M is the number of 
-  * Geode servers that contain the data for the given region. Th default value of N is 1.
-  */
-object ServerSplitsPartitioner extends GeodeRDDPartitioner {
-
-  override val name = "ServerSplits"
-
-  override def partitions[K: ClassTag, V: ClassTag]
-  (conn: GeodeConnection, md: RegionMetadata, env: Map[String, String]): Array[Partition] = {
-    if (md == null) throw new RuntimeException("RegionMetadata is null")
-    val n = try { env.getOrElse(NumberPartitionsPerServerPropKey, "2").toInt } catch { case e: NumberFormatException => 2 }
-    if (!md.isPartitioned || md.getServerBucketMap == null || md.getServerBucketMap.isEmpty)
-      Array[Partition](new GeodeRDDPartition(0, Set.empty))
-    else {
-      val map = mapAsScalaMap(md.getServerBucketMap)
-        .map { case (srv, set) => (srv, asScalaSet(set).map(_.toInt)) }.toList
-        .map { case (srv, set) => (srv.getHostName, set) }
-       doPartitions(map, md.getTotalBuckets, n)
-    }
-  }
-
-  /** Converts server to bucket ID set list to array of RDD partitions */
-  def doPartitions(serverBucketMap: List[(String, mutable.Set[Int])], totalBuckets: Int, n: Int)
-    : Array[Partition] = {
-
-    // method that calculates the group size for splitting "k" items into "g" groups
-    def groupSize(k: Int, g: Int): Int = scala.math.ceil(k / g.toDouble).toInt
-
-    // 1. convert list of server and bucket set pairs to a list of server and sorted bucket set pairs
-    val srvToSortedBucketSet = serverBucketMap.map { case (srv, set) => (srv, SortedSet[Int]() ++ set) }
-
-    // 2. split bucket set of each server into n splits if possible, and server to Seq(server)
-    val srvToSplitedBuckeSet = srvToSortedBucketSet.flatMap { case (host, set) =>
-      if (set.isEmpty) Nil else set.grouped(groupSize(set.size, n)).toList.map(s => (Seq(host), s)) }
-
-    // 3. calculate empty bucket IDs by removing all bucket sets of all servers from the full bucket sets
-    val emptyIDs = SortedSet[Int]() ++ ((0 until totalBuckets).toSet /: srvToSortedBucketSet) {case (s1, (k, s2)) => s1 &~ s2}
-
-    // 4. distribute empty bucket IDs to all partitions evenly.
-    //    The empty buckets do not contain data when partitions are created, but they may contain data
-    //    when RDD is materialized, so need to include those bucket IDs in the partitions.
-    val srvToFinalBucketSet = if (emptyIDs.isEmpty) srvToSplitedBuckeSet
-      else srvToSplitedBuckeSet.zipAll(
-        emptyIDs.grouped(groupSize(emptyIDs.size, srvToSplitedBuckeSet.size)).toList, (Nil, Set.empty), Set.empty).map
-          { case ((server, set1), set2) => (server, SortedSet[Int]() ++ set1 ++ set2) }
-
-    // 5. create array of partitions w/ 0-based index
-    (0 until srvToFinalBucketSet.size).toList.zip(srvToFinalBucketSet).map
-      { case (i, (srv, set)) => new GeodeRDDPartition(i, set, srv) }.toArray
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDWriter.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDWriter.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDWriter.scala
deleted file mode 100644
index dba15f3..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRDDWriter.scala
+++ /dev/null
@@ -1,82 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector._
-import org.apache.spark.{Logging, TaskContext}
-
-import scala.collection.Iterator
-import java.util.{HashMap => JMap}
-
-/** This trait provide some common code for pair and non-pair RDD writer */
-private[rdd] abstract class GeodeRDDWriterBase (opConf: Map[String, String]) extends Serializable {
-
-  val batchSize = try { opConf.getOrElse(RDDSaveBatchSizePropKey, RDDSaveBatchSizeDefault.toString).toInt}
-                  catch { case e: NumberFormatException => RDDSaveBatchSizeDefault }
-
-  def mapDump(map: Map[_, _], num: Int): String = {
-    val firstNum = map.take(num + 1)
-    if (firstNum.size > num) s"$firstNum ..." else s"$firstNum"    
-  }  
-}
-
-/**
- * Writer object that provides write function that saves non-pair RDD partitions to Geode.
- * Those functions will be executed on Spark executors.
- * @param regionPath the full path of the region where the data is written to
- */
-class GeodeRDDWriter[T, K, V] 
-  (regionPath: String, connConf: GeodeConnectionConf, opConf: Map[String, String] = Map.empty)
-  extends GeodeRDDWriterBase(opConf) with Serializable with Logging {
-
-  def write(func: T => (K, V))(taskContext: TaskContext, data: Iterator[T]): Unit = {
-    val region: Region[K, V] = connConf.getConnection.getRegionProxy[K, V](regionPath)
-    var count = 0
-    val chunks = data.grouped(batchSize)
-    chunks.foreach { chunk =>
-      val map = chunk.foldLeft(new JMap[K, V]()){case (m, t) => val (k, v) = func(t); m.put(k, v); m}
-      region.putAll(map)
-      count += chunk.length
-    }
-    logDebug(s"$count entries (batch.size = $batchSize) are saved to region $regionPath")
-  }
-}
-
-
-/**
- * Writer object that provides write function that saves pair RDD partitions to Geode.
- * Those functions will be executed on Spark executors.
- * @param regionPath the full path of the region where the data is written to
- */
-class GeodePairRDDWriter[K, V]
-  (regionPath: String, connConf: GeodeConnectionConf, opConf: Map[String, String] = Map.empty)
-  extends GeodeRDDWriterBase(opConf) with Serializable with Logging {
-
-  def write(taskContext: TaskContext, data: Iterator[(K, V)]): Unit = {
-    val region: Region[K, V] = connConf.getConnection.getRegionProxy[K, V](regionPath)
-    var count = 0
-    val chunks = data.grouped(batchSize)
-    chunks.foreach { chunk =>
-      val map = chunk.foldLeft(new JMap[K, V]()){case (m, (k,v)) => m.put(k,v); m}
-      region.putAll(map)
-      count += chunk.length
-    }
-    logDebug(s"$count entries (batch.batch = $batchSize) are saved to region $regionPath")
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRegionRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRegionRDD.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRegionRDD.scala
deleted file mode 100644
index 6980c0f..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/internal/rdd/GeodeRegionRDD.scala
+++ /dev/null
@@ -1,138 +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 io.pivotal.geode.spark.connector.internal.rdd
-
-import scala.collection.Seq
-import scala.reflect.ClassTag
-import org.apache.spark.rdd.RDD
-import org.apache.spark.{TaskContext, Partition, SparkContext}
-import io.pivotal.geode.spark.connector.{GeodeConnectionConf, PreferredPartitionerPropKey}
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRDDPartitioner._
-
-/**
- * This class exposes Geode region as a RDD.
- * @param sc the Spark Context
- * @param regionPath the full path of the region
- * @param connConf the GeodeConnectionConf to access the region
- * @param opConf the parameters for this operation, such as preferred partitioner.
- */
-class GeodeRegionRDD[K, V] private[connector]
-  (@transient sc: SparkContext,
-   val regionPath: String,
-   val connConf: GeodeConnectionConf,
-   val opConf: Map[String, String] = Map.empty,
-   val whereClause: Option[String] = None 
-  ) (implicit ctk: ClassTag[K], ctv: ClassTag[V])
-  extends RDD[(K, V)](sc, Seq.empty) {
-
-  /** validate region existence when GeodeRDD object is created */
-  validate()
-
-  /** Validate region, and make sure it exists. */
-  private def validate(): Unit = connConf.getConnection.validateRegion[K, V](regionPath)
-
-  def kClassTag = ctk
-  
-  def vClassTag = ctv
-
-  /**
-   * method `copy` is used by method `where` that creates new immutable
-   * GeodeRDD instance based this instance.
-   */
-  private def copy(
-    regionPath: String = regionPath,
-    connConf: GeodeConnectionConf = connConf,
-    opConf: Map[String, String] = opConf,
-    whereClause: Option[String] = None
-  ): GeodeRegionRDD[K, V] = {
-
-    require(sc != null,
-    """RDD transformation requires a non-null SparkContext. Unfortunately
-      |SparkContext in this GeodeRDD is null. This can happen after 
-      |GeodeRDD has been deserialized. SparkContext is not Serializable,
-      |therefore it deserializes to null. RDD transformations are not allowed
-      |inside lambdas used in other RDD transformations.""".stripMargin )
-
-    new GeodeRegionRDD[K, V](sc, regionPath, connConf, opConf, whereClause)
-  }
-
-  /** When where clause is specified, OQL query
-    * `select key, value from /<region-path>.entries where <where clause> `
-    * is used to filter the dataset.
-    */
-  def where(whereClause: Option[String]): GeodeRegionRDD[K, V] = {
-    if (whereClause.isDefined) copy(whereClause = whereClause)
-    else this
-  }
-
-  /** this version is for Java API that doesn't use scala.Option */
-  def where(whereClause: String): GeodeRegionRDD[K, V] = {
-    if (whereClause == null || whereClause.trim.isEmpty) this
-    else copy(whereClause = Option(whereClause.trim))
-  }
-
-  /**
-   * Use preferred partitioner generate partitions. `defaultReplicatedRegionPartitioner`
-   * will be used if it's a replicated region. 
-   */
-  override def getPartitions: Array[Partition] = {
-    val conn = connConf.getConnection
-    val md = conn.getRegionMetadata[K, V](regionPath)
-    md match {
-      case None => throw new RuntimeException(s"region $regionPath was not found.")
-      case Some(data) =>
-        logInfo(s"""RDD id=${this.id} region=$regionPath conn=${connConf.locators.mkString(",")}, env=$opConf""")
-        val p = if (data.isPartitioned) preferredPartitioner else defaultReplicatedRegionPartitioner
-        val splits = p.partitions[K, V](conn, data, opConf)
-        logDebug(s"""RDD id=${this.id} region=$regionPath partitions=\n  ${splits.mkString("\n  ")}""")
-        splits
-    }
-  }
-
-  /**
-   * provide preferred location(s) (host name(s)) of the given partition. 
-   * Only some partitioner implementation(s) provides this info, which is
-   * useful when Spark cluster and Geode cluster share some hosts.
-   */
-  override def getPreferredLocations(split: Partition) =
-    split.asInstanceOf[GeodeRDDPartition].locations
-
-  /**
-   * Get preferred partitioner. return `defaultPartitionedRegionPartitioner` if none
-   * preference is specified. 
-   */
-  private def preferredPartitioner = 
-    GeodeRDDPartitioner(opConf.getOrElse(
-      PreferredPartitionerPropKey, GeodeRDDPartitioner.defaultPartitionedRegionPartitioner.name))
-
-  /** materialize a RDD partition */
-  override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val partition = split.asInstanceOf[GeodeRDDPartition]
-    logDebug(s"compute RDD id=${this.id} partition $partition")
-    connConf.getConnection.getRegionData[K,V](regionPath, whereClause, partition)
-    // new InterruptibleIterator(context, split.asInstanceOf[GeodeRDDPartition[K, V]].iterator)
-  }
-}
-
-object GeodeRegionRDD {
-
-  def apply[K: ClassTag, V: ClassTag](sc: SparkContext, regionPath: String,
-    connConf: GeodeConnectionConf, opConf: Map[String, String] = Map.empty)
-    : GeodeRegionRDD[K, V] =
-    new GeodeRegionRDD[K, V](sc, regionPath, connConf, opConf)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRegionRDD.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRegionRDD.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRegionRDD.scala
deleted file mode 100644
index f859173..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRegionRDD.scala
+++ /dev/null
@@ -1,26 +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 io.pivotal.geode.spark.connector.javaapi
-
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRegionRDD
-import org.apache.spark.api.java.JavaPairRDD
-
-class GeodeJavaRegionRDD[K, V](rdd: GeodeRegionRDD[K, V]) extends JavaPairRDD[K, V](rdd)(rdd.kClassTag, rdd.vClassTag) {
-  
-  def where(whereClause: String): GeodeJavaRegionRDD[K, V] = new GeodeJavaRegionRDD(rdd.where(whereClause))
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/JavaAPIHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/JavaAPIHelper.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/JavaAPIHelper.scala
deleted file mode 100644
index ffa6195..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/javaapi/JavaAPIHelper.scala
+++ /dev/null
@@ -1,53 +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 io.pivotal.geode.spark.connector.javaapi
-
-import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
-import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream}
-
-import scala.reflect.ClassTag
-import scala.collection.JavaConversions._
-
-/**
- *  A helper class to make it possible to access components written in Scala from Java code.
- */
-private[connector] object JavaAPIHelper {
-
-  /** Returns a `ClassTag` of a given runtime class. */
-  def getClassTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
-
-  /**
-   * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef].
-   * see JavaSparkContext.fakeClassTag in Spark for more info.
-   */
-  def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
-
-  /** Converts a Java `Properties` to a Scala immutable `Map[String, String]`. */
-  def propertiesToScalaMap[K, V](props: java.util.Properties): Map[String, String] =
-    Map(props.toSeq: _*)
-
-  /** convert a JavaRDD[(K,V)] to JavaPairRDD[K,V] */
-  def toJavaPairRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] =
-    JavaPairRDD.fromJavaRDD(rdd)
-
-  /** convert a JavaDStream[(K,V)] to JavaPairDStream[K,V] */
-  def toJavaPairDStream[K, V](ds: JavaDStream[(K, V)]): JavaPairDStream[K, V] =
-    JavaPairDStream.fromJavaDStream(ds)
-
-  /** an empty Map[String, String] for default opConf **/
-  val emptyStrStrMap: Map[String, String] = Map.empty
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/package.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/package.scala
deleted file mode 100644
index 6f9a780..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/package.scala
+++ /dev/null
@@ -1,69 +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 io.pivotal.geode.spark
-
-import io.pivotal.geode.spark.connector.internal.rdd.{ServerSplitsPartitioner, OnePartitionPartitioner}
-import org.apache.spark.SparkContext
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SQLContext
-
-import scala.reflect.ClassTag
-
-/**
- * The root package of Geode connector for Apache Spark.
- * Provides handy implicit conversions that add geode-specific
- * methods to `SparkContext` and `RDD`.
- */
-package object connector {
-
-  /** constants */
-  final val GeodeLocatorPropKey = "spark.geode.locators"
-  // partitioner related keys and values
-  final val PreferredPartitionerPropKey = "preferred.partitioner"
-  final val NumberPartitionsPerServerPropKey = "number.partitions.per.server"
-  final val OnePartitionPartitionerName = OnePartitionPartitioner.name
-  final val ServerSplitsPartitionerName = ServerSplitsPartitioner.name
-
-  final val RDDSaveBatchSizePropKey = "rdd.save.batch.size"
-  final val RDDSaveBatchSizeDefault = 10000
-  
-  /** implicits */
-  
-  implicit def toSparkContextFunctions(sc: SparkContext): GeodeSparkContextFunctions =
-    new GeodeSparkContextFunctions(sc)
-
-  implicit def toSQLContextFunctions(sqlContext: SQLContext): GeodeSQLContextFunctions =
-    new GeodeSQLContextFunctions(sqlContext)
-
-  implicit def toGeodePairRDDFunctions[K: ClassTag, V: ClassTag]
-    (self: RDD[(K, V)]): GeodePairRDDFunctions[K, V] = new GeodePairRDDFunctions(self)
-
-  implicit def toGeodeRDDFunctions[T: ClassTag]
-    (self: RDD[T]): GeodeRDDFunctions[T] = new GeodeRDDFunctions(self)
-
-  /** utility implicits */
-  
-  /** convert Map[String, String] to java.util.Properties */
-  implicit def map2Properties(map: Map[String,String]): java.util.Properties =
-    (new java.util.Properties /: map) {case (props, (k,v)) => props.put(k,v); props}
-
-  /** internal util methods */
-  
-  private[connector] def getRddPartitionsInfo(rdd: RDD[_], sep: String = "\n  "): String =
-    rdd.partitions.zipWithIndex.map{case (p,i) => s"$i: $p loc=${rdd.preferredLocations(p)}"}.mkString(sep)
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/GeodeDStreamFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/GeodeDStreamFunctions.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/GeodeDStreamFunctions.scala
deleted file mode 100644
index 4d46429..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/GeodeDStreamFunctions.scala
+++ /dev/null
@@ -1,89 +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 io.pivotal.geode.spark.connector.streaming
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodePairRDDWriter, GeodeRDDWriter}
-import org.apache.spark.Logging
-import org.apache.spark.api.java.function.PairFunction
-import org.apache.spark.streaming.dstream.DStream
-
-/**
- * Extra geode functions on DStream of non-pair elements through an implicit conversion.
- * Import `io.pivotal.geode.spark.connector.streaming._` at the top of your program to
- * use these functions.
- */
-class GeodeDStreamFunctions[T](val dstream: DStream[T]) extends Serializable with Logging {
-
-  /**
-   * Save the DStream of non-pair elements to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored
-   * @param func the function that converts elements of the DStream to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */
-  def saveToGeode[K, V](
-      regionPath: String, 
-      func: T => (K, V), 
-      connConf: GeodeConnectionConf = defaultConnectionConf, 
-      opConf: Map[String, String] = Map.empty): Unit = {
-    connConf.getConnection.validateRegion[K, V](regionPath)
-    val writer = new GeodeRDDWriter[T, K, V](regionPath, connConf, opConf)
-    logInfo(s"""Save DStream region=$regionPath conn=${connConf.locators.mkString(",")}""")
-    dstream.foreachRDD(rdd => rdd.sparkContext.runJob(rdd, writer.write(func) _))
-  }
-
-  /** this version of saveToGeode is just for Java API */
-  def saveToGeode[K, V](
-      regionPath: String,
-      func: PairFunction[T, K, V],
-      connConf: GeodeConnectionConf,
-      opConf: Map[String, String] ): Unit = {
-    saveToGeode[K, V](regionPath, func.call _, connConf, opConf)
-  }
-
-  private[connector] def defaultConnectionConf: GeodeConnectionConf =
-    GeodeConnectionConf(dstream.context.sparkContext.getConf)
-}
-
-
-/**
- * Extra geode functions on DStream of (key, value) pairs through an implicit conversion.
- * Import `io.pivotal.geode.spark.connector.streaming._` at the top of your program to
- * use these functions.
- */
-class GeodePairDStreamFunctions[K, V](val dstream: DStream[(K,V)]) extends Serializable with Logging {
-
-  /**
-   * Save the DStream of pairs to Geode key-value store without any conversion
-   * @param regionPath the full path of region that the DStream is stored
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */
-  def saveToGeode(
-      regionPath: String, 
-      connConf: GeodeConnectionConf = defaultConnectionConf, 
-      opConf: Map[String, String] = Map.empty): Unit = {
-    connConf.getConnection.validateRegion[K, V](regionPath)
-    val writer = new GeodePairRDDWriter[K, V](regionPath, connConf, opConf)
-    logInfo(s"""Save DStream region=$regionPath conn=${connConf.locators.mkString(",")}""")
-    dstream.foreachRDD(rdd => rdd.sparkContext.runJob(rdd, writer.write _))
-  }
-
-  private[connector] def defaultConnectionConf: GeodeConnectionConf =
-    GeodeConnectionConf(dstream.context.sparkContext.getConf)
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/package.scala b/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/package.scala
deleted file mode 100644
index 0d1f1eb..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/scala/io/pivotal/geode/spark/connector/streaming/package.scala
+++ /dev/null
@@ -1,32 +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 io.pivotal.geode.spark.connector
-
-import org.apache.spark.streaming.dstream.DStream
-
-/**
- * Provides handy implicit conversions that add gemfire-specific methods to `DStream`.
- */
-package object streaming {
-
-  implicit def toGeodeDStreamFunctions[T](ds: DStream[T]): GeodeDStreamFunctions[T] =
-    new GeodeDStreamFunctions[T](ds)
-
-  implicit def toGeodePairDStreamFunctions[K, V](ds: DStream[(K, V)]): GeodePairDStreamFunctions[K, V] =
-    new GeodePairDStreamFunctions[K, V](ds)
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/java/io/pivotal/geode/spark/connector/JavaAPITest.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/java/io/pivotal/geode/spark/connector/JavaAPITest.java b/geode-spark-connector/geode-spark-connector/src/test/java/io/pivotal/geode/spark/connector/JavaAPITest.java
deleted file mode 100644
index 142907e..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/java/io/pivotal/geode/spark/connector/JavaAPITest.java
+++ /dev/null
@@ -1,163 +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 io.pivotal.geode.spark.connector;
-
-import io.pivotal.geode.spark.connector.javaapi.*;
-import org.apache.spark.SparkContext;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.rdd.RDD;
-import org.apache.spark.sql.SQLContext;
-//import org.apache.spark.sql.api.java.JavaSQLContext;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaPairDStream;
-import org.apache.spark.streaming.dstream.DStream;
-import org.junit.Test;
-import org.scalatest.junit.JUnitSuite;
-import scala.Function1;
-import scala.Function2;
-import scala.Tuple2;
-import scala.Tuple3;
-import scala.collection.mutable.LinkedList;
-import scala.reflect.ClassTag;
-
-import static org.junit.Assert.*;
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.*;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
-
-public class JavaAPITest extends JUnitSuite {
-
-  @SuppressWarnings( "unchecked" )
-  public Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> createCommonMocks() {
-    SparkContext mockSparkContext = mock(SparkContext.class);
-    GeodeConnectionConf mockConnConf = mock(GeodeConnectionConf.class);
-    GeodeConnection mockConnection = mock(GeodeConnection.class);
-    when(mockConnConf.getConnection()).thenReturn(mockConnection);
-    when(mockConnConf.locators()).thenReturn(new LinkedList());
-    return new Tuple3<>(mockSparkContext, mockConnConf, mockConnection);
-  }
-  
-  @Test
-  public void testSparkContextFunction() throws Exception {
-    Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> tuple3 = createCommonMocks();
-    GeodeJavaSparkContextFunctions wrapper = javaFunctions(tuple3._1());
-    assertTrue(tuple3._1() == wrapper.sc);
-    String regionPath = "testregion";
-    JavaPairRDD<String, String> rdd = wrapper.geodeRegion(regionPath, tuple3._2());
-    verify(tuple3._3()).validateRegion(regionPath);
-  }
-
-  @Test
-  public void testJavaSparkContextFunctions() throws Exception {
-    SparkContext mockSparkContext = mock(SparkContext.class);
-    JavaSparkContext mockJavaSparkContext = mock(JavaSparkContext.class);
-    when(mockJavaSparkContext.sc()).thenReturn(mockSparkContext);
-    GeodeJavaSparkContextFunctions wrapper = javaFunctions(mockJavaSparkContext);
-    assertTrue(mockSparkContext == wrapper.sc);
-  }
-  
-  @Test
-  @SuppressWarnings( "unchecked" )
-  public void testJavaPairRDDFunctions() throws Exception {
-    JavaPairRDD<String, Integer> mockPairRDD = mock(JavaPairRDD.class);
-    RDD<Tuple2<String, Integer>> mockTuple2RDD = mock(RDD.class);
-    when(mockPairRDD.rdd()).thenReturn(mockTuple2RDD);
-    GeodeJavaPairRDDFunctions wrapper = javaFunctions(mockPairRDD);
-    assertTrue(mockTuple2RDD == wrapper.rddf.rdd());
-
-    Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> tuple3 = createCommonMocks();
-    when(mockTuple2RDD.sparkContext()).thenReturn(tuple3._1());
-    String regionPath = "testregion";
-    wrapper.saveToGeode(regionPath, tuple3._2());
-    verify(mockTuple2RDD, times(1)).sparkContext();
-    verify(tuple3._1(), times(1)).runJob(eq(mockTuple2RDD), any(Function2.class), any(ClassTag.class));
-  }
-
-  @Test
-  @SuppressWarnings( "unchecked" )
-  public void testJavaRDDFunctions() throws Exception {
-    JavaRDD<String> mockJavaRDD = mock(JavaRDD.class);
-    RDD<String> mockRDD = mock(RDD.class);
-    when(mockJavaRDD.rdd()).thenReturn(mockRDD);
-    GeodeJavaRDDFunctions wrapper = javaFunctions(mockJavaRDD);
-    assertTrue(mockRDD == wrapper.rddf.rdd());
-
-    Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> tuple3 = createCommonMocks();
-    when(mockRDD.sparkContext()).thenReturn(tuple3._1());
-    PairFunction<String, String, Integer> mockPairFunc = mock(PairFunction.class);
-    String regionPath = "testregion";
-    wrapper.saveToGeode(regionPath, mockPairFunc, tuple3._2());
-    verify(mockRDD, times(1)).sparkContext();
-    verify(tuple3._1(), times(1)).runJob(eq(mockRDD), any(Function2.class), any(ClassTag.class));
-  }
-
-  @Test
-  @SuppressWarnings( "unchecked" )
-  public void testJavaPairDStreamFunctions() throws Exception {
-    JavaPairDStream<String, String> mockJavaDStream = mock(JavaPairDStream.class);
-    DStream<Tuple2<String, String>> mockDStream = mock(DStream.class);
-    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
-    GeodeJavaPairDStreamFunctions wrapper = javaFunctions(mockJavaDStream);
-    assertTrue(mockDStream == wrapper.dsf.dstream());
-
-    Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> tuple3 = createCommonMocks();
-    String regionPath = "testregion";
-    wrapper.saveToGeode(regionPath, tuple3._2());
-    verify(tuple3._2()).getConnection();
-    verify(tuple3._3()).validateRegion(regionPath);
-    verify(mockDStream).foreachRDD(any(Function1.class));
-  }
-
-  @Test
-  @SuppressWarnings( "unchecked" )
-  public void testJavaPairDStreamFunctionsWithTuple2DStream() throws Exception {
-    JavaDStream<Tuple2<String, String>> mockJavaDStream = mock(JavaDStream.class);
-    DStream<Tuple2<String, String>> mockDStream = mock(DStream.class);
-    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
-    GeodeJavaPairDStreamFunctions wrapper = javaFunctions(toJavaPairDStream(mockJavaDStream));
-    assertTrue(mockDStream == wrapper.dsf.dstream());
-  }
-
-  @Test
-  @SuppressWarnings( "unchecked" )
-  public void testJavaDStreamFunctions() throws Exception {
-    JavaDStream<String> mockJavaDStream = mock(JavaDStream.class);
-    DStream<String> mockDStream = mock(DStream.class);
-    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
-    GeodeJavaDStreamFunctions wrapper = javaFunctions(mockJavaDStream);
-    assertTrue(mockDStream == wrapper.dsf.dstream());
-
-    Tuple3<SparkContext, GeodeConnectionConf, GeodeConnection> tuple3 = createCommonMocks();
-    PairFunction<String, String, Integer> mockPairFunc = mock(PairFunction.class);
-    String regionPath = "testregion";
-    wrapper.saveToGeode(regionPath, mockPairFunc, tuple3._2());
-    verify(tuple3._2()).getConnection();
-    verify(tuple3._3()).validateRegion(regionPath);
-    verify(mockDStream).foreachRDD(any(Function1.class));
-  }
-
-  @Test
-  public void testSQLContextFunction() throws Exception {
-    SQLContext mockSQLContext = mock(SQLContext.class);
-    GeodeJavaSQLContextFunctions wrapper = javaFunctions(mockSQLContext);
-    assertTrue(wrapper.scf.getClass() == GeodeSQLContextFunctions.class);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployerTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployerTest.scala
deleted file mode 100644
index 4e45dc2..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/GeodeFunctionDeployerTest.scala
+++ /dev/null
@@ -1,58 +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 io.pivotal.geode.spark.connector
-
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{FunSuite, Matchers}
-import org.apache.commons.httpclient.HttpClient
-import java.io.File
-
-
-class GeodeFunctionDeployerTest extends FunSuite with Matchers with MockitoSugar {
-  val mockHttpClient: HttpClient = mock[HttpClient]
-    
-  test("jmx url creation") {
-    val jmxHostAndPort = "localhost:7070"
-    val expectedUrlString = "http://" + jmxHostAndPort + "/gemfire/v1/deployed"
-    val gfd = new GeodeFunctionDeployer(mockHttpClient);
-    val urlString = gfd.constructURLString(jmxHostAndPort)
-    assert(urlString === expectedUrlString)
-  }
-    
-  test("missing jar file") {
-    val missingJarFileLocation = "file:///somemissingjarfilethatdoesnot.exist"
-    val gfd = new GeodeFunctionDeployer(mockHttpClient);
-    intercept[RuntimeException] { gfd.jarFileHandle(missingJarFileLocation)}
-  }
-  
-  test("deploy with missing jar") {
-    val missingJarFileLocation = "file:///somemissingjarfilethatdoesnot.exist"
-    val gfd = new GeodeFunctionDeployer(mockHttpClient);
-    intercept[RuntimeException] {(gfd.deploy("localhost:7070", missingJarFileLocation).contains("Deployed"))}
-    intercept[RuntimeException] {(gfd.deploy("localhost", 7070, missingJarFileLocation).contains("Deployed"))}
-  }
-  
-  test("successful mocked deploy") {
-    val gfd = new GeodeFunctionDeployer(mockHttpClient);
-    val jar = new File("README.md");
-    assert(gfd.deploy("localhost:7070", jar).contains("Deployed"))
-  }
-  
-
-    
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
deleted file mode 100644
index 798912c..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
+++ /dev/null
@@ -1,82 +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 io.pivotal.geode.spark.connector.internal
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{FunSuite, Matchers}
-
-class DefaultGeodeConnectionManagerTest extends FunSuite  with Matchers with MockitoSugar {
-
-  test("DefaultGeodeConnectionFactory get/closeConnection") {
-    // note: connConf 1-4 share the same set of locators
-    val connConf1 = new GeodeConnectionConf(Seq(("host1", 1234)))
-    val connConf2 = new GeodeConnectionConf(Seq(("host2", 5678)))
-    val connConf3 = new GeodeConnectionConf(Seq(("host1", 1234), ("host2", 5678)))
-    val connConf4 = new GeodeConnectionConf(Seq(("host2", 5678), ("host1", 1234)))
-    val connConf5 = new GeodeConnectionConf(Seq(("host5", 3333)))
-
-    val props: Map[String, String] = Map.empty
-    val mockConnFactory: DefaultGeodeConnectionFactory = mock[DefaultGeodeConnectionFactory]
-    val mockConn1 = mock[DefaultGeodeConnection]
-    val mockConn2 = mock[DefaultGeodeConnection]
-    when(mockConnFactory.newConnection(connConf3.locators, props)).thenReturn(mockConn1)
-    when(mockConnFactory.newConnection(connConf5.locators, props)).thenReturn(mockConn2)
-
-    assert(DefaultGeodeConnectionManager.getConnection(connConf3)(mockConnFactory) == mockConn1)
-    // note: following 3 lines do not trigger connFactory.newConnection(...)
-    assert(DefaultGeodeConnectionManager.getConnection(connConf1)(mockConnFactory) == mockConn1)
-    assert(DefaultGeodeConnectionManager.getConnection(connConf2)(mockConnFactory) == mockConn1)
-    assert(DefaultGeodeConnectionManager.getConnection(connConf4)(mockConnFactory) == mockConn1)
-    assert(DefaultGeodeConnectionManager.getConnection(connConf5)(mockConnFactory) == mockConn2)
-
-    // connFactory.newConnection(...) were invoked only twice
-    verify(mockConnFactory, times(1)).newConnection(connConf3.locators, props)
-    verify(mockConnFactory, times(1)).newConnection(connConf5.locators, props)
-    assert(DefaultGeodeConnectionManager.connections.size == 3)
-
-    DefaultGeodeConnectionManager.closeConnection(connConf1)
-    assert(DefaultGeodeConnectionManager.connections.size == 1)
-    DefaultGeodeConnectionManager.closeConnection(connConf5)
-    assert(DefaultGeodeConnectionManager.connections.isEmpty)
-  }
-  
-  test("DefaultGeodeConnectionFactory newConnection(...) throws RuntimeException") {
-    val connConf1 = new GeodeConnectionConf(Seq(("host1", 1234)))
-    val props: Map[String, String] = Map.empty
-    val mockConnFactory: DefaultGeodeConnectionFactory = mock[DefaultGeodeConnectionFactory]
-    when(mockConnFactory.newConnection(connConf1.locators, props)).thenThrow(new RuntimeException())
-    intercept[RuntimeException] { DefaultGeodeConnectionManager.getConnection(connConf1)(mockConnFactory) }
-    verify(mockConnFactory, times(1)).newConnection(connConf1.locators, props)
-  }
-
-  test("DefaultGeodeConnectionFactory close() w/ non-exist connection") {
-    val props: Map[String, String] = Map.empty
-    val mockConnFactory: DefaultGeodeConnectionFactory = mock[DefaultGeodeConnectionFactory]
-    val connConf1 = new GeodeConnectionConf(Seq(("host1", 1234)))
-    val connConf2 = new GeodeConnectionConf(Seq(("host2", 5678)))
-    val mockConn1 = mock[DefaultGeodeConnection]
-    when(mockConnFactory.newConnection(connConf1.locators, props)).thenReturn(mockConn1)
-    assert(DefaultGeodeConnectionManager.getConnection(connConf1)(mockConnFactory) == mockConn1)
-    assert(DefaultGeodeConnectionManager.connections.size == 1)
-    // connection does not exists in the connection manager
-    DefaultGeodeConnectionManager.closeConnection(connConf2)
-    assert(DefaultGeodeConnectionManager.connections.size == 1)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
deleted file mode 100644
index f2303e7..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
+++ /dev/null
@@ -1,254 +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 io.pivotal.geode.spark.connector.internal.geodefunctions
-
-import com.gemstone.gemfire.DataSerializer
-import com.gemstone.gemfire.cache.execute.{ResultCollector, ResultSender}
-import com.gemstone.gemfire.cache.query.internal.types.{ObjectTypeImpl, StructTypeImpl}
-import com.gemstone.gemfire.cache.query.types.ObjectType
-import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput, HeapDataOutputStream}
-import com.gemstone.gemfire.internal.cache.{CachedDeserializable, CachedDeserializableFactory}
-import org.scalatest.{BeforeAndAfter, FunSuite}
-import scala.collection.JavaConversions._
-import scala.concurrent.{Await, ExecutionContext, Future}
-import ExecutionContext.Implicits.global
-import scala.concurrent.duration._
-
-class StructStreamingResultSenderAndCollectorTest extends FunSuite with BeforeAndAfter  {
-
-  /** 
-    * A test ResultSender that connects struct ResultSender and ResultCollector 
-    * Note: this test ResultSender has to copy the data (byte array) since the
-    *       StructStreamingResultSender will reuse the byte array.
-    */
-  class LocalResultSender(collector: ResultCollector[Array[Byte], _], num: Int = 1) extends ResultSender[Object] {
-
-    var finishedNum = 0
-    
-    override def sendResult(result: Object): Unit =
-      collector.addResult(null, result.asInstanceOf[Array[Byte]].clone())
-
-    /** exception should be sent via lastResult() */
-    override def sendException(throwable: Throwable): Unit = 
-      throw new UnsupportedOperationException("sendException is not supported.")
-
-    override def lastResult(result: Object): Unit = {
-      collector.addResult(null, result.asInstanceOf[Array[Byte]].clone())
-      this.synchronized {
-        finishedNum += 1
-        if (finishedNum == num)
-          collector.endResults()
-      }
-    }
-  }
-
-  /** common variables */
-  var collector: StructStreamingResultCollector = _
-  var baseSender: LocalResultSender = _
-  /** common types */
-  val objType = new ObjectTypeImpl("java.lang.Object").asInstanceOf[ObjectType]
-  val TwoColType = new StructTypeImpl(Array("key", "value"), Array(objType, objType))
-  val OneColType = new StructTypeImpl(Array("value"), Array(objType))
-
-  before {
-    collector = new StructStreamingResultCollector
-    baseSender = new LocalResultSender(collector, 1)
-  }
-  
-  test("transfer simple data") {
-    verifySimpleTransfer(sendDataType = true)
-  }
-
-  test("transfer simple data with no type info") {
-    verifySimpleTransfer(sendDataType = false)
-  }
-
-  def verifySimpleTransfer(sendDataType: Boolean): Unit = {
-    val iter = (0 to 9).map(i => Array(i.asInstanceOf[Object], (i.toString * 5).asInstanceOf[Object])).toIterator
-    val dataType = if (sendDataType) TwoColType else null
-    new StructStreamingResultSender(baseSender, dataType , iter).send()
-    // println("type: " + collector.getResultType.toString)
-    assert(TwoColType.equals(collector.getResultType))
-    val iter2 = collector.getResult
-    (0 to 9).foreach { i =>
-      assert(iter2.hasNext)
-      val o = iter2.next()
-      assert(o.size == 2)
-      assert(o(0).asInstanceOf[Int] == i)
-      assert(o(1).asInstanceOf[String] == i.toString * 5)
-    }
-    assert(! iter2.hasNext)
-  }
-
-  
-  /**
-   * A test iterator that generate integer data
-   * @param start the 1st value
-   * @param n number of integers generated
-   * @param genExcp generate Exception if true. This is used to test exception handling.
-   */
-  def intIterator(start: Int, n: Int, genExcp: Boolean): Iterator[Array[Object]] = {
-    new Iterator[Array[Object]] {
-      val max = if (genExcp) start + n else start + n - 1
-      var index: Int = start - 1
-
-      override def hasNext: Boolean = if (index < max) true else false
-
-      override def next(): Array[Object] =
-        if (index < (start + n - 1)) {
-          index += 1
-          Array(index.asInstanceOf[Object])
-        } else throw new RuntimeException("simulated error")
-    }
-  }
-
-  test("transfer data with 0 row") {
-    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 0, genExcp = false)).send()
-    // println("type: " + collector.getResultType.toString)
-    assert(collector.getResultType == null)
-    val iter = collector.getResult
-    assert(! iter.hasNext)
-  }
-
-  test("transfer data with 10K rows") {
-    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 10000, genExcp = false)).send()
-    // println("type: " + collector.getResultType.toString)
-    assert(OneColType.equals(collector.getResultType))
-    val iter = collector.getResult
-    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
-    (1 to 10000).foreach { i =>
-      assert(iter.hasNext)
-      val o = iter.next()
-      assert(o.size == 1)
-      assert(o(0).asInstanceOf[Int] == i)
-    }
-    assert(! iter.hasNext)
-  }
-
-  test("transfer data with 10K rows with 2 sender") {
-    baseSender = new LocalResultSender(collector, 2)
-    val total = 300
-    val sender1 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(1, total/2, genExcp = false), "sender1").send()}
-    val sender2 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(total/2+1, total/2, genExcp = false), "sender2").send()}
-    Await.result(sender1, 1.seconds)
-    Await.result(sender2, 1.seconds)
-
-    // println("type: " + collector.getResultType.toString)
-    assert(OneColType.equals(collector.getResultType))
-    val iter = collector.getResult
-    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
-    val set = scala.collection.mutable.Set[Int]()
-    (1 to total).foreach { i =>
-      assert(iter.hasNext)
-      val o = iter.next()
-      assert(o.size == 1)
-      assert(! set.contains(o(0).asInstanceOf[Int]))
-      set.add(o(0).asInstanceOf[Int])
-    }
-    assert(! iter.hasNext)
-  }
-
-  test("transfer data with 10K rows with 2 sender with error") {
-    baseSender = new LocalResultSender(collector, 2)
-    val total = 1000
-    val sender1 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(1, total/2, genExcp = false), "sender1").send()}
-    val sender2 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(total/2+1, total/2, genExcp = true), "sender2").send()}
-    Await.result(sender1, 1 seconds)
-    Await.result(sender2, 1 seconds)
-
-    // println("type: " + collector.getResultType.toString)
-    assert(OneColType.equals(collector.getResultType))
-    val iter = collector.getResult
-    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
-    val set = scala.collection.mutable.Set[Int]()
-    intercept[RuntimeException] {
-      (1 to total).foreach { i =>
-        assert(iter.hasNext)
-        val o = iter.next()
-        assert(o.size == 1)
-        assert(! set.contains(o(0).asInstanceOf[Int]))
-        set.add(o(0).asInstanceOf[Int])
-      }
-    }
-    // println(s"rows received: ${set.size}")
-  }
-
-  test("transfer data with Exception") {
-    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 200, genExcp = true)).send()
-    // println("type: " + collector.getResultType.toString)
-    val iter = collector.getResult
-    intercept[RuntimeException] ( iter.foreach(_.mkString(",")) )
-  }
-
-  def stringPairIterator(n: Int, genExcp: Boolean): Iterator[Array[Object]] =
-    intIterator(1, n, genExcp).map(x => Array(s"key-${x(0)}", s"value-${x(0)}"))
-
-  test("transfer string pair data with 200 rows") {
-    new StructStreamingResultSender(baseSender, TwoColType, stringPairIterator(1000, genExcp = false)).send()
-    // println("type: " + collector.getResultType.toString)
-    assert(TwoColType.equals(collector.getResultType))
-    val iter = collector.getResult
-    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
-    (1 to 1000).foreach { i =>
-      assert(iter.hasNext)
-      val o = iter.next()
-      assert(o.size == 2)
-      assert(o(0) == s"key-$i")
-      assert(o(1) == s"value-$i")
-    }
-    assert(! iter.hasNext)
-  }
-
-  /**
-   * Usage notes: There are 3 kinds of data to transfer:
-   * (1) object, (2) byte array of serialized object, and (3) byte array
-   * this test shows how to handle all of them.
-   */
-  test("DataSerializer usage") {
-    val outBuf = new HeapDataOutputStream(1024, null)
-    val inBuf = new ByteArrayDataInput()
-
-    // 1. a regular object
-    val hello = "Hello World!" * 30
-    // serialize the data
-    DataSerializer.writeObject(hello, outBuf)
-    val bytesHello = outBuf.toByteArray.clone()
-    // de-serialize the data
-    inBuf.initialize(bytesHello, Version.CURRENT)
-    val hello2 = DataSerializer.readObject(inBuf).asInstanceOf[Object]
-    assert(hello == hello2)
-    
-    // 2. byte array of serialized object
-    // serialize: byte array from `CachedDeserializable`
-    val cd: CachedDeserializable = CachedDeserializableFactory.create(bytesHello)
-    outBuf.reset()
-    DataSerializer.writeByteArray(cd.getSerializedValue, outBuf)
-    // de-serialize the data in 2 steps
-    inBuf.initialize(outBuf.toByteArray.clone(), Version.CURRENT)
-    val bytesHello2: Array[Byte] = DataSerializer.readByteArray(inBuf)
-    inBuf.initialize(bytesHello2, Version.CURRENT)
-    val hello3 = DataSerializer.readObject(inBuf).asInstanceOf[Object]
-    assert(hello == hello3)
-
-    // 3. byte array
-    outBuf.reset()
-    DataSerializer.writeByteArray(bytesHello, outBuf)
-    inBuf.initialize(outBuf.toByteArray.clone(), Version.CURRENT)
-    val bytesHello3: Array[Byte] = DataSerializer.readByteArray(inBuf)
-    assert(bytesHello sameElements bytesHello3)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParserTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParserTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParserTest.scala
deleted file mode 100644
index 54394e8..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/io/pivotal/geode/spark/connector/internal/oql/QueryParserTest.scala
+++ /dev/null
@@ -1,83 +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 io.pivotal.geode.spark.connector.internal.oql
-
-import org.scalatest.FunSuite
-
-class QueryParserTest extends FunSuite {
-
-  test("select * from /r1") {
-    val r = QueryParser.parseOQL("select * from /r1").get
-    assert(r == "List(/r1)")
-  }
-
-  test("select c2 from /r1") {
-    val r = QueryParser.parseOQL("select c2 from /r1").get
-    assert(r == "List(/r1)")
-  }
-
-  test("select key, value from /r1.entries") {
-    val r = QueryParser.parseOQL("select key, value from /r1.entries").get
-    assert(r == "List(/r1.entries)")
-  }
-
-  test("select c1, c2 from /r1 where col1 > 100 and col2 <= 120 or c3 = 2") {
-    val r = QueryParser.parseOQL("select c1, c2 from /r1 where col1 > 100 and col2 <= 120 or c3 = 2").get
-    assert(r == "List(/r1)")
-  }
-
-  test("select * from /r1/r2 where c1 >= 200") {
-    val r = QueryParser.parseOQL("select * from /r1/r2 where c1 >= 200").get
-    assert(r == "List(/r1/r2)")
-  }
-
-  test("import io.pivotal select c1, c2, c3 from /r1/r2, /r3/r4 where c1 <= 15 and c2 = 100") {
-    val r = QueryParser.parseOQL("import io.pivotal select c1, c2, c3 from /r1/r2, /r3/r4 where c1 <= 15 and c2 = 100").get
-    assert(r == "List(/r1/r2, /r3/r4)")
-  }
-
-  test("SELECT distinct f1, f2 FROM /r1/r2 WHere f = 100") {
-    val r = QueryParser.parseOQL("SELECT distinct f1, f2 FROM /r1/r2 WHere f = 100").get
-    assert(r == "List(/r1/r2)")
-  }
-
-  test("IMPORT io.pivotal.geode IMPORT com.mypackage SELECT key,value FROM /root/sub.entries WHERE status = 'active' ORDER BY id desc") {
-    val r = QueryParser.parseOQL("IMPORT io.pivotal.geode IMPORT com.mypackage SELECT key,value FROM /root/sub.entries WHERE status = 'active' ORDER BY id desc").get
-    assert(r == "List(/root/sub.entries)")
-  }
-
-  test("select distinct p.ID, p.status from /region p where p.ID > 5 order by p.status") {
-    val r = QueryParser.parseOQL("select distinct p.ID, p.status from /region p where p.ID > 5 order by p.status").get
-    assert(r == "List(/region)")
-  }
-
-  test("SELECT DISTINCT * FROM /QueryRegion1 r1,  /QueryRegion2 r2 WHERE r1.ID = r2.ID") {
-    val r = QueryParser.parseOQL("SELECT DISTINCT * FROM /QueryRegion1 r1,  /QueryRegion2 r2 WHERE r1.ID = r2.ID").get
-    assert(r == "List(/QueryRegion1, /QueryRegion2)")
-  }
-
-  test("SELECT id, \"type\", positions, status FROM /obj_obj_region WHERE status = 'active'") {
-    val r = QueryParser.parseOQL("SELECT id, \"type\", positions, status FROM /obj_obj_region WHERE status = 'active'").get
-    println("r.type=" + r.getClass.getName + " r=" + r)
-    assert(r == "List(/obj_obj_region)")
-  }
-
-  test("SELECT r.id, r.\"type\", r.positions, r.status FROM /obj_obj_region r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'") {
-    val r = QueryParser.parseOQL("SELECT r.id, r.\"type\", r.positions, r.status FROM /obj_obj_region r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'").get
-    assert(r == "List(/obj_obj_region, r.positions.values)")
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/ConnectorImplicitsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/ConnectorImplicitsTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/ConnectorImplicitsTest.scala
deleted file mode 100644
index b0464cc..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/ConnectorImplicitsTest.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 unittest.io.pivotal.geode.spark.connector
-
-import io.pivotal.geode.spark.connector._
-import org.apache.spark.SparkContext
-import org.apache.spark.sql.SQLContext
-import org.scalatest.FunSuite
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.Matchers
-
-class ConnectorImplicitsTest extends FunSuite with Matchers with MockitoSugar {
-
-  test("implicit map2Properties") {
-    verifyProperties(Map.empty)
-    verifyProperties(Map("One" -> "1", "Two" -> "2", "Three" ->"3"))
-  }
-  
-  def verifyProperties(map: Map[String, String]): Unit = {
-    val props: java.util.Properties = map
-    assert(props.size() == map.size)
-    map.foreach(p => assert(props.getProperty(p._1) == p._2))    
-  }
-
-  test("Test Implicit SparkContext Conversion") {
-    val mockSparkContext = mock[SparkContext]
-    val gfscf: GeodeSparkContextFunctions = mockSparkContext
-    assert(gfscf.isInstanceOf[GeodeSparkContextFunctions])
-  }
-
-  test("Test Implicit SQLContext Conversion") {
-    val mockSQLContext = mock[SQLContext]
-    val gfscf: GeodeSQLContextFunctions = mockSQLContext
-    assert(gfscf.isInstanceOf[GeodeSQLContextFunctions])
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeConnectionConfTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeConnectionConfTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeConnectionConfTest.scala
deleted file mode 100644
index a3076f4..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeConnectionConfTest.scala
+++ /dev/null
@@ -1,100 +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 unittest.io.pivotal.geode.spark.connector
-
-import org.apache.spark.SparkConf
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{Matchers, FunSuite}
-import io.pivotal.geode.spark.connector._
-
-class GeodeConnectionConfTest extends FunSuite with Matchers with MockitoSugar {
-
-  test("apply(SparkConf) w/ GeodeLocator property and empty geodeProps") {
-    val (host1, port1) = ("host1", 1234)
-    val (host2, port2) = ("host2", 5678)
-    val conf = new SparkConf().set(GeodeLocatorPropKey, s"$host1[$port1],$host2[$port2]")
-    val connConf = GeodeConnectionConf(conf)
-    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
-    assert(connConf.geodeProps.isEmpty)
-  }
-  
-  test("apply(SparkConf) w/ GeodeLocator property and geode properties") {
-    val (host1, port1) = ("host1", 1234)
-    val (host2, port2) = ("host2", 5678)
-    val (propK1, propV1) = ("ack-severe-alert-threshold", "1")
-    val (propK2, propV2) = ("ack-wait-threshold", "10")
-    val conf = new SparkConf().set(GeodeLocatorPropKey, s"$host1[$port1],$host2[$port2]")
-                              .set(s"spark.geode.$propK1", propV1).set(s"spark.geode.$propK2", propV2)
-    val connConf = GeodeConnectionConf(conf)
-    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
-    assert(connConf.geodeProps == Map(propK1 -> propV1, propK2 -> propV2))
-  }
-
-  test("apply(SparkConf) w/o GeodeLocator property") {
-    intercept[RuntimeException] { GeodeConnectionConf(new SparkConf()) }
-  }
-
-  test("apply(SparkConf) w/ invalid GeodeLocator property") {
-    val conf = new SparkConf().set(GeodeLocatorPropKey, "local^host:1234")
-    intercept[Exception] { GeodeConnectionConf(conf) }
-  }
-
-  test("apply(locatorStr, geodeProps) w/ valid locatorStr and non geodeProps") {
-    val (host1, port1) = ("host1", 1234)
-    val connConf = GeodeConnectionConf(s"$host1:$port1")
-    assert(connConf.locators == Seq((host1, port1)))
-    assert(connConf.geodeProps.isEmpty)
-  }
-
-  test("apply(locatorStr, geodeProps) w/ valid locatorStr and non-empty geodeProps") {
-    val (host1, port1) = ("host1", 1234)
-    val (host2, port2) = ("host2", 5678)
-    val (propK1, propV1) = ("ack-severe-alert-threshold", "1")
-    val (propK2, propV2) = ("ack-wait-threshold", "10")
-    val props = Map(propK1 -> propV1, propK2 -> propV2)
-    val connConf = GeodeConnectionConf(s"$host1:$port1,$host2:$port2", props)
-    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
-    assert(connConf.geodeProps == props)
-  }
-
-  test("apply(locatorStr, geodeProps) w/ invalid locatorStr") {
-    intercept[Exception] { GeodeConnectionConf("local~host:4321") }
-  }
-
-  test("constructor w/ empty (host,port) pairs") {
-    intercept[IllegalArgumentException] { new GeodeConnectionConf(Seq.empty) }
-  }
-
-  test("getConnection() normal") {
-    implicit val mockFactory = mock[GeodeConnectionManager]
-    val mockConnection = mock[GeodeConnection]
-    when(mockFactory.getConnection(org.mockito.Matchers.any[GeodeConnectionConf])).thenReturn(mockConnection)
-    val connConf = GeodeConnectionConf("localhost:1234")
-    assert(connConf.getConnection == mockConnection)
-    verify(mockFactory).getConnection(connConf)
-  }
-
-  test("getConnection() failure") {
-    implicit val mockFactory = mock[GeodeConnectionManager]
-    when(mockFactory.getConnection(org.mockito.Matchers.any[GeodeConnectionConf])).thenThrow(new RuntimeException)
-    val connConf = GeodeConnectionConf("localhost:1234")
-    intercept[RuntimeException] { connConf.getConnection }
-    verify(mockFactory).getConnection(connConf)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeDStreamFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeDStreamFunctionsTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeDStreamFunctionsTest.scala
deleted file mode 100644
index bcba7e1..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeDStreamFunctionsTest.scala
+++ /dev/null
@@ -1,79 +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 unittest.io.pivotal.geode.spark.connector
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.{GeodeConnection, GeodeConnectionConf}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.dstream.DStream
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{Matchers, FunSuite}
-import org.mockito.Matchers.{eq => mockEq, any => mockAny}
-
-import scala.reflect.ClassTag
-
-class GeodeDStreamFunctionsTest extends FunSuite with Matchers with MockitoSugar {
-
-  test("test GeodePairDStreamFunctions Implicit") {
-    import io.pivotal.geode.spark.connector.streaming._
-    val mockDStream = mock[DStream[(Int, String)]]
-    // the implicit make the following line valid
-    val pairDStream: GeodePairDStreamFunctions[Int, String] = mockDStream
-    pairDStream shouldBe a[GeodePairDStreamFunctions[_, _]]
-  }
-
-  test("test GeodeDStreamFunctions Implicit") {
-    import io.pivotal.geode.spark.connector.streaming._
-    val mockDStream = mock[DStream[String]]
-    // the implicit make the following line valid
-    val dstream: GeodeDStreamFunctions[String] = mockDStream
-    dstream shouldBe a[GeodeDStreamFunctions[_]]
-  }
-
-  def createMocks[K, V](regionPath: String)
-    (implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]])
-    : (String, GeodeConnectionConf, GeodeConnection, Region[K, V]) = {
-    val mockConnection = mock[GeodeConnection]
-    val mockConnConf = mock[GeodeConnectionConf]
-    val mockRegion = mock[Region[K, V]]
-    when(mockConnConf.getConnection).thenReturn(mockConnection)
-    when(mockConnConf.locators).thenReturn(Seq.empty)
-    (regionPath, mockConnConf, mockConnection, mockRegion)
-  }
-
-  test("test GeodePairDStreamFunctions.saveToGeode()") {
-    import io.pivotal.geode.spark.connector.streaming._
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
-    val mockDStream = mock[DStream[(String, String)]]
-    mockDStream.saveToGeode(regionPath, mockConnConf)
-    verify(mockConnConf).getConnection
-    verify(mockConnection).validateRegion[String, String](regionPath)
-    verify(mockDStream).foreachRDD(mockAny[(RDD[(String, String)]) => Unit])
-  }
-
-  test("test GeodeDStreamFunctions.saveToGeode()") {
-    import io.pivotal.geode.spark.connector.streaming._
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, Int]("test")
-    val mockDStream = mock[DStream[String]]
-    mockDStream.saveToGeode[String, Int](regionPath,  (s: String) => (s, s.length), mockConnConf)
-    verify(mockConnConf).getConnection
-    verify(mockConnection).validateRegion[String, String](regionPath)
-    verify(mockDStream).foreachRDD(mockAny[(RDD[String]) => Unit])
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeRDDFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeRDDFunctionsTest.scala b/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeRDDFunctionsTest.scala
deleted file mode 100644
index 96e5f26..0000000
--- a/geode-spark-connector/geode-spark-connector/src/test/scala/unittest/io/pivotal/geode/spark/connector/GeodeRDDFunctionsTest.scala
+++ /dev/null
@@ -1,139 +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 unittest.io.pivotal.geode.spark.connector
-
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector._
-import io.pivotal.geode.spark.connector.internal.rdd.{GeodeRDDWriter, GeodePairRDDWriter}
-import org.apache.spark.{TaskContext, SparkContext}
-import org.apache.spark.rdd.RDD
-import org.mockito.Mockito._
-import org.scalatest.mock.MockitoSugar
-import org.scalatest.{FunSuite, Matchers}
-import collection.JavaConversions._
-import scala.reflect.ClassTag
-import org.mockito.Matchers.{eq => mockEq, any => mockAny}
-
-class GeodeRDDFunctionsTest extends FunSuite with Matchers with MockitoSugar {
-
-  test("test PairRDDFunction Implicit") {
-    import io.pivotal.geode.spark.connector._
-    val mockRDD = mock[RDD[(Int, String)]]
-    // the implicit make the following line valid
-    val pairRDD: GeodePairRDDFunctions[Int, String] = mockRDD
-    pairRDD shouldBe a [GeodePairRDDFunctions[_, _]]
-  }
-  
-  test("test RDDFunction Implicit") {
-    import io.pivotal.geode.spark.connector._
-    val mockRDD = mock[RDD[String]]
-    // the implicit make the following line valid
-    val nonPairRDD: GeodeRDDFunctions[String] = mockRDD
-    nonPairRDD shouldBe a [GeodeRDDFunctions[_]]
-  }
-
-  def createMocks[K, V](regionPath: String)
-    (implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]]): (String, GeodeConnectionConf, GeodeConnection, Region[K, V]) = {
-    val mockConnection = mock[GeodeConnection]
-    val mockConnConf = mock[GeodeConnectionConf]
-    val mockRegion = mock[Region[K, V]]
-    when(mockConnConf.getConnection).thenReturn(mockConnection)
-    when(mockConnection.getRegionProxy[K, V](regionPath)).thenReturn(mockRegion)
-    // mockRegion shouldEqual mockConn.getRegionProxy[K, V](regionPath)
-    (regionPath, mockConnConf, mockConnection, mockRegion)
-  }
-
-  test("test GeodePairRDDWriter") {
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
-    val writer = new GeodePairRDDWriter[String, String](regionPath, mockConnConf)
-    val data = List(("1", "one"), ("2", "two"), ("3", "three"))
-    writer.write(null, data.toIterator)
-    val expectedMap: Map[String, String] = data.toMap
-    verify(mockRegion).putAll(expectedMap)
-  }
-
-  test("test GeodeNonPairRDDWriter") {
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[Int, String]("test")
-    val writer = new GeodeRDDWriter[String, Int, String](regionPath, mockConnConf)
-    val data = List("a", "ab", "abc")
-    val f: String => (Int, String) = s => (s.length, s)
-    writer.write(f)(null, data.toIterator)
-    val expectedMap: Map[Int, String] = data.map(f).toMap
-    verify(mockRegion).putAll(expectedMap)
-  }
-  
-  test("test PairRDDFunctions.saveToGeode") {
-    verifyPairRDDFunction(useOpConf = false)
-  }
-
-  test("test PairRDDFunctions.saveToGeode w/ opConf") {
-    verifyPairRDDFunction(useOpConf = true)
-  }
-  
-  def verifyPairRDDFunction(useOpConf: Boolean): Unit = {
-    import io.pivotal.geode.spark.connector._
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
-    val mockRDD = mock[RDD[(String, String)]]
-    val mockSparkContext = mock[SparkContext]
-    when(mockRDD.sparkContext).thenReturn(mockSparkContext)
-    val result = 
-      if (useOpConf) 
-        mockRDD.saveToGeode(regionPath, mockConnConf, Map(RDDSaveBatchSizePropKey -> "5000"))
-      else
-        mockRDD.saveToGeode(regionPath, mockConnConf)
-    verify(mockConnection, times(1)).validateRegion[String, String](regionPath)
-    result === Unit
-    verify(mockSparkContext, times(1)).runJob[(String, String), Unit](
-      mockEq(mockRDD), mockAny[(TaskContext, Iterator[(String, String)]) => Unit])(mockAny(classOf[ClassTag[Unit]]))
-
-    // Note: current implementation make following code not compilable
-    //       so not negative test for this case
-    //  val rdd: RDD[(K, V)] = ...
-    //  rdd.saveToGeode(regionPath, s => (s.length, s))
-  }
-
-  test("test RDDFunctions.saveToGeode") {
-    verifyRDDFunction(useOpConf = false)
-  }
-
-  test("test RDDFunctions.saveToGeode w/ opConf") {
-    verifyRDDFunction(useOpConf = true)
-  }
-  
-  def verifyRDDFunction(useOpConf: Boolean): Unit = {
-    import io.pivotal.geode.spark.connector._
-    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[Int, String]("test")
-    val mockRDD = mock[RDD[(String)]]
-    val mockSparkContext = mock[SparkContext]
-    when(mockRDD.sparkContext).thenReturn(mockSparkContext)
-    val result = 
-      if (useOpConf)
-        mockRDD.saveToGeode(regionPath, s => (s.length, s), mockConnConf, Map(RDDSaveBatchSizePropKey -> "5000"))
-      else
-        mockRDD.saveToGeode(regionPath, s => (s.length, s), mockConnConf)
-    verify(mockConnection, times(1)).validateRegion[Int, String](regionPath)
-    result === Unit
-    verify(mockSparkContext, times(1)).runJob[String, Unit](
-      mockEq(mockRDD), mockAny[(TaskContext, Iterator[String]) => Unit])(mockAny(classOf[ClassTag[Unit]]))
-
-    // Note: current implementation make following code not compilable
-    //       so not negative test for this case
-    //  val rdd: RDD[T] = ...   // T is not a (K, V) tuple
-    //  rdd.saveToGeode(regionPath)
-  }
-  
-}



[04/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RDDJoinRegionIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RDDJoinRegionIntegrationTest.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RDDJoinRegionIntegrationTest.scala
deleted file mode 100644
index b7a1dda..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RDDJoinRegionIntegrationTest.scala
+++ /dev/null
@@ -1,300 +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 ittest.io.pivotal.geode.spark.connector
-
-import java.util.Properties
-
-import io.pivotal.geode.spark.connector._
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.internal.DefaultGeodeConnectionManager
-import ittest.io.pivotal.geode.spark.connector.testkit.GeodeCluster
-import ittest.io.pivotal.geode.spark.connector.testkit.IOUtils
-import org.apache.spark.{SparkContext, SparkConf}
-import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
-import java.util.{HashMap => JHashMap}
-
-class RDDJoinRegionIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GeodeCluster {
-
-  var sc: SparkContext = null
-  val numServers = 3
-  val numObjects = 1000
-
-  override def beforeAll() {
-    // start geode cluster, and spark context
-    val settings = new Properties()
-    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml")
-    settings.setProperty("num-of-servers", numServers.toString)
-    val locatorPort = GeodeCluster.start(settings)
-
-    // start spark context in local mode
-    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
-      "log4j.logger.io.pivotal.geode.spark.connector" -> "DEBUG")
-    val conf = new SparkConf()
-      .setAppName("RDDJoinRegionIntegrationTest")
-      .setMaster("local[2]")
-      .set(GeodeLocatorPropKey, s"localhost[$locatorPort]")
-    sc = new SparkContext(conf)
-  }
-
-  override def afterAll() {
-    // stop connection, spark context, and geode cluster
-    DefaultGeodeConnectionManager.closeConnection(GeodeConnectionConf(sc.getConf))
-    sc.stop()
-    GeodeCluster.stop()
-  }
-
-//  def matchMaps[K,V](map1:Map[K,V], map2:Map[K,V]) = {
-//    assert(map1.size == map2.size)
-//    map1.foreach(e => {
-//      assert(map2.contains(e._1))
-//      assert (e._2 == map2.get(e._1).get)
-//    })
-//  }
-  
-  // -------------------------------------------------------------------------------------------- 
-  // PairRDD.joinGeodeRegion[K2 <: K, V2](regionPath, connConf): GeodeJoinRDD[(K, V), K, V2]
-  // -------------------------------------------------------------------------------------------- 
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K, V2], replicated region", JoinTest) {
-    verifyPairRDDJoinRegionWithSameKeyType("rr_str_int_region")
-  }
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K, V2], partitioned region", JoinTest) {
-    verifyPairRDDJoinRegionWithSameKeyType("pr_str_int_region")
-  }
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K, V2], partitioned redundant region", JoinTest) {
-    verifyPairRDDJoinRegionWithSameKeyType("pr_r_str_int_region")
-  }
-
-  def verifyPairRDDJoinRegionWithSameKeyType(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => ("k_" + x, x*2))
-    val rdd = sc.parallelize(data)
-
-    val rdd2 = rdd.joinGeodeRegion[String, Int](regionPath, connConf)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (0 until 50).map(i => ((s"k_$i", i*2), i)).toMap
-    // matchMaps[(String, Int), Int](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-
-  // ------------------------------------------------------------------------------------------------------
-  // PairRDD.joinGeodeRegion[K2, V2](regionPath, ((K, V)) => K2, connConf): GeodeJoinRDD[(K, V), K2, V2]
-  // -------------------------------------------------------------------------------------------------------
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K2, V2], replicated region", JoinTest) {
-    verifyPairRDDJoinRegionWithDiffKeyType("rr_str_int_region")
-  }
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K2, V2], partitioned region", JoinTest) {
-    verifyPairRDDJoinRegionWithDiffKeyType("pr_str_int_region")
-  }
-
-  test("PairRDD.joinGeodeRegion: RDD[K, V] with Region[K2, V2], partitioned redundant region", JoinTest) {
-    verifyPairRDDJoinRegionWithDiffKeyType("pr_r_str_int_region")
-  }
-
-  def verifyPairRDDJoinRegionWithDiffKeyType(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => (x, x*2))
-    val rdd = sc.parallelize(data)
-
-    val func :((Int, Int)) => String = pair => s"k_${pair._1}"
-
-    val rdd2 = rdd.joinGeodeRegion[String, Int](regionPath, func /*, connConf*/)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (0 until 50).map(i => ((i, i*2), i)).toMap
-    // matchMaps[(Int, Int), Int](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-
-  // ------------------------------------------------------------------------------------------------ 
-  // PairRDD.outerJoinGeodeRegion[K2 <: K, V2](regionPath, connConf): GeodeJoinRDD[(K, V), K, V2]
-  // ------------------------------------------------------------------------------------------------ 
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K, V2], replicated region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithSameKeyType("rr_str_int_region")
-  }
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K, V2], partitioned region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithSameKeyType("pr_str_int_region")
-  }
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K, V2], partitioned redundant region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithSameKeyType("pr_r_str_int_region")
-  }
-
-  def verifyPairRDDOuterJoinRegionWithSameKeyType(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => ("k_" + x, x*2))
-    val rdd = sc.parallelize(data)
-
-    val rdd2 = rdd.outerJoinGeodeRegion[String, Int](regionPath /*, connConf*/)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (-5 until 50).map {
-      i => if (i < 0) ((s"k_$i", i * 2), None)
-      else ((s"k_$i", i*2), Some(i))}.toMap
-    // matchMaps[(String, Int), Option[Int]](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-
-  // ------------------------------------------------------------------------------------------------------
-  // PairRDD.joinGeodeRegion[K2, V2](regionPath, ((K, V)) => K2, connConf): GeodeJoinRDD[(K, V), K2, V2]
-  // -------------------------------------------------------------------------------------------------------
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K2, V2], replicated region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithDiffKeyType("rr_str_int_region")
-  }
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K2, V2], partitioned region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithDiffKeyType("pr_str_int_region")
-  }
-
-  test("PairRDD.outerJoinGeodeRegion: RDD[K, V] with Region[K2, V2], partitioned redundant region", OuterJoinTest) {
-    verifyPairRDDOuterJoinRegionWithDiffKeyType("pr_r_str_int_region")
-  }
-
-  def verifyPairRDDOuterJoinRegionWithDiffKeyType(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => (x, x*2))
-    val rdd = sc.parallelize(data)
-
-    val func :((Int, Int)) => String = pair => s"k_${pair._1}"
-
-    val rdd2 = rdd.outerJoinGeodeRegion[String, Int](regionPath, func, connConf)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (-5 until 50).map {
-      i => if (i < 0) ((i, i * 2), None)
-      else ((i, i*2), Some(i))}.toMap
-    // matchMaps[(Int, Int), Option[Int]](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-
-  // --------------------------------------------------------------------------------------------
-  // RDD.joinGeodeRegion[K, V](regionPath, T => K,  connConf): GeodeJoinRDD[T, K, V]
-  // --------------------------------------------------------------------------------------------
-
-  test("RDD.joinGeodeRegion: RDD[T] with Region[K, V], replicated region", JoinTest) {
-    verifyRDDJoinRegion("rr_str_int_region")
-  }
-
-  test("RDD.joinGeodeRegion: RDD[T] with Region[K, V], partitioned region", JoinTest) {
-    verifyRDDJoinRegion("pr_str_int_region")
-  }
-
-  test("RDD.joinGeodeRegion: RDD[T] with Region[K, V], partitioned redundant region", JoinTest) {
-    verifyRDDJoinRegion("pr_r_str_int_region")
-  }
-
-  def verifyRDDJoinRegion(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => s"k_$x")
-    val rdd = sc.parallelize(data)
-
-    val rdd2 = rdd.joinGeodeRegion[String, Int](regionPath, x => x, connConf)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (0 until 50).map(i => (s"k_$i", i)).toMap
-    // matchMaps[String, Int](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-
-  // --------------------------------------------------------------------------------------------
-  // RDD.outerJoinGeodeRegion[K, V](regionPath, T => K, connConf): GeodeJoinRDD[T, K, V]
-  // --------------------------------------------------------------------------------------------
-
-  test("RDD.outerJoinGeodeRegion: RDD[T] with Region[K, V], replicated region", OnlyTest) {
-    verifyRDDOuterJoinRegion("rr_str_int_region")
-  }
-
-  test("RDD.outerJoinGeodeRegion: RDD[T] with Region[K, V], partitioned region", OnlyTest) {
-    verifyRDDOuterJoinRegion("pr_str_int_region")
-  }
-
-  test("RDD.outerJoinGeodeRegion: RDD[T] with Region[K, V], partitioned redundant region", OnlyTest) {
-    verifyRDDOuterJoinRegion("pr_r_str_int_region")
-  }
-
-  def verifyRDDOuterJoinRegion(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("k_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val data = (-5 until 50).map(x => s"k_$x")
-    val rdd = sc.parallelize(data)
-
-    val rdd2 = rdd.outerJoinGeodeRegion[String, Int](regionPath, x => x /*, connConf */)
-    val rdd2Content = rdd2.collect()
-
-    val expectedMap = (-5 until 50).map {
-      i => if (i < 0) (s"k_$i", None)
-           else (s"k_$i", Some(i))}.toMap
-    // matchMaps[String, Option[Int]](expectedMap, rdd2Content.toMap)
-    assert(expectedMap == rdd2Content.toMap)
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RetrieveRegionIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RetrieveRegionIntegrationTest.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RetrieveRegionIntegrationTest.scala
deleted file mode 100644
index 1ad843e..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/RetrieveRegionIntegrationTest.scala
+++ /dev/null
@@ -1,253 +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 ittest.io.pivotal.geode.spark.connector
-
-import java.util.Properties
-
-import io.pivotal.geode.spark.connector._
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.internal.DefaultGeodeConnectionManager
-import ittest.io.pivotal.geode.spark.connector.testkit.GeodeCluster
-import ittest.io.pivotal.geode.spark.connector.testkit.IOUtils
-import org.apache.spark.{SparkContext, SparkConf}
-import org.scalatest.{Tag, BeforeAndAfterAll, FunSuite, Matchers}
-import java.util.{HashMap => JHashMap}
-
-
-class RetrieveRegionIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GeodeCluster {
-
-  var sc: SparkContext = null
-  val numServers = 4
-  val numObjects = 1000
-
-  override def beforeAll() {
-    // start geode cluster, and spark context
-    val settings = new Properties()
-    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml")
-    settings.setProperty("num-of-servers", numServers.toString)
-    val locatorPort = GeodeCluster.start(settings)
-
-    // start spark context in local mode
-    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
-                            "log4j.logger.io.pivotal.geode.spark.connector" -> "DEBUG")
-    val conf = new SparkConf()
-      .setAppName("RetrieveRegionIntegrationTest")
-      .setMaster("local[2]")
-      .set(GeodeLocatorPropKey, s"localhost[$locatorPort]")
-    sc = new SparkContext(conf)
-  }
-
-  override def afterAll() {
-    // stop connection, spark context, and geode cluster
-    DefaultGeodeConnectionManager.closeConnection(GeodeConnectionConf(sc.getConf))
-    sc.stop()
-    GeodeCluster.stop()
-  }
-  
-  def executeTest[K,V](regionName:String, numObjects:Int, entriesMap:java.util.Map[K,V]) = {
-    //Populate some data in the region
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[K, V] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-    verifyRetrieveRegion[K,V](regionName, entriesMap)
-  }
-    
-  def verifyRetrieveRegion[K,V](regionName:String, entriesMap:java.util.Map[K,V])  = {
-    val rdd = sc.geodeRegion(regionName)
-    val collectedObjs = rdd.collect()
-    collectedObjs should have length entriesMap.size
-    import scala.collection.JavaConverters._
-    matchMaps[K,V](entriesMap.asScala.toMap, collectedObjs.toMap)
-  }
- 
-  def matchMaps[K,V](map1:Map[K,V], map2:Map[K,V]) = {
-    assert(map1.size == map2.size)
-    map1.foreach(e => {
-      assert(map2.contains(e._1))
-      assert (e._2 == map2.get(e._1).get)
-      }
-    )
-  }
-  
-  //Retrieve region for Partitioned Region where some nodes are empty (empty iterator)
-  //This test has to run first...the rest of the tests always use the same num objects
-  test("Retrieve Region for PR where some nodes are empty (Empty Iterator)") {
-    val numObjects = numServers - 1
-    val entriesMap:JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
-    executeTest[String, Int]("rr_str_int_region", numObjects, entriesMap)
-  }
-
-  //Test for retrieving from region containing string key and int value
-  def verifyRetrieveStringStringRegion(regionName:String) = {
-    val entriesMap:JHashMap[String, String] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, "value_" + i))
-    executeTest[String, String](regionName, numObjects, entriesMap)
-  }
-
-  test("Retrieve Region with replicate redundant string string") {
-    verifyRetrieveStringStringRegion("rr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned string string") {
-    verifyRetrieveStringStringRegion("pr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned redundant string string") {
-    verifyRetrieveStringStringRegion("pr_r_obj_obj_region")
-  }
-  
-
-  //Test for retrieving from region containing string key and string value
-  def verifyRetrieveStringIntRegion(regionName:String) = {
-    val entriesMap:JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
-    executeTest[String, Int](regionName, numObjects, entriesMap)
-  }
-
-  test("Retrieve Region with replicate string int region") {
-    verifyRetrieveStringIntRegion("rr_str_int_region")
-  }
-
-  test("Retrieve Region with partitioned string int region") {
-    verifyRetrieveStringIntRegion("pr_str_int_region")
-  }
-
-  test("Retrieve Region with partitioned redundant string int region") {
-    verifyRetrieveStringIntRegion("pr_r_str_int_region")
-  }
-
-  //Tests for retrieving from region containing string key and object value
-  def verifyRetrieveStringObjectRegion(regionName:String) = {
-    val entriesMap:JHashMap[String, Object] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, new Employee("ename" + i, i)))
-    executeTest[String, Object](regionName, numObjects, entriesMap)
-  }
-
-  test("Retrieve Region with replicate string obj") {
-    verifyRetrieveStringObjectRegion("rr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned string obj") {
-    verifyRetrieveStringObjectRegion("pr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned redundant string obj") {
-    verifyRetrieveStringObjectRegion("pr_r_obj_obj_region")
-  }
-
-  //Test for retrieving from region containing string key and map value
-  def verifyRetrieveStringMapRegion(regionName:String) = {
-    val entriesMap:JHashMap[String,JHashMap[String,String]] = new JHashMap()
-    (0 until numObjects).map(i => {
-      val hashMap:JHashMap[String, String] = new JHashMap()
-      hashMap.put("mapKey:" + i, "mapValue:" + i)
-      entriesMap.put("key_" + i, hashMap)
-    })
-    executeTest(regionName, numObjects, entriesMap)
-  }
-
-  test("Retrieve Region with replicate string map region") {
-    verifyRetrieveStringMapRegion("rr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned string map region") {
-    verifyRetrieveStringMapRegion("pr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partitioned redundant string map region") {
-    verifyRetrieveStringMapRegion("pr_r_obj_obj_region")
-  }
-  
-  //Test and helpers specific for retrieving from region containing string key and byte[] value
-  def executeTestWithByteArrayValues[K](regionName:String, numObjects:Int, entriesMap:java.util.Map[K,Array[Byte]]) = {
-    //Populate some data in the region
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[K, Array[Byte]] = conn.getRegionProxy(regionName)
-    rgn.putAll(entriesMap)
-    verifyRetrieveRegionWithByteArrayValues[K](regionName, entriesMap)
-  }
-  
-  def verifyRetrieveRegionWithByteArrayValues[K](regionName:String, entriesMap:java.util.Map[K,Array[Byte]])  = {
-    val rdd = sc.geodeRegion(regionName)
-    val collectedObjs = rdd.collect()
-    collectedObjs should have length entriesMap.size
-    import scala.collection.JavaConverters._
-    matchByteArrayMaps[K](entriesMap.asScala.toMap, collectedObjs.toMap)
-  }
-  
-  def matchByteArrayMaps[K](map1:Map[K,Array[Byte]], map2:Map[K,Array[Byte]]) = {
-    map1.foreach(e => {
-      assert(map2.contains(e._1))
-      assert (java.util.Arrays.equals(e._2, map2.get(e._1).get))
-      }
-    )
-    assert(map1.size == map2.size)
-
-  }
-  
-  def verifyRetrieveStringByteArrayRegion(regionName:String) = {
-    val entriesMap:JHashMap[String, Array[Byte]] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, Array[Byte](192.toByte, 168.toByte, 0, i.toByte)))
-    executeTestWithByteArrayValues[String](regionName, numObjects, entriesMap)
-  }
-      
-  test("Retrieve Region with replicate region string byte[] region") {
-    verifyRetrieveStringByteArrayRegion("rr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partition region string byte[] region") {
-    verifyRetrieveStringByteArrayRegion("pr_obj_obj_region")
-  }
-
-  test("Retrieve Region with partition redundant region string byte[] region") {
-    verifyRetrieveStringByteArrayRegion("pr_r_obj_obj_region")
-  }
-
-  test("Retrieve Region with where clause on partitioned redundant region", FilterTest) {
-    verifyRetrieveRegionWithWhereClause("pr_r_str_int_region")
-  }
-
-  test("Retrieve Region with where clause on partitioned region", FilterTest) {
-    verifyRetrieveRegionWithWhereClause("pr_str_int_region")
-  }
-
-  test("Retrieve Region with where clause on replicated region", FilterTest) {
-    verifyRetrieveRegionWithWhereClause("rr_str_int_region")
-  }
-
-  def verifyRetrieveRegionWithWhereClause(regionPath: String): Unit = {
-    val entriesMap: JHashMap[String, Int] = new JHashMap()
-    (0 until numObjects).map(i => entriesMap.put("key_" + i, i))
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, Int] = conn.getRegionProxy(regionPath)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(entriesMap)
-
-    val rdd = sc.geodeRegion(regionPath).where("value.intValue() < 50")
-    val expectedMap = (0 until 50).map(i => (s"key_$i", i)).toMap
-    val collectedObjs = rdd.collect()
-    // collectedObjs should have length expectedMap.size
-    matchMaps[String, Int](expectedMap, collectedObjs.toMap)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/package.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/package.scala
deleted file mode 100644
index b8571d8..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/package.scala
+++ /dev/null
@@ -1,29 +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 ittest.io.pivotal.geode.spark
-
-import org.scalatest.Tag
-
-package object connector {
-
-  object OnlyTest extends Tag("OnlyTest")
-  object FetchDataTest extends Tag("FetchDateTest")
-  object FilterTest extends Tag("FilterTest")
-  object JoinTest extends Tag("JoinTest")
-  object OuterJoinTest extends Tag("OuterJoinTest")  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeCluster.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeCluster.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeCluster.scala
deleted file mode 100644
index 18b2fd7..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeCluster.scala
+++ /dev/null
@@ -1,47 +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 ittest.io.pivotal.geode.spark.connector.testkit
-
-import java.util.Properties
-
-trait GeodeCluster {
-  def startGeodeCluster(settings: Properties): Int = {
-    println("=== GeodeCluster start()")
-    GeodeCluster.start(settings)
-  }
-}
-
-object GeodeCluster {
-  private var geode: Option[GeodeRunner] = None
-
-  def start(settings: Properties): Int = {
-    geode.map(_.stopGeodeCluster()) // Clean up any old running Geode instances
-    val runner = new GeodeRunner(settings)
-    geode = Some(runner)
-    runner.getLocatorPort
-  }
-
-  def stop(): Unit = {
-    println("=== GeodeCluster shutdown: " + geode.toString)
-    geode match {
-      case None => println("Nothing to shutdown.")
-      case Some(runner) => runner.stopGeodeCluster()
-    }
-    geode = None
-    println("=== GeodeCluster shutdown finished.")
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeRunner.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeRunner.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeRunner.scala
deleted file mode 100644
index 725a012..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/GeodeRunner.scala
+++ /dev/null
@@ -1,148 +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 ittest.io.pivotal.geode.spark.connector.testkit
-
-import java.io.{IOException, File}
-import java.net.InetAddress
-import java.util.Properties
-import org.apache.commons.httpclient.HttpClient
-import org.apache.commons.io.FileUtils
-import org.apache.commons.io.filefilter.IOFileFilter
-
-/**
-* A class that manages Geode locator and servers.  Uses gfsh to
-* start and stop the locator and servers.
-*/
-class GeodeRunner(settings: Properties) {
-  val gfshCmd = new File(getCurrentDirectory, "../../geode-assembly/build/install/apache-geode/bin/gfsh").toString
-  val cacheXMLFile = settings.get("cache-xml-file")
-  val numServers: Int = settings.get("num-of-servers").asInstanceOf[String].toInt
-  val cwd = new File(".").getAbsolutePath
-  val geodeFunctionsTargetDir = new File("../geode-functions/target")
-  val testroot = "target/testgeode"
-  val classpath = new File(cwd, "target/scala-2.10/it-classes/")
-  val locatorPort = startGeodeCluster(numServers)
-
-  def getLocatorPort: Int = locatorPort
-
-  private def getCurrentDirectory = new File( "." ).getCanonicalPath
-  
-  private def startGeodeCluster(numServers: Int): Int = {
-    //ports(0) for Geode locator, the other ports are for Geode servers
-    val ports: Seq[Int] = IOUtils.getRandomAvailableTCPPorts(2 + numServers)
-    startGeodeLocator(ports(0), ports(1))
-    startGeodeServers(ports(0), ports.drop(2))
-    registerFunctions(ports(1))
-    ports(0)
-  }
-
-  private def startGeodeLocator(locatorPort: Int, jmxHttpPort:Int) {
-    println(s"=== GeodeRunner: starting locator on port $locatorPort")
-    val locatorDir = new File(cwd, s"$testroot/locator")
-    if (locatorDir.exists())
-      FileUtils.deleteDirectory(locatorDir)
-    IOUtils.mkdir(locatorDir)
-    new ProcessBuilder()
-      .command(gfshCmd, "start", "locator",
-        "--name=locator",
-        s"--dir=$locatorDir",
-        s"--port=$locatorPort",
-        s"--J=-Dgemfire.jmx-manager-http-port=$jmxHttpPort")
-      .inheritIO()
-      .start()
-
-    // Wait 30 seconds for locator to start
-    println(s"=== GeodeRunner: waiting for locator on port $locatorPort")
-    if (!IOUtils.waitForPortOpen(InetAddress.getByName("localhost"), locatorPort, 30000))
-      throw new IOException("Failed to start Geode locator.")
-    println(s"=== GeodeRunner: done waiting for locator on port $locatorPort")
-  }
-
-  private def startGeodeServers(locatorPort: Int, serverPorts: Seq[Int]) {
-    val procs = for (i <- 0 until serverPorts.length) yield {
-      println(s"=== GeodeRunner: starting server${i+1} with clientPort ${serverPorts(i)}")
-      val serverDir = new File(cwd, s"$testroot/server${i+1}")
-      if (serverDir.exists())
-        FileUtils.deleteDirectory(serverDir)
-      IOUtils.mkdir(serverDir)
-      new ProcessBuilder()
-        .command(gfshCmd, "start", "server",
-          s"--name=server${i+1}",
-          s"--locators=localhost[$locatorPort]",
-          s"--bind-address=localhost",
-          s"--server-port=${serverPorts(i)}",
-          s"--dir=$serverDir",
-          s"--cache-xml-file=$cacheXMLFile",
-          s"--classpath=$classpath")
-        .inheritIO()
-        .start()
-    }
-    procs.foreach(p => p.waitFor)
-    println(s"All $serverPorts.length servers have been started") 
-  }
-  
-  private def registerFunctions(jmxHttpPort:Int) {
-    import scala.collection.JavaConversions._
-    FileUtils.listFiles(geodeFunctionsTargetDir, fileFilter, dirFilter).foreach{  f => registerFunction(jmxHttpPort, f)}
-  }
-  
-  def fileFilter = new IOFileFilter {
-    def accept (file: File) = file.getName.endsWith(".jar") && file.getName.startsWith("geode-functions")
-    def accept (dir: File, name: String) = name.endsWith(".jar") && name.startsWith("geode-functions")
-  }
-  
-  def dirFilter = new IOFileFilter {
-    def accept (file: File) = file.getName.startsWith("scala")
-    def accept (dir: File, name: String) = name.startsWith("scala")
-  }
-  
-  private def registerFunction(jmxHttpPort:Int, jar:File) {
-    println("Deploying:" + jar.getName)
-    import io.pivotal.geode.spark.connector.GeodeFunctionDeployer
-    val deployer = new GeodeFunctionDeployer(new HttpClient())
-    deployer.deploy("localhost", jmxHttpPort, jar)
-  }
-
-  def stopGeodeCluster(): Unit = {
-    stopGeodeServers(numServers)
-    stopGeodeLocator()
-    if (!IOUtils.waitForPortClose(InetAddress.getByName("localhost"), getLocatorPort, 30000))
-      throw new IOException(s"Failed to stop Geode locator at port $getLocatorPort.")
-    println(s"Successfully stop Geode locator at port $getLocatorPort.")
-  }
-
-  private def stopGeodeLocator() {
-    println(s"=== GeodeRunner: stop locator")
-    val p = new ProcessBuilder()
-      .inheritIO()
-      .command(gfshCmd, "stop", "locator", s"--dir=$testroot/locator")
-      .start()
-     p.waitFor()
-  }
-
-  private def stopGeodeServers(numServers: Int) {
-   val procs = for (i <-1 to numServers) yield {
-       println(s"=== GeodeRunner: stop server $i.")
-       new ProcessBuilder()
-        .inheritIO()
-        .command(gfshCmd, "stop", "server", s"--dir=$testroot/server$i")
-        .start()
-   }
-   procs.foreach(p => p.waitFor())
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/IOUtils.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/IOUtils.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/IOUtils.scala
deleted file mode 100644
index 21a9232..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/testkit/IOUtils.scala
+++ /dev/null
@@ -1,94 +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 ittest.io.pivotal.geode.spark.connector.testkit
-
-import java.io.{File, IOException}
-import java.net.{InetAddress, Socket}
-import com.gemstone.gemfire.internal.AvailablePort
-import scala.util.Try
-import org.apache.log4j.PropertyConfigurator
-import java.util.Properties
-
-object IOUtils {
-
-  /** Makes a new directory or throws an `IOException` if it cannot be made */
-  def mkdir(dir: File): File = {
-    if (!dir.mkdirs())
-      throw new IOException(s"Could not create dir $dir")
-    dir
-  }
-
-  private def socketPortProb(host: InetAddress, port: Int) = Iterator.continually {
-    Try {
-      Thread.sleep(100)
-      new Socket(host, port).close()
-    }
-  }
-  
-  /**
-   * Waits until a port at the given address is open or timeout passes.
-   * @return true if managed to connect to the port, false if timeout happened first
-   */
-  def waitForPortOpen(host: InetAddress, port: Int, timeout: Long): Boolean = {
-    val startTime = System.currentTimeMillis()
-    socketPortProb(host, port)
-      .dropWhile(p => p.isFailure && System.currentTimeMillis() - startTime < timeout)
-      .next()
-      .isSuccess
-  }
-
-  /**
-   * Waits until a port at the given address is close or timeout passes.
-   * @return true if host:port is un-connect-able, false if timeout happened first
-   */
-  def waitForPortClose(host: InetAddress, port: Int, timeout: Long): Boolean = {
-    val startTime = System.currentTimeMillis()
-    socketPortProb(host, port)
-      .dropWhile(p => p.isSuccess && System.currentTimeMillis() - startTime < timeout)
-      .next()
-      .isFailure
-  }
-
-  /**
-   * Returns array of unique randomly available tcp ports of specified count.
-   */
-  def getRandomAvailableTCPPorts(count: Int): Seq[Int] =
-    (0 until count).map(x => AvailablePort.getRandomAvailablePortKeeper(AvailablePort.SOCKET))
-      .map{x => x.release(); x.getPort}.toArray
-
-  /**
-   * config a log4j properties used for integration tests
-   */
-  def configTestLog4j(level: String, props: (String, String)*): Unit = {
-    val pro = new Properties()
-    props.foreach(p => pro.put(p._1, p._2))
-    configTestLog4j(level, pro)
-  }
-
-  def configTestLog4j(level: String, props: Properties): Unit = {
-    val pro = new Properties()
-    pro.put("log4j.rootLogger", s"$level, console")
-    pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender")
-    pro.put("log4j.appender.console.target", "System.err")
-    pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout")
-    pro.put("log4j.appender.console.layout.ConversionPattern",
-      "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n")
-    pro.putAll(props)
-    PropertyConfigurator.configure(pro)
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
deleted file mode 100644
index 67f9e57..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
+++ /dev/null
@@ -1,28 +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.spark.streaming
-
-import org.apache.spark.util.ManualClock
-
-object ManualClockHelper {
-
-  def addToTime(ssc: StreamingContext, timeToAdd: Long): Unit = {
-    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    clock.advance(timeToAdd)
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala
deleted file mode 100644
index fce1e67..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala
+++ /dev/null
@@ -1,44 +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.spark.streaming
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.dstream.InputDStream
-
-import scala.reflect.ClassTag
-
-class TestInputDStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
-  extends InputDStream[T](ssc_) {
-
-  def start() {}
-
-  def stop() {}
-
-  def compute(validTime: Time): Option[RDD[T]] = {
-    logInfo("Computing RDD for time " + validTime)
-    val index = ((validTime - zeroTime) / slideDuration - 1).toInt
-    val selectedInput = if (index < input.size) input(index) else Seq[T]()
-
-    // lets us test cases where RDDs are not created
-    if (selectedInput == null)
-      return None
-
-    val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
-    logInfo("Created RDD " + rdd.id + " with " + selectedInput)
-    Some(rdd)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaDStreamFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaDStreamFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaDStreamFunctions.java
deleted file mode 100644
index e7c7cf9..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaDStreamFunctions.java
+++ /dev/null
@@ -1,86 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.streaming.GeodeDStreamFunctions;
-import org.apache.spark.api.java.function.PairFunction;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import java.util.Properties;
-
-import static io.pivotal.geode.spark.connector.javaapi.JavaAPIHelper.*;
-
-/**
- * A Java API wrapper over {@link org.apache.spark.streaming.api.java.JavaDStream}
- * to provide Geode Spark Connector functionality.
- *
- * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */ 
-public class GeodeJavaDStreamFunctions<T> {
-  
-  public final GeodeDStreamFunctions<T> dsf;
-
-  public GeodeJavaDStreamFunctions(JavaDStream<T> ds) {
-    this.dsf = new GeodeDStreamFunctions<T>(ds.dstream());
-  }
-
-  /**
-   * Save the JavaDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored  
-   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */
-  public <K, V> void saveToGeode(
-    String regionPath, PairFunction<T, K, V> func, GeodeConnectionConf connConf, Properties opConf) {
-    dsf.saveToGeode(regionPath, func, connConf, propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the JavaDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored  
-   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
-   * @param opConf the optional  parameters for this operation
-   */
-  public <K, V> void saveToGeode(
-          String regionPath, PairFunction<T, K, V> func, Properties opConf) {
-    dsf.saveToGeode(regionPath, func, dsf.defaultConnectionConf(), propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the JavaDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored
-   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   */
-  public <K, V> void saveToGeode(
-          String regionPath, PairFunction<T, K, V> func, GeodeConnectionConf connConf) {
-    dsf.saveToGeode(regionPath, func, connConf, emptyStrStrMap());
-  }
-
-  /**
-   * Save the JavaDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored
-   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
-   */
-  public <K, V> void saveToGeode(
-          String regionPath, PairFunction<T, K, V> func) {
-    dsf.saveToGeode(regionPath, func, dsf.defaultConnectionConf(), emptyStrStrMap());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairDStreamFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairDStreamFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairDStreamFunctions.java
deleted file mode 100644
index 2c83255..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairDStreamFunctions.java
+++ /dev/null
@@ -1,77 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.streaming.GeodePairDStreamFunctions;
-import org.apache.spark.streaming.api.java.JavaPairDStream;
-import java.util.Properties;
-
-import static io.pivotal.geode.spark.connector.javaapi.JavaAPIHelper.*;
-
-/**
- * A Java API wrapper over {@link org.apache.spark.streaming.api.java.JavaPairDStream}
- * to provide Geode Spark Connector functionality.
- *
- * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */
-public class GeodeJavaPairDStreamFunctions<K, V> {
-  
-  public final GeodePairDStreamFunctions<K, V> dsf;
-
-  public GeodeJavaPairDStreamFunctions(JavaPairDStream<K, V> ds) {    
-    this.dsf = new GeodePairDStreamFunctions<K, V>(ds.dstream());
-  }
-
-  /**
-   * Save the JavaPairDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored  
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the optional parameters for this operation
-   */  
-  public void saveToGeode(String regionPath, GeodeConnectionConf connConf, Properties opConf) {
-    dsf.saveToGeode(regionPath, connConf, propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the JavaPairDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored  
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   */
-  public void saveToGeode(String regionPath, GeodeConnectionConf connConf) {
-    dsf.saveToGeode(regionPath, connConf, emptyStrStrMap());
-  }
-
-  /**
-   * Save the JavaPairDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored
-   * @param opConf the optional parameters for this operation
-   */
-  public void saveToGeode(String regionPath, Properties opConf) {
-    dsf.saveToGeode(regionPath, dsf.defaultConnectionConf(), propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the JavaPairDStream to Geode key-value store.
-   * @param regionPath the full path of region that the DStream is stored
-   */
-  public void saveToGeode(String regionPath) {
-    dsf.saveToGeode(regionPath, dsf.defaultConnectionConf(), emptyStrStrMap());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairRDDFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairRDDFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairRDDFunctions.java
deleted file mode 100644
index 3278a5b..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaPairRDDFunctions.java
+++ /dev/null
@@ -1,238 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.GeodePairRDDFunctions;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeJoinRDD;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeOuterJoinRDD;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.function.Function;
-import scala.Option;
-import scala.Tuple2;
-import scala.reflect.ClassTag;
-
-import java.util.Properties;
-
-import static io.pivotal.geode.spark.connector.javaapi.JavaAPIHelper.*;
-
-/**
- * A Java API wrapper over {@link org.apache.spark.api.java.JavaPairRDD} to provide Geode Spark
- * Connector functionality.
- *
- * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */
-public class GeodeJavaPairRDDFunctions<K, V> {
-
-  public final GeodePairRDDFunctions<K, V> rddf;
-
-  public GeodeJavaPairRDDFunctions(JavaPairRDD<K, V> rdd) {
-    this.rddf = new GeodePairRDDFunctions<K, V>(rdd.rdd());
-  }
-
-  /**
-   * Save the pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the parameters for this operation
-   */
-  public void saveToGeode(String regionPath, GeodeConnectionConf connConf, Properties opConf) {
-    rddf.saveToGeode(regionPath, connConf, propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored
-   * @param opConf the parameters for this operation
-   */
-  public void saveToGeode(String regionPath, Properties opConf) {
-    rddf.saveToGeode(regionPath, rddf.defaultConnectionConf(), propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   */
-  public void saveToGeode(String regionPath, GeodeConnectionConf connConf) {
-    rddf.saveToGeode(regionPath, connConf, emptyStrStrMap());
-  }
-
-  /**
-   * Save the pair RDD to Geode key-value store with the default GeodeConnector.
-   * @param regionPath the full path of region that the RDD is stored
-   */
-  public void saveToGeode(String regionPath) {
-    rddf.saveToGeode(regionPath, rddf.defaultConnectionConf(), emptyStrStrMap());
-  }
-
-  /**
-   * Return an JavaPairRDD containing all pairs of elements with matching keys in
-   * this RDD&lt;K, V> and the Geode `Region&lt;K, V2>`. Each pair of elements
-   * will be returned as a ((k, v), v2) tuple, where (k, v) is in this RDD and
-   * (k, v2) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;&lt;K, V>, V2>
-   */  
-  public <V2> JavaPairRDD<Tuple2<K, V>, V2> joinGeodeRegion(String regionPath) {
-    return joinGeodeRegion(regionPath, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Return an JavaPairRDD containing all pairs of elements with matching keys in
-   * this RDD&lt;K, V> and the Geode `Region&lt;K, V2>`. Each pair of elements
-   * will be returned as a ((k, v), v2) tuple, where (k, v) is in this RDD and
-   * (k, v2) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;&lt;K, V>, V2>
-   */
-  public <V2> JavaPairRDD<Tuple2<K, V>, V2> joinGeodeRegion(
-    String regionPath, GeodeConnectionConf connConf) {
-    GeodeJoinRDD<Tuple2<K, V>, K, V2> rdd = rddf.joinGeodeRegion(regionPath, connConf);
-    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
-    ClassTag<V2> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in this
-   * RDD&lt;K, V> and the Geode `Region&lt;K2, V2>`. The join key from RDD
-   * element is generated by `func(K, V) => K2`, and the key from the Geode
-   * region is just the key of the key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
-   * where (k, v) is in this RDD and (k2, v2) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param <K2> the key type of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, V2>
-   */
-  public <K2, V2> JavaPairRDD<Tuple2<K, V>, V2> joinGeodeRegion(
-    String regionPath, Function<Tuple2<K, V>, K2> func) {
-    return joinGeodeRegion(regionPath, func, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in this
-   * RDD&lt;K, V> and the Geode `Region&lt;K2, V2>`. The join key from RDD 
-   * element is generated by `func(K, V) => K2`, and the key from the Geode 
-   * region is just the key of the key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
-   * where (k, v) is in this RDD and (k2, v2) is in the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <K2> the key type of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, V2>
-   */  
-  public <K2, V2> JavaPairRDD<Tuple2<K, V>, V2> joinGeodeRegion(
-    String regionPath, Function<Tuple2<K, V>, K2> func, GeodeConnectionConf connConf) {
-    GeodeJoinRDD<Tuple2<K, V>, K2, V2> rdd = rddf.joinGeodeRegion(regionPath, func, connConf);
-    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
-    ClassTag<V2> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;K, V> and the Geode `Region&lt;K, V2>`.
-   * For each element (k, v) in this RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key k.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
-   */
-  public <V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGeodeRegion(String regionPath) {
-    return outerJoinGeodeRegion(regionPath, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;K, V> and the Geode `Region&lt;K, V2>`.
-   * For each element (k, v) in this RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key k.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
-   */  
-  public <V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGeodeRegion(
-    String regionPath, GeodeConnectionConf connConf) {
-    GeodeOuterJoinRDD<Tuple2<K, V>, K, V2> rdd = rddf.outerJoinGeodeRegion(regionPath, connConf);
-    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
-    ClassTag<Option<V2>> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;K, V> and the Geode `Region&lt;K2, V2>`.
-   * The join key from RDD element is generated by `func(K, V) => K2`, and the
-   * key from region is just the key of the key/value pair.
-   *
-   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key `func(k, v)`.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param <K2> the key type of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
-   */
-  public <K2, V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGeodeRegion(
-    String regionPath, Function<Tuple2<K, V>, K2> func) {
-    return outerJoinGeodeRegion(regionPath, func, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;K, V> and the Geode `Region&lt;K2, V2>`.
-   * The join key from RDD element is generated by `func(K, V) => K2`, and the
-   * key from region is just the key of the key/value pair.
-   *
-   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
-   * all pairs ((k, v), Some(v2)) for v2 in the Geode region, or the pair
-   * ((k, v), None)) if no element in the Geode region have key `func(k, v)`.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element (K, V)
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <K2> the key type of the Geode region
-   * @param <V2> the value type of the Geode region
-   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
-   */
-  public <K2, V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGeodeRegion(
-    String regionPath, Function<Tuple2<K, V>, K2> func, GeodeConnectionConf connConf) {
-    GeodeOuterJoinRDD<Tuple2<K, V>, K2, V2> rdd = rddf.outerJoinGeodeRegion(regionPath, func, connConf);
-    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
-    ClassTag<Option<V2>> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRDDFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRDDFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRDDFunctions.java
deleted file mode 100644
index e4f6f36..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaRDDFunctions.java
+++ /dev/null
@@ -1,178 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.GeodeRDDFunctions;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeJoinRDD;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeOuterJoinRDD;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.PairFunction;
-import scala.Option;
-import scala.reflect.ClassTag;
-
-import java.util.Properties;
-
-import static io.pivotal.geode.spark.connector.javaapi.JavaAPIHelper.*;
-
-/**
- * A Java API wrapper over {@link org.apache.spark.api.java.JavaRDD} to provide Geode Spark
- * Connector functionality.
- *
- * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */
-public class GeodeJavaRDDFunctions<T> {
-
-  public final GeodeRDDFunctions<T> rddf;
-
-  public GeodeJavaRDDFunctions(JavaRDD<T> rdd) {
-    this.rddf = new GeodeRDDFunctions<T>(rdd.rdd());
-  }
-
-  /**
-   * Save the non-pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored  
-   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param opConf the parameters for this operation
-   */  
-  public <K, V> void saveToGeode(
-    String regionPath, PairFunction<T, K, V> func, GeodeConnectionConf connConf, Properties opConf) {
-    rddf.saveToGeode(regionPath, func, connConf, propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the non-pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored  
-   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   */
-  public <K, V> void saveToGeode(
-    String regionPath, PairFunction<T, K, V> func, GeodeConnectionConf connConf) {
-    rddf.saveToGeode(regionPath, func, connConf, emptyStrStrMap());
-  }
-
-  /**
-   * Save the non-pair RDD to Geode key-value store.
-   * @param regionPath the full path of region that the RDD is stored
-   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
-   * @param opConf the parameters for this operation
-   */
-  public <K, V> void saveToGeode(
-    String regionPath, PairFunction<T, K, V> func, Properties opConf) {
-    rddf.saveToGeode(regionPath, func, rddf.defaultConnectionConf(), propertiesToScalaMap(opConf));
-  }
-
-  /**
-   * Save the non-pair RDD to Geode key-value store with default GeodeConnector.
-   * @param regionPath the full path of region that the RDD is stored  
-   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
-   */
-  public <K, V> void saveToGeode(String regionPath, PairFunction<T, K, V> func) {
-    rddf.saveToGeode(regionPath, func, rddf.defaultConnectionConf(), emptyStrStrMap());
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in this
-   * RDD&lt;T> and the Geode `Region&lt;K, V>`. The join key from RDD
-   * element is generated by `func(T) => K`, and the key from the Geode
-   * region is just the key of the key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a (t, v2) tuple,
-   * where t is from this RDD and v is from the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element T
-   * @param <K> the key type of the Geode region
-   * @param <V> the value type of the Geode region
-   * @return JavaPairRDD&lt;T, V>
-   */
-  public <K, V> JavaPairRDD<T, V> joinGeodeRegion(String regionPath, Function<T, K> func) {
-    return joinGeodeRegion(regionPath, func, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Return an RDD containing all pairs of elements with matching keys in this
-   * RDD&lt;T> and the Geode `Region&lt;K, V>`. The join key from RDD
-   * element is generated by `func(T) => K`, and the key from the Geode
-   * region is just the key of the key/value pair.
-   *
-   * Each pair of elements of result RDD will be returned as a (t, v2) tuple,
-   * where t is from this RDD and v is from the Geode region.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element T
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <K> the key type of the Geode region
-   * @param <V> the value type of the Geode region
-   * @return JavaPairRDD&lt;T, V>
-   */
-  public <K, V> JavaPairRDD<T, V> joinGeodeRegion(
-    String regionPath, Function<T, K> func, GeodeConnectionConf connConf) {
-    GeodeJoinRDD<T, K, V> rdd = rddf.joinGeodeRegion(regionPath, func, connConf);
-    ClassTag<T> kt = fakeClassTag();
-    ClassTag<V> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;T> and the Geode `Region&lt;K, V>`.
-   * The join key from RDD element is generated by `func(T) => K`, and the
-   * key from region is just the key of the key/value pair.
-   *
-   * For each element (t) in this RDD, the resulting RDD will either contain
-   * all pairs (t, Some(v)) for v in the Geode region, or the pair
-   * (t, None) if no element in the Geode region have key `func(t)`.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element T
-   * @param <K> the key type of the Geode region
-   * @param <V> the value type of the Geode region
-   * @return JavaPairRDD&lt;T, Option&lt;V>>
-   */
-  public <K, V> JavaPairRDD<T, Option<V>> outerJoinGeodeRegion(String regionPath, Function<T, K> func) {
-    return outerJoinGeodeRegion(regionPath, func, rddf.defaultConnectionConf());
-  }
-
-  /**
-   * Perform a left outer join of this RDD&lt;T> and the Geode `Region&lt;K, V>`.
-   * The join key from RDD element is generated by `func(T) => K`, and the
-   * key from region is just the key of the key/value pair.
-   *
-   * For each element (t) in this RDD, the resulting RDD will either contain
-   * all pairs (t, Some(v)) for v in the Geode region, or the pair
-   * (t, None) if no element in the Geode region have key `func(t)`.
-   *
-   * @param regionPath the region path of the Geode region
-   * @param func the function that generates region key from RDD element T
-   * @param connConf the GeodeConnectionConf object that provides connection to Geode cluster
-   * @param <K> the key type of the Geode region
-   * @param <V> the value type of the Geode region
-   * @return JavaPairRDD&lt;T, Option&lt;V>>
-   */
-  public <K, V> JavaPairRDD<T, Option<V>> outerJoinGeodeRegion(
-    String regionPath, Function<T, K> func, GeodeConnectionConf connConf) {
-    GeodeOuterJoinRDD<T, K, V> rdd = rddf.outerJoinGeodeRegion(regionPath, func, connConf);
-    ClassTag<T> kt = fakeClassTag();
-    ClassTag<Option<V>> vt = fakeClassTag();
-    return new JavaPairRDD<>(rdd, kt, vt);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSQLContextFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSQLContextFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSQLContextFunctions.java
deleted file mode 100644
index 3471bf90..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSQLContextFunctions.java
+++ /dev/null
@@ -1,49 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.GeodeSQLContextFunctions;
-import org.apache.spark.sql.DataFrame;
-import org.apache.spark.sql.SQLContext;
-
-/**
- * Java API wrapper over {@link org.apache.spark.sql.SQLContext} to provide Geode
- * OQL functionality.
- *
- * <p></p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */
-public class GeodeJavaSQLContextFunctions {
-
-  public final GeodeSQLContextFunctions scf;
-
-  public GeodeJavaSQLContextFunctions(SQLContext sqlContext) {
-    scf = new GeodeSQLContextFunctions(sqlContext);
-  }
-
-  public <T> DataFrame geodeOQL(String query) {
-    DataFrame df = scf.geodeOQL(query, scf.defaultConnectionConf());
-    return df;
-  }
-
-  public <T> DataFrame geodeOQL(String query, GeodeConnectionConf connConf) {
-    DataFrame df = scf.geodeOQL(query, connConf);
-    return df;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSparkContextFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSparkContextFunctions.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSparkContextFunctions.java
deleted file mode 100644
index ce6b1ff..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaSparkContextFunctions.java
+++ /dev/null
@@ -1,87 +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 io.pivotal.geode.spark.connector.javaapi;
-
-
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRegionRDD;
-import io.pivotal.geode.spark.connector.internal.rdd.GeodeRegionRDD$;
-import org.apache.spark.SparkContext;
-import static io.pivotal.geode.spark.connector.javaapi.JavaAPIHelper.*;
-
-import scala.reflect.ClassTag;
-import java.util.Properties;
-
-/**
- * Java API wrapper over {@link org.apache.spark.SparkContext} to provide Geode
- * Connector functionality.
- *
- * <p></p>To obtain an instance of this wrapper, use one of the factory methods in {@link
- * io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil} class.</p>
- */
-public class GeodeJavaSparkContextFunctions {
-
-  public final SparkContext sc;
-
-  public GeodeJavaSparkContextFunctions(SparkContext sc) {
-    this.sc = sc;
-  }
-
-  /**
-   * Expose a Geode region as a JavaPairRDD
-   * @param regionPath the full path of the region
-   * @param connConf the GeodeConnectionConf that can be used to access the region
-   * @param opConf the parameters for this operation, such as preferred partitioner.
-   */
-  public <K, V> GeodeJavaRegionRDD<K, V> geodeRegion(
-    String regionPath, GeodeConnectionConf connConf, Properties opConf) {
-    ClassTag<K> kt = fakeClassTag();
-    ClassTag<V> vt = fakeClassTag();    
-    GeodeRegionRDD<K, V>  rdd =  GeodeRegionRDD$.MODULE$.apply(
-      sc, regionPath, connConf, propertiesToScalaMap(opConf), kt, vt);
-    return new GeodeJavaRegionRDD<>(rdd);
-  }
-
-  /**
-   * Expose a Geode region as a JavaPairRDD with default GeodeConnector and no preferred partitioner.
-   * @param regionPath the full path of the region
-   */
-  public <K, V> GeodeJavaRegionRDD<K, V> geodeRegion(String regionPath) {
-    GeodeConnectionConf connConf = GeodeConnectionConf.apply(sc.getConf());
-    return geodeRegion(regionPath, connConf, new Properties());
-  }
-
-  /**
-   * Expose a Geode region as a JavaPairRDD with no preferred partitioner.
-   * @param regionPath the full path of the region
-   * @param connConf the GeodeConnectionConf that can be used to access the region
-   */
-  public <K, V> GeodeJavaRegionRDD<K, V> geodeRegion(String regionPath, GeodeConnectionConf connConf) {
-    return geodeRegion(regionPath, connConf, new Properties());
-  }
-
-  /**
-   * Expose a Geode region as a JavaPairRDD with default GeodeConnector.
-   * @param regionPath the full path of the region
-   * @param opConf the parameters for this operation, such as preferred partitioner.
-   */
-  public <K, V> GeodeJavaRegionRDD<K, V> geodeRegion(String regionPath, Properties opConf) {
-    GeodeConnectionConf connConf = GeodeConnectionConf.apply(sc.getConf());
-    return geodeRegion(regionPath, connConf, opConf);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaUtil.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaUtil.java b/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaUtil.java
deleted file mode 100644
index 41fe7e5..0000000
--- a/geode-spark-connector/geode-spark-connector/src/main/java/io/pivotal/geode/spark/connector/javaapi/GeodeJavaUtil.java
+++ /dev/null
@@ -1,122 +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 io.pivotal.geode.spark.connector.javaapi;
-
-import org.apache.spark.SparkContext;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.sql.SQLContext;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.JavaPairDStream;
-import scala.Tuple2;
-
-import io.pivotal.geode.spark.connector.package$;
-
-/**
- * The main entry point to Spark Geode Connector Java API.
- *
- * There are several helpful static factory methods which build useful wrappers
- * around Spark Context, Streaming Context and RDD. There are also helper methods
- * to convert JavaRDD<Tuple2<K, V>> to JavaPairRDD<K, V>.
- */
-public final class GeodeJavaUtil {
-  
-  /** constants */
-  public static String GeodeLocatorPropKey = package$.MODULE$.GeodeLocatorPropKey();
-  // partitioner related keys and values
-  public static String PreferredPartitionerPropKey = package$.MODULE$.PreferredPartitionerPropKey();
-  public static String NumberPartitionsPerServerPropKey = package$.MODULE$.NumberPartitionsPerServerPropKey();
-  public static String OnePartitionPartitionerName = package$.MODULE$.OnePartitionPartitionerName();
-  public static String ServerSplitsPartitionerName = package$.MODULE$.ServerSplitsPartitionerName();
-  public static String RDDSaveBatchSizePropKey = package$.MODULE$.RDDSaveBatchSizePropKey();
-  public static int RDDSaveBatchSizeDefault = package$.MODULE$.RDDSaveBatchSizeDefault();
-  
-  /** The private constructor is used prevents user from creating instance of this class. */
-  private GeodeJavaUtil() { }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaSparkContextFunctions} based
-   * on an existing {@link SparkContext} instance.
-   */
-  public static GeodeJavaSparkContextFunctions javaFunctions(SparkContext sc) {
-    return new GeodeJavaSparkContextFunctions(sc);
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaSparkContextFunctions} based
-   * on an existing {@link JavaSparkContext} instance.
-   */
-  public static GeodeJavaSparkContextFunctions javaFunctions(JavaSparkContext jsc) {
-    return new GeodeJavaSparkContextFunctions(JavaSparkContext.toSparkContext(jsc));
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaPairRDDFunctions} based on an
-   * existing {@link org.apache.spark.api.java.JavaPairRDD} instance.
-   */
-  public static <K, V> GeodeJavaPairRDDFunctions<K, V> javaFunctions(JavaPairRDD<K, V> rdd) {
-    return new GeodeJavaPairRDDFunctions<K, V>(rdd);
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaRDDFunctions} based on an
-   * existing {@link org.apache.spark.api.java.JavaRDD} instance.
-   */
-  public static <T> GeodeJavaRDDFunctions<T> javaFunctions(JavaRDD<T> rdd) {
-    return new GeodeJavaRDDFunctions<T>(rdd);
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaPairDStreamFunctions} based on an
-   * existing {@link org.apache.spark.streaming.api.java.JavaPairDStream} instance.
-   */
-  public static <K, V> GeodeJavaPairDStreamFunctions<K, V> javaFunctions(JavaPairDStream<K, V> ds) {
-    return new GeodeJavaPairDStreamFunctions<>(ds);
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaDStreamFunctions} based on an
-   * existing {@link org.apache.spark.streaming.api.java.JavaDStream} instance.
-   */
-  public static <T> GeodeJavaDStreamFunctions<T> javaFunctions(JavaDStream<T> ds) {
-    return new GeodeJavaDStreamFunctions<>(ds);
-  }
-
-  /** Convert an instance of {@link org.apache.spark.api.java.JavaRDD}&lt;&lt;Tuple2&lt;K, V&gt;&gt;
-   * to a {@link org.apache.spark.api.java.JavaPairRDD}&lt;K, V&gt;.
-   */
-  public static <K, V> JavaPairRDD<K, V> toJavaPairRDD(JavaRDD<Tuple2<K, V>> rdd) {
-    return JavaAPIHelper.toJavaPairRDD(rdd);
-  }
-
-  /** Convert an instance of {@link org.apache.spark.streaming.api.java.JavaDStream}&lt;&lt;Tuple2&lt;K, V&gt;&gt;
-   * to a {@link org.apache.spark.streaming.api.java.JavaPairDStream}&lt;K, V&gt;.
-   */
-  public static <K, V> JavaPairDStream<K, V> toJavaPairDStream(JavaDStream<Tuple2<K, V>> ds) {
-    return JavaAPIHelper.toJavaPairDStream(ds);
-  }
-
-  /**
-   * A static factory method to create a {@link GeodeJavaSQLContextFunctions} based
-   * on an existing {@link SQLContext} instance.
-   */
-  public static GeodeJavaSQLContextFunctions javaFunctions(SQLContext sqlContext) {
-    return new GeodeJavaSQLContextFunctions(sqlContext);
-  }
-
-}


[05/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRegionRDDTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRegionRDDTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRegionRDDTest.scala
new file mode 100644
index 0000000..48f83c9
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRegionRDDTest.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 unittest.io.pivotal.gemfire.spark.connector.rdd
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFireRDDPartition, GemFireRegionRDD}
+import io.pivotal.gemfire.spark.connector.{GemFireConnectionConf, GemFireConnection}
+import org.apache.spark.{TaskContext, Partition, SparkContext}
+import org.mockito.Mockito._
+import org.mockito.Matchers.{eq => mockEq, any => mockAny}
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{Matchers, FunSuite}
+
+import scala.reflect.ClassTag
+
+class GemFireRegionRDDTest extends FunSuite with Matchers with MockitoSugar {
+
+  /** create common mocks, not all mocks are used by all tests */
+  def createMocks[K, V](regionPath: String)(implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]])
+    : (String, Region[K,V], GemFireConnectionConf, GemFireConnection) = {
+    val mockConnection = mock[GemFireConnection]
+    val mockRegion = mock[Region[K, V]]
+    val mockConnConf = mock[GemFireConnectionConf]
+    when(mockConnConf.getConnection).thenReturn(mockConnection)
+    when(mockConnection.getRegionProxy[K, V](regionPath)).thenReturn(mockRegion)
+    when(mockConnConf.locators).thenReturn(Seq.empty)
+    (regionPath, mockRegion, mockConnConf, mockConnection)
+  }
+  
+  test("create GemFireRDD with non-existing region") {
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    when(mockConnConf.getConnection).thenReturn(mockConnection)
+    when(mockConnection.validateRegion[String,String](regionPath)).thenThrow(new RuntimeException)
+    val mockSparkContext = mock[SparkContext]
+    intercept[RuntimeException] { GemFireRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf) }
+    verify(mockConnConf).getConnection
+    verify(mockConnection).validateRegion[String, String](regionPath)
+  }
+  
+  test("getPartitions with non-existing region") {
+    // region exists when RDD is created, but get removed before getPartitions() is invoked
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(None)
+    val mockSparkContext = mock[SparkContext]
+    intercept[RuntimeException] { GemFireRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf).getPartitions }
+  }
+
+  test("getPartitions with replicated region and not preferred env") {
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    implicit val mockConnConf2 = mockConnConf
+    val mockSparkContext = mock[SparkContext]
+    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, false, 0, null)))
+    val partitions = GemFireRegionRDD(mockSparkContext, regionPath, mockConnConf).partitions
+    verifySinglePartition(partitions)
+  }
+
+  def verifySinglePartition(partitions: Array[Partition]): Unit = {
+    assert(1 == partitions.size)
+    assert(partitions(0).index === 0)
+    assert(partitions(0).isInstanceOf[GemFireRDDPartition])
+    assert(partitions(0).asInstanceOf[GemFireRDDPartition].bucketSet.isEmpty)
+  }
+
+  test("getPartitions with replicated region and preferred OnePartitionPartitioner") {
+    // since it's replicated region, so OnePartitionPartitioner will be used, i.e., override preferred partitioner
+    import io.pivotal.gemfire.spark.connector.{PreferredPartitionerPropKey, OnePartitionPartitionerName}
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, false, 0, null)))
+    implicit val mockConnConf2 = mockConnConf
+    val mockSparkContext = mock[SparkContext]
+    val env = Map(PreferredPartitionerPropKey -> OnePartitionPartitionerName)
+    val partitions = GemFireRegionRDD(mockSparkContext, regionPath, mockConnConf, env).partitions
+    verifySinglePartition(partitions)
+  }
+
+  test("getPartitions with partitioned region and not preferred env") {
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    implicit val mockConnConf2 = mockConnConf
+    val mockSparkContext = mock[SparkContext]
+    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, true, 2, null)))
+    val partitions = GemFireRegionRDD(mockSparkContext, regionPath, mockConnConf).partitions
+    verifySinglePartition(partitions)
+  }
+
+  test("GemFireRDD.compute() method") {
+    val (regionPath, mockRegion, mockConnConf, mockConnection) = createMocks[String, String]("test")
+    implicit val mockConnConf2 = mockConnConf
+    val mockIter = mock[Iterator[(String, String)]]
+    val partition = GemFireRDDPartition(0, Set.empty)
+    when(mockConnection.getRegionMetadata[String, String](regionPath)).thenReturn(Some(new RegionMetadata(regionPath, true, 2, null)))
+    when(mockConnection.getRegionData[String, String](regionPath, None, partition)).thenReturn(mockIter)
+    val mockSparkContext = mock[SparkContext]
+    val rdd = GemFireRegionRDD[String, String](mockSparkContext, regionPath, mockConnConf)
+    val partitions = rdd.partitions
+    assert(1 == partitions.size)
+    val mockTaskContext = mock[TaskContext]
+    rdd.compute(partitions(0), mockTaskContext)        
+    verify(mockConnection).getRegionData[String, String](mockEq(regionPath), mockEq(None), mockEq(partition))
+    // verify(mockConnection).getRegionData[String, String](regionPath, Set.empty.asInstanceOf[Set[Int]], "gemfireRDD 0.0")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/Emp.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/Emp.java b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/Emp.java
new file mode 100644
index 0000000..03e15a0
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/Emp.java
@@ -0,0 +1,95 @@
+/*
+ * 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 demo;
+
+import java.io.Serializable;
+
+/**
+ * This is a demo class used in doc/?.md
+ */
+public class Emp implements Serializable {
+
+  private int id;
+  
+  private String lname;
+
+  private String fname;
+
+  private int age;
+
+  private String loc;
+
+  public Emp(int id, String lname, String fname, int age, String loc) {
+    this.id = id;
+    this.lname = lname;
+    this.fname = fname;
+    this.age = age;
+    this.loc = loc;
+  }
+
+  public int getId() {
+    return id;
+  }
+
+  public String getLname() {
+    return lname;
+  }
+
+  public String getFname() {
+    return fname;
+  }
+
+  public int getAge() {
+    return age;
+  }
+
+  public String getLoc() {
+    return loc;
+  }
+
+  @Override
+  public String toString() {
+    return "Emp(" + id + ", " + lname + ", " + fname + ", " + age + ", " + loc + ")";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    Emp emp = (Emp) o;
+
+    if (age != emp.age) return false;
+    if (id != emp.id) return false;
+    if (fname != null ? !fname.equals(emp.fname) : emp.fname != null) return false;
+    if (lname != null ? !lname.equals(emp.lname) : emp.lname != null) return false;
+    if (loc != null ? !loc.equals(emp.loc) : emp.loc != null) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = id;
+    result = 31 * result + (lname != null ? lname.hashCode() : 0);
+    result = 31 * result + (fname != null ? fname.hashCode() : 0);
+    result = 31 * result + age;
+    result = 31 * result + (loc != null ? loc.hashCode() : 0);
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
new file mode 100644
index 0000000..41654a5
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/OQLJavaDemo.java
@@ -0,0 +1,59 @@
+/*
+ * 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 demo;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.SQLContext;
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
+
+
+/**
+ * This Spark application demonstrates how to get region data from GemFire using GemFire
+ * OQL Java API. The result is a Spark DataFrame.
+ * <p>
+ * In order to run it, you will need to start a GemFire cluster, and run demo PairRDDSaveJavaDemo
+ * first to create some data in the region.
+ * <p>
+ * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
+ * should be generated under gemfire-spark-demos/basic-demos/target/scala-2.10/. 
+ * Then run the following command to start a Spark job:
+ * <pre>
+ *   <path to spark>/bin/spark-submit --master=local[2] --class demo.OQLJavaDemo \
+ *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
+ * </pre>
+ */
+public class OQLJavaDemo {
+
+  public static void main(String[] argv) {
+
+    if (argv.length != 1) {
+      System.err.printf("Usage: OQLJavaDemo <locators>\n");
+      return;
+    }
+
+    SparkConf conf = new SparkConf().setAppName("OQLJavaDemo");
+    conf.set(GemFireLocatorPropKey, argv[0]); // "192.168.1.47[10335]"
+    JavaSparkContext sc = new JavaSparkContext(conf);
+    SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
+    DataFrame df = javaFunctions(sqlContext).gemfireOQL("select * from /str_str_region");
+    System.out.println("======= DataFrame =======\n");
+    df.show();
+    sc.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
new file mode 100644
index 0000000..84f87af
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/PairRDDSaveJavaDemo.java
@@ -0,0 +1,86 @@
+/*
+ * 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 demo;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import scala.Tuple2;
+import java.util.*;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
+
+/**
+ * This Spark application demonstrates how to save a RDD to GemFire using GemFire Spark 
+ * Connector with Java.
+ * <p/>
+ * In order to run it, you will need to start GemFire cluster, and create the following region
+ * with GFSH:
+ * <pre>
+ * gfsh> create region --name=str_str_region --type=REPLICATE \
+ *         --key-constraint=java.lang.String --value-constraint=java.lang.String
+ * </pre>
+ * 
+ * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
+ * should be generated under gemfire-spark-demos/basic-demos/target/scala-2.10/. 
+ * Then run the following command to start a Spark job:
+ * <pre>
+ *   <path to spark>/bin/spark-submit --master=local[2] --class demo.PairRDDSaveJavaDemo \
+ *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
+ * </pre>
+ * 
+ * Verify the data was saved to GemFire with GFSH:
+ * <pre>gfsh> query --query="select * from /str_str_region.entrySet"  </pre>
+ */
+public class PairRDDSaveJavaDemo {
+
+  public static void main(String[] argv) {
+
+    if (argv.length != 1) {
+      System.err.printf("Usage: PairRDDSaveJavaDemo <locators>\n");
+      return;
+    }
+
+    SparkConf conf = new SparkConf().setAppName("PairRDDSaveJavaDemo");
+    conf.set(GemFireLocatorPropKey, argv[0]);
+    JavaSparkContext sc = new JavaSparkContext(conf);
+    GemFireConnectionConf connConf = GemFireConnectionConf.apply(conf);
+
+    List<Tuple2<String, String>> data = new ArrayList<>();
+    data.add(new Tuple2<>("7", "seven"));
+    data.add(new Tuple2<>("8", "eight"));
+    data.add(new Tuple2<>("9", "nine"));
+
+    List<Tuple2<String, String>> data2 = new ArrayList<Tuple2<String, String>>();
+    data2.add(new Tuple2<>("11", "eleven"));
+    data2.add(new Tuple2<>("12", "twelve"));
+    data2.add(new Tuple2<>("13", "thirteen"));
+
+    // method 1: generate JavaPairRDD directly
+    JavaPairRDD<String, String> rdd1 =  sc.parallelizePairs(data);
+    javaFunctions(rdd1).saveToGemfire("str_str_region", connConf);
+
+    // method 2: convert JavaRDD<Tuple2<K,V>> to JavaPairRDD<K, V>
+    JavaRDD<Tuple2<String, String>> rdd2 =  sc.parallelize(data2);
+    javaFunctions(toJavaPairRDD(rdd2)).saveToGemfire("str_str_region", connConf);
+       
+    sc.stop();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
new file mode 100644
index 0000000..5fc5aeb
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RDDSaveJavaDemo.java
@@ -0,0 +1,85 @@
+/*
+ * 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 demo;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
+
+/**
+ * This Spark application demonstrates how to save a RDD to GemFire using GemFire Spark
+ * Connector with Java.
+ * <p/>
+ * In order to run it, you will need to start GemFire cluster, and create the following region
+ * with GFSH:
+ * <pre>
+ * gfsh> create region --name=str_int_region --type=REPLICATE \
+ *         --key-constraint=java.lang.String --value-constraint=java.lang.Integer
+ * </pre>
+ *
+ * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
+ * should be generated under gemfire-spark-demos/basic-demos/target/scala-2.10/.
+ * Then run the following command to start a Spark job:
+ * <pre>
+ *   <path to spark>/bin/spark-submit --master=local[2] --class demo.RDDSaveJavaDemo \
+ *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
+ * </pre>
+ *
+ * Verify the data was saved to GemFire with GFSH:
+ * <pre>gfsh> query --query="select * from /str_int_region.entrySet"  </pre>
+ */
+public class RDDSaveJavaDemo {
+
+  public static void main(String[] argv) {
+
+    if (argv.length != 1) {
+      System.err.printf("Usage: RDDSaveJavaDemo <locators>\n");
+      return;
+    }
+
+    SparkConf conf = new SparkConf().setAppName("RDDSaveJavaDemo");
+    conf.set(GemFireLocatorPropKey, argv[0]);
+    JavaSparkContext sc = new JavaSparkContext(conf);
+
+    List<String> data = new ArrayList<String>();
+    data.add("abcdefg");
+    data.add("abcdefgh");
+    data.add("abcdefghi");
+    JavaRDD<String> rdd =  sc.parallelize(data);
+
+    GemFireConnectionConf connConf = GemFireConnectionConf.apply(conf);
+
+    PairFunction<String, String, Integer> func =  new PairFunction<String, String, Integer>() {
+      @Override public Tuple2<String, Integer> call(String s) throws Exception {
+        return new Tuple2<String, Integer>(s, s.length());
+      }
+    };
+
+    javaFunctions(rdd).saveToGemfire("str_int_region", func, connConf);
+
+    sc.stop();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
new file mode 100644
index 0000000..7c1d7bb
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/java/demo/RegionToRDDJavaDemo.java
@@ -0,0 +1,57 @@
+/*
+ * 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 demo;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
+
+/**
+ * This Spark application demonstrates how to expose a region in GemFire as a RDD using GemFire
+ * Spark Connector with Java.
+ * <p>
+ * In order to run it, you will need to start GemFire cluster, and run demo PairRDDSaveJavaDemo
+ * first to create some data in the region.
+ * <p>
+ * Once you compile and package the demo, the jar file basic-demos_2.10-0.5.0.jar
+ * should be generated under gemfire-spark-demos/basic-demos/target/scala-2.10/. 
+ * Then run the following command to start a Spark job:
+ * <pre>
+ *   <path to spark>/bin/spark-submit --master=local[2] --class demo.RegionToRDDJavaDemo \
+ *       <path to>/basic-demos_2.10-0.5.0.jar <locator host>:<port>
+ * </pre>
+ */
+public class RegionToRDDJavaDemo {
+
+  public static void main(String[] argv) {
+
+    if (argv.length != 1) {
+      System.err.printf("Usage: RegionToRDDJavaDemo <locators>\n");
+      return;
+    }
+    
+    SparkConf conf = new SparkConf().setAppName("RegionToRDDJavaDemo"); 
+    conf.set(GemFireLocatorPropKey, argv[0]);
+    JavaSparkContext sc = new JavaSparkContext(conf);
+
+    JavaPairRDD<String, String> rdd = javaFunctions(sc).gemfireRegion("str_str_region");
+    System.out.println("=== gemfireRegion =======\n" + rdd.collect() + "\n=========================");
+    
+    sc.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
new file mode 100644
index 0000000..f67c32e
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-demos/basic-demos/src/main/scala/demo/NetworkWordCount.scala
@@ -0,0 +1,75 @@
+/*
+ * 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 demo
+
+import org.apache.spark.SparkConf
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import io.pivotal.gemfire.spark.connector.GemFireLocatorPropKey
+import io.pivotal.gemfire.spark.connector.streaming._
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ * <p><p>
+ * In order to run it, you will need to start GemFire cluster, and create the following region
+ * with GFSH:
+ * <pre>
+ * gfsh> create region --name=str_int_region --type=REPLICATE \
+ *         --key-constraint=java.lang.String --value-constraint=java.lang.Integer
+ * </pre> 
+ *
+ * <p>To run this on your local machine, you need to first run a net cat server
+ *    `$ nc -lk 9999`
+ * and then run the example
+ *    `$ bin/spark-submit --master=local[2] --class demo.NetworkWordCount <path to>/basic-demos_2.10-0.5.0.jar localhost 9999 locatorHost:port`
+ * 
+ * <p><p> check result that was saved to GemFire with GFSH:
+ * <pre>gfsh> query --query="select * from /str_int_region.entrySet"  </pre>
+ */
+object NetworkWordCount {
+  
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: NetworkWordCount <hostname> <port> <gemfire locator>")
+      System.exit(1)
+    }
+
+    val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+      val currentCount = values.foldLeft(0)(_ + _)
+      val previousCount = state.getOrElse(0)
+      Some(currentCount + previousCount)
+    }
+    
+    // Create the context with a 1 second batch size
+    val sparkConf = new SparkConf().setAppName("NetworkWordCount").set(GemFireLocatorPropKey, args(2))
+    val ssc = new StreamingContext(sparkConf, Seconds(1))
+    ssc.checkpoint(".")
+    
+    // Create a socket stream on target ip:port and count the
+    // words in input stream of \n delimited text (eg. generated by 'nc')
+    // Note that no duplication in storage level only for running locally.
+    // Replication necessary in distributed scenario for fault tolerance.
+    val lines = ssc.socketTextStream(args(0), args(1).toInt)
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+    val runningCounts = wordCounts.updateStateByKey[Int](updateFunc)
+    // runningCounts.print()
+    runningCounts.saveToGemfire("str_int_region")
+    ssc.start()
+    ssc.awaitTermination()
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Employee.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Employee.java b/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Employee.java
deleted file mode 100644
index 9fba9e1..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Employee.java
+++ /dev/null
@@ -1,54 +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 ittest.io.pivotal.geode.spark.connector;
-
-import java.io.Serializable;
-
-public class Employee implements Serializable {
-
-  private String name;
-
-  private int age;
-
-  public Employee(String n, int a) {
-    name = n;
-    age = a;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public int getAge() {
-    return age;
-  }
-
-  public String toString() {
-    return new StringBuilder().append("Employee[name=").append(name).
-            append(", age=").append(age).
-            append("]").toString();
-  }
-  
-  public boolean equals(Object o) {
-    if (o instanceof Employee) {
-      return ((Employee) o).name.equals(name) && ((Employee) o).age == age;
-    }
-    return false;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/JavaApiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/JavaApiIntegrationTest.java b/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/JavaApiIntegrationTest.java
deleted file mode 100644
index 8f5a045..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/JavaApiIntegrationTest.java
+++ /dev/null
@@ -1,424 +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 ittest.io.pivotal.geode.spark.connector;
-
-import com.gemstone.gemfire.cache.Region;
-import io.pivotal.geode.spark.connector.GeodeConnection;
-import io.pivotal.geode.spark.connector.GeodeConnectionConf;
-import io.pivotal.geode.spark.connector.GeodeConnectionConf$;
-import io.pivotal.geode.spark.connector.internal.DefaultGeodeConnectionManager$;
-import io.pivotal.geode.spark.connector.javaapi.GeodeJavaRegionRDD;
-import ittest.io.pivotal.geode.spark.connector.testkit.GeodeCluster$;
-import ittest.io.pivotal.geode.spark.connector.testkit.IOUtils;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.PairFunction;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.scalatest.junit.JUnitSuite;
-import io.pivotal.geode.spark.connector.package$;
-import scala.Tuple2;
-import scala.Option;
-import scala.Some;
-
-import java.util.*;
-
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.RDDSaveBatchSizePropKey;
-import static io.pivotal.geode.spark.connector.javaapi.GeodeJavaUtil.javaFunctions;
-import static org.junit.Assert.*;
-
-public class JavaApiIntegrationTest extends JUnitSuite {
-
-  static JavaSparkContext jsc = null;
-  static GeodeConnectionConf connConf = null;
-  
-  static int numServers = 2;
-  static int numObjects = 1000;
-  static String regionPath = "pr_str_int_region";
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // start geode cluster, and spark context
-    Properties settings = new Properties();
-    settings.setProperty("cache-xml-file", "src/it/resources/test-retrieve-regions.xml");
-    settings.setProperty("num-of-servers", Integer.toString(numServers));
-    int locatorPort = GeodeCluster$.MODULE$.start(settings);
-
-    // start spark context in local mode
-    Properties props = new Properties();
-    props.put("log4j.logger.org.apache.spark", "INFO");
-    props.put("log4j.logger.io.pivotal.geode.spark.connector","DEBUG");
-    IOUtils.configTestLog4j("ERROR", props);
-    SparkConf conf = new SparkConf()
-            .setAppName("RetrieveRegionIntegrationTest")
-            .setMaster("local[2]")
-            .set(package$.MODULE$.GeodeLocatorPropKey(), "localhost:"+ locatorPort);
-    // sc = new SparkContext(conf);
-    jsc = new JavaSparkContext(conf);
-    connConf = GeodeConnectionConf.apply(jsc.getConf());
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    // stop connection, spark context, and geode cluster
-    DefaultGeodeConnectionManager$.MODULE$.closeConnection(GeodeConnectionConf$.MODULE$.apply(jsc.getConf()));
-    jsc.stop();
-    GeodeCluster$.MODULE$.stop();
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   utility methods
-  // --------------------------------------------------------------------------------------------
-
-  private <K,V> void matchMapAndPairList(Map<K,V> map, List<Tuple2<K,V>> list) {
-    assertTrue("size mismatch \nmap: " + map.toString() + "\nlist: " + list.toString(), map.size() == list.size());
-    for (Tuple2<K, V> p : list) {
-      assertTrue("value mismatch: k=" + p._1() + " v1=" + p._2() + " v2=" + map.get(p._1()),
-                 p._2().equals(map.get(p._1())));
-    }
-  }
-
-  private Region<String, Integer> prepareStrIntRegion(String regionPath, int start, int stop) {
-    HashMap<String, Integer> entriesMap = new HashMap<>();
-    for (int i = start; i < stop; i ++) {
-      entriesMap.put("k_" + i, i);
-    }
-
-    GeodeConnection conn = connConf.getConnection();
-    Region<String, Integer> region = conn.getRegionProxy(regionPath);
-    region.removeAll(region.keySetOnServer());
-    region.putAll(entriesMap);
-    return region;
-  }
-
-  private JavaPairRDD<String, Integer> prepareStrIntJavaPairRDD(int start, int stop) {
-    List<Tuple2<String, Integer>> data = new ArrayList<>();
-    for (int i = start; i < stop; i ++) {
-      data.add(new Tuple2<>("k_" + i, i));
-    }
-    return jsc.parallelizePairs(data);
-  }
-
-  private JavaPairRDD<Integer, Integer> prepareIntIntJavaPairRDD(int start, int stop) {
-    List<Tuple2<Integer, Integer>> data = new ArrayList<>();
-    for (int i = start; i < stop; i ++) {
-      data.add(new Tuple2<>(i, i * 2));
-    }
-    return jsc.parallelizePairs(data);
-  }
-
-  private JavaRDD<Integer> prepareIntJavaRDD(int start, int stop) {
-    List<Integer> data = new ArrayList<>();
-    for (int i = start; i < stop; i ++) {
-      data.add(i);
-    }
-    return jsc.parallelize(data);
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaRDD.saveToGeode
-  // --------------------------------------------------------------------------------------------
-
-  static class IntToStrIntPairFunction implements PairFunction<Integer, String, Integer> {
-    @Override public Tuple2<String, Integer> call(Integer x) throws Exception {
-      return new Tuple2<>("k_" + x, x);
-    }
-  }
-
-  @Test
-  public void testRDDSaveToGeodeWithDefaultConnConfAndOpConf() throws Exception {
-    verifyRDDSaveToGeode(true, true);
-  }
-
-  @Test
-  public void testRDDSaveToGeodeWithDefaultConnConf() throws Exception {
-    verifyRDDSaveToGeode(true, false);
-  }
-  
-  @Test
-  public void testRDDSaveToGeodeWithConnConfAndOpConf() throws Exception {
-    verifyRDDSaveToGeode(false, true);
-  }
-
-  @Test
-  public void testRDDSaveToGeodeWithConnConf() throws Exception {
-    verifyRDDSaveToGeode(false, false);
-  }
-  
-  public void verifyRDDSaveToGeode(boolean useDefaultConnConf, boolean useOpConf) throws Exception {
-    Region<String, Integer> region = prepareStrIntRegion(regionPath, 0, 0);  // remove all entries
-    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(0, numObjects);
-
-    PairFunction<Integer, String, Integer> func = new IntToStrIntPairFunction();
-    Properties opConf = new Properties();
-    opConf.put(RDDSaveBatchSizePropKey, "200");
-
-    if (useDefaultConnConf) {
-      if (useOpConf)
-        javaFunctions(rdd1).saveToGeode(regionPath, func, opConf);
-      else
-        javaFunctions(rdd1).saveToGeode(regionPath, func);
-    } else {
-      if (useOpConf)
-        javaFunctions(rdd1).saveToGeode(regionPath, func, connConf, opConf);
-      else
-        javaFunctions(rdd1).saveToGeode(regionPath, func, connConf);
-    }
-    
-    Set<String> keys = region.keySetOnServer();
-    Map<String, Integer> map = region.getAll(keys);
-
-    List<Tuple2<String, Integer>> expectedList = new ArrayList<>();
-
-    for (int i = 0; i < numObjects; i ++) {
-      expectedList.add(new Tuple2<>("k_" + i, i));
-    }
-    matchMapAndPairList(map, expectedList);
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaPairRDD.saveToGeode
-  // --------------------------------------------------------------------------------------------
-
-  @Test
-  public void testPairRDDSaveToGeodeWithDefaultConnConfAndOpConf() throws Exception {
-    verifyPairRDDSaveToGeode(true, true);
-  }
-
-  @Test
-  public void testPairRDDSaveToGeodeWithDefaultConnConf() throws Exception {
-    verifyPairRDDSaveToGeode(true, false);
-  }
-  
-  @Test
-  public void testPairRDDSaveToGeodeWithConnConfAndOpConf() throws Exception {
-    verifyPairRDDSaveToGeode(false, true);
-  }
-
-  @Test
-  public void testPairRDDSaveToGeodeWithConnConf() throws Exception {
-    verifyPairRDDSaveToGeode(false, false);
-  }
-  
-  public void verifyPairRDDSaveToGeode(boolean useDefaultConnConf, boolean useOpConf) throws Exception {
-    Region<String, Integer> region = prepareStrIntRegion(regionPath, 0, 0);  // remove all entries
-    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(0, numObjects);
-    Properties opConf = new Properties();
-    opConf.put(RDDSaveBatchSizePropKey, "200");
-
-    if (useDefaultConnConf) {
-      if (useOpConf)
-        javaFunctions(rdd1).saveToGeode(regionPath, opConf);
-      else
-        javaFunctions(rdd1).saveToGeode(regionPath);
-    } else {
-      if (useOpConf)
-        javaFunctions(rdd1).saveToGeode(regionPath, connConf, opConf);
-      else
-        javaFunctions(rdd1).saveToGeode(regionPath, connConf);
-    }
-
-    Set<String> keys = region.keySetOnServer();
-    Map<String, Integer> map = region.getAll(keys);
-
-    List<Tuple2<String, Integer>> expectedList = new ArrayList<>();
-    for (int i = 0; i < numObjects; i ++) {
-      expectedList.add(new Tuple2<>("k_" + i, i));
-    }
-    matchMapAndPairList(map, expectedList);
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaSparkContext.geodeRegion and where clause
-  // --------------------------------------------------------------------------------------------
-
-  @Test
-  public void testJavaSparkContextGeodeRegion() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);  // remove all entries
-    Properties emptyProps = new Properties();
-    GeodeJavaRegionRDD<String, Integer> rdd1 = javaFunctions(jsc).geodeRegion(regionPath);
-    GeodeJavaRegionRDD<String, Integer> rdd2 = javaFunctions(jsc).geodeRegion(regionPath, emptyProps);
-    GeodeJavaRegionRDD<String, Integer> rdd3 = javaFunctions(jsc).geodeRegion(regionPath, connConf);
-    GeodeJavaRegionRDD<String, Integer> rdd4 = javaFunctions(jsc).geodeRegion(regionPath, connConf, emptyProps);
-    GeodeJavaRegionRDD<String, Integer> rdd5 = rdd1.where("value.intValue() < 50");
-
-    HashMap<String, Integer> expectedMap = new HashMap<>();
-    for (int i = 0; i < numObjects; i ++) {
-      expectedMap.put("k_" + i, i);
-    }
-
-    matchMapAndPairList(expectedMap, rdd1.collect());
-    matchMapAndPairList(expectedMap, rdd2.collect());
-    matchMapAndPairList(expectedMap, rdd3.collect());
-    matchMapAndPairList(expectedMap, rdd4.collect());
-
-    HashMap<String, Integer> expectedMap2 = new HashMap<>();
-    for (int i = 0; i < 50; i ++) {
-      expectedMap2.put("k_" + i, i);
-    }
-
-    matchMapAndPairList(expectedMap2, rdd5.collect());
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaPairRDD.joinGeodeRegion
-  // --------------------------------------------------------------------------------------------
-
-  @Test
-  public void testPairRDDJoinWithSameKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(-5, 10);
-
-    JavaPairRDD<Tuple2<String, Integer>, Integer> rdd2a = javaFunctions(rdd1).joinGeodeRegion(regionPath);
-    JavaPairRDD<Tuple2<String, Integer>, Integer> rdd2b = javaFunctions(rdd1).joinGeodeRegion(regionPath, connConf);
-    // System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Tuple2<String, Integer>, Integer> expectedMap = new HashMap<>();
-    for (int i = 0; i < 10; i ++) {
-      expectedMap.put(new Tuple2<>("k_" + i, i), i);
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-  static class IntIntPairToStrKeyFunction implements Function<Tuple2<Integer, Integer>, String> {
-    @Override public String call(Tuple2<Integer, Integer> pair) throws Exception {
-      return "k_" + pair._1();
-    }
-  }
-
-  @Test
-  public void testPairRDDJoinWithDiffKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaPairRDD<Integer, Integer> rdd1 = prepareIntIntJavaPairRDD(-5, 10);
-    Function<Tuple2<Integer, Integer>, String> func = new IntIntPairToStrKeyFunction();
-
-    JavaPairRDD<Tuple2<Integer, Integer>, Integer> rdd2a = javaFunctions(rdd1).joinGeodeRegion(regionPath, func);
-    JavaPairRDD<Tuple2<Integer, Integer>, Integer> rdd2b = javaFunctions(rdd1).joinGeodeRegion(regionPath, func, connConf);
-    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Tuple2<Integer, Integer>, Integer> expectedMap = new HashMap<>();
-    for (int i = 0; i < 10; i ++) {
-      expectedMap.put(new Tuple2<>(i, i * 2), i);
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaPairRDD.outerJoinGeodeRegion
-  // --------------------------------------------------------------------------------------------
-
-  @Test
-  public void testPairRDDOuterJoinWithSameKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaPairRDD<String, Integer> rdd1 = prepareStrIntJavaPairRDD(-5, 10);
-
-    JavaPairRDD<Tuple2<String, Integer>, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath);
-    JavaPairRDD<Tuple2<String, Integer>, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath, connConf);
-    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Tuple2<String, Integer>, Option<Integer>> expectedMap = new HashMap<>();
-    for (int i = -5; i < 10; i ++) {
-      if (i < 0)
-        expectedMap.put(new Tuple2<>("k_" + i, i), Option.apply((Integer) null));
-      else
-        expectedMap.put(new Tuple2<>("k_" + i, i), Some.apply(i));
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-  @Test
-  public void testPairRDDOuterJoinWithDiffKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaPairRDD<Integer, Integer> rdd1 = prepareIntIntJavaPairRDD(-5, 10);
-    Function<Tuple2<Integer, Integer>, String> func = new IntIntPairToStrKeyFunction();
-
-    JavaPairRDD<Tuple2<Integer, Integer>, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath, func);
-    JavaPairRDD<Tuple2<Integer, Integer>, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath, func, connConf);
-    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Tuple2<Integer, Integer>, Option<Integer>> expectedMap = new HashMap<>();
-    for (int i = -5; i < 10; i ++) {
-      if (i < 0)
-        expectedMap.put(new Tuple2<>(i, i * 2), Option.apply((Integer) null));
-      else
-        expectedMap.put(new Tuple2<>(i, i * 2), Some.apply(i));
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaRDD.joinGeodeRegion
-  // --------------------------------------------------------------------------------------------
-
-  static class IntToStrKeyFunction implements Function<Integer, String> {
-    @Override public String call(Integer x) throws Exception {
-      return "k_" + x;
-    }
-  }
-
-  @Test
-  public void testRDDJoinWithSameKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(-5, 10);
-
-    Function<Integer, String> func = new IntToStrKeyFunction();
-    JavaPairRDD<Integer, Integer> rdd2a = javaFunctions(rdd1).joinGeodeRegion(regionPath, func);
-    JavaPairRDD<Integer, Integer> rdd2b = javaFunctions(rdd1).joinGeodeRegion(regionPath, func, connConf);
-    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Integer, Integer> expectedMap = new HashMap<>();
-    for (int i = 0; i < 10; i ++) {
-      expectedMap.put(i, i);
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //   JavaRDD.outerJoinGeodeRegion
-  // --------------------------------------------------------------------------------------------
-
-  @Test
-  public void testRDDOuterJoinWithSameKeyType() throws Exception {
-    prepareStrIntRegion(regionPath, 0, numObjects);
-    JavaRDD<Integer> rdd1 = prepareIntJavaRDD(-5, 10);
-
-    Function<Integer, String> func = new IntToStrKeyFunction();
-    JavaPairRDD<Integer, Option<Integer>> rdd2a = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath, func);
-    JavaPairRDD<Integer, Option<Integer>> rdd2b = javaFunctions(rdd1).outerJoinGeodeRegion(regionPath, func, connConf);
-    //System.out.println("=== Result RDD =======\n" + rdd2a.collect() + "\n=========================");
-
-    HashMap<Integer, Option<Integer>> expectedMap = new HashMap<>();
-    for (int i = -5; i < 10; i ++) {
-      if (i < 0)
-        expectedMap.put(i, Option.apply((Integer) null));
-      else
-        expectedMap.put(i, Some.apply(i));
-    }
-    matchMapAndPairList(expectedMap, rdd2a.collect());
-    matchMapAndPairList(expectedMap, rdd2b.collect());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Portfolio.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Portfolio.java b/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Portfolio.java
deleted file mode 100644
index 1457db9..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Portfolio.java
+++ /dev/null
@@ -1,109 +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 ittest.io.pivotal.geode.spark.connector;
-
-import java.io.Serializable;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Properties;
-import com.gemstone.gemfire.cache.Declarable;
-
-/**
- * A stock portfolio that consists of multiple {@link Position} objects that
- * represent shares of stock (a "security").  Instances of
- * <code>Portfolio</code> can be stored in a Geode <code>Region</code> and
- * their contents can be queried using the Geode query service.
- * </p>
- * This class is <code>Serializable</code> because we want it to be distributed
- * to multiple members of a distributed system.  Because this class is
- * <code>Declarable</code>, we can describe instances of it in a Geode
- * <code>cache.xml</code> file.
- * </p>
- *
- */
-public class Portfolio implements Declarable, Serializable {
-
-  private static final long serialVersionUID = 9097335119586059309L;
-
-  private int id;  /* id is used as the entry key and is stored in the entry */
-  private String type;
-  private Map<String,Position> positions = new LinkedHashMap<String,Position>();
-  private String status;
-
-  public Portfolio(Properties props) {
-    init(props);
-  }
-
-  @Override
-  public void init(Properties props) {
-    this.id = Integer.parseInt(props.getProperty("id"));
-    this.type = props.getProperty("type", "type1");
-    this.status = props.getProperty("status", "active");
-
-    // get the positions. These are stored in the properties object
-    // as Positions, not String, so use Hashtable protocol to get at them.
-    // the keys are named "positionN", where N is an integer.
-    for (Map.Entry<Object, Object> entry: props.entrySet()) {
-      String key = (String)entry.getKey();
-      if (key.startsWith("position")) {
-        Position pos = (Position)entry.getValue();
-        this.positions.put(pos.getSecId(), pos);
-      }
-    }
-  }
-
-  public void setType(String t) {this.type = t; }
-
-  public String getStatus(){
-    return status;
-  }
-
-  public int getId(){
-    return this.id;
-  }
-
-  public Map<String,Position> getPositions(){
-    return this.positions;
-  }
-
-  public String getType() {
-    return this.type;
-  }
-
-  public boolean isActive(){
-    return status.equals("active");
-  }
-
-  @Override
-  public String toString(){
-    StringBuilder buf = new StringBuilder();
-    buf.append("\n\tPortfolio [id=" + this.id + " status=" + this.status);
-    buf.append(" type=" + this.type);
-    boolean firstTime = true;
-    for (Map.Entry<String, Position> entry: positions.entrySet()) {
-      if (!firstTime) {
-        buf.append(", ");
-      }
-      buf.append("\n\t\t");
-      buf.append(entry.getKey() + ":" + entry.getValue());
-      firstTime = false;
-    }
-    buf.append("]");
-    return buf.toString();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Position.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Position.java b/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Position.java
deleted file mode 100644
index d6f8d1f..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/java/ittest/io/pivotal/geode/spark/connector/Position.java
+++ /dev/null
@@ -1,73 +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 ittest.io.pivotal.geode.spark.connector;
-
-import java.io.Serializable;
-import java.util.Properties;
-import com.gemstone.gemfire.cache.Declarable;
-
-/**
- * Represents a number of shares of a stock ("security") held in a {@link
- * Portfolio}.
- * </p>
- * This class is <code>Serializable</code> because we want it to be distributed
- * to multiple members of a distributed system.  Because this class is
- * <code>Declarable</code>, we can describe instances of it in a Geode
- * <code>cache.xml</code> file.
- * </p>
- *
- */
-public class Position implements Declarable, Serializable {
-
-  private static final long serialVersionUID = -8229531542107983344L;
-
-  private String secId;
-  private double qty;
-  private double mktValue;
-
-  public Position(Properties props) {
-    init(props);
-  }
-
-  @Override
-  public void init(Properties props) {
-    this.secId = props.getProperty("secId");
-    this.qty = Double.parseDouble(props.getProperty("qty"));
-    this.mktValue = Double.parseDouble(props.getProperty("mktValue"));
-  }
-
-  public String getSecId(){
-    return this.secId;
-  }
-
-  public double getQty(){
-    return this.qty;
-  }
-
-  public double getMktValue() {
-    return this.mktValue;
-  }
-
-  @Override
-  public String toString(){
-    return new StringBuilder()
-            .append("Position [secId=").append(secId)
-            .append(" qty=").append(this.qty)
-            .append(" mktValue=").append(mktValue).append("]").toString();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/resources/test-regions.xml
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/resources/test-regions.xml b/geode-spark-connector/geode-spark-connector/src/it/resources/test-regions.xml
deleted file mode 100644
index 79893d6..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/resources/test-regions.xml
+++ /dev/null
@@ -1,49 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
-<!DOCTYPE cache PUBLIC
-  "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
-  "http://www.gemstone.com/dtd/cache6_5.dtd" >
-
-<cache>
-  <!-- test region for OQL test -->
-  <region name="obj_obj_region" refid="PARTITION_REDUNDANT" />
-
-  <region name="obj_obj_rep_region" refid="REPLICATE" />
-
-  <region name="str_int_region" refid="PARTITION_REDUNDANT">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.Integer</value-constraint>
-    </region-attributes>
-  </region>
-
-  <region name="str_str_region" refid="PARTITION_REDUNDANT">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.String</value-constraint>
-    </region-attributes>
-  </region>
-
-  <region name="str_str_rep_region" refid="REPLICATE">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.String</value-constraint>
-    </region-attributes>
-  </region>
-</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/resources/test-retrieve-regions.xml
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/resources/test-retrieve-regions.xml b/geode-spark-connector/geode-spark-connector/src/it/resources/test-retrieve-regions.xml
deleted file mode 100644
index 3023959..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/resources/test-retrieve-regions.xml
+++ /dev/null
@@ -1,57 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  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.
--->
-
-<!DOCTYPE cache PUBLIC
-  "-//GemStone Systems, Inc.//GemFire Declarative Caching 6.5//EN"
-  "http://www.gemstone.com/dtd/cache6_5.dtd" >
-
-<cache>
-  <!-- combinations of key, value types with region types -->
-  <region name="pr_r_obj_obj_region" refid="PARTITION_REDUNDANT" />
-  <region name="pr_obj_obj_region" refid="PARTITION" />
-  <region name="rr_obj_obj_region" refid="REPLICATE" />
-  <region name="rr_p_obj_obj_region" refid="REPLICATE_PERSISTENT" />
-
-  <region name="pr_r_str_int_region" refid="PARTITION_REDUNDANT">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.Integer</value-constraint>
-    </region-attributes>
-  </region>
-  
-  <region name="pr_str_int_region" refid="PARTITION">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.Integer</value-constraint>
-    </region-attributes>
-  </region>
-
-  <region name="rr_str_int_region" refid="REPLICATE">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.Integer</value-constraint>
-    </region-attributes>
-  </region>
-  
-  <region name="rr_p_str_int_region" refid="REPLICATE_PERSISTENT">
-    <region-attributes>
-      <key-constraint>java.lang.String</key-constraint>
-      <value-constraint>java.lang.Integer</value-constraint>
-    </region-attributes>
-  </region>
-</cache>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/BasicIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/BasicIntegrationTest.scala b/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/BasicIntegrationTest.scala
deleted file mode 100644
index a26bcbd..0000000
--- a/geode-spark-connector/geode-spark-connector/src/it/scala/ittest/io/pivotal/geode/spark/connector/BasicIntegrationTest.scala
+++ /dev/null
@@ -1,598 +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 ittest.io.pivotal.geode.spark.connector
-
-import java.util.Properties
-import com.gemstone.gemfire.cache.query.QueryService
-import com.gemstone.gemfire.cache.query.internal.StructImpl
-import io.pivotal.geode.spark.connector._
-import com.gemstone.gemfire.cache.Region
-import io.pivotal.geode.spark.connector.internal.{RegionMetadata, DefaultGeodeConnectionManager}
-import io.pivotal.geode.spark.connector.internal.oql.{RDDConverter, QueryRDD}
-import ittest.io.pivotal.geode.spark.connector.testkit.GeodeCluster
-import ittest.io.pivotal.geode.spark.connector.testkit.IOUtils
-import org.apache.spark.streaming.{Seconds, StreamingContext, TestInputDStream}
-import org.apache.spark.{SparkContext, SparkConf}
-import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers}
-import scala.collection.JavaConversions
-import scala.reflect.ClassTag
-
-case class Number(str: String, len: Int)
-
-class BasicIntegrationTest extends FunSuite with Matchers with BeforeAndAfterAll with GeodeCluster {
-
-  var sc: SparkContext = null
-
-  override def beforeAll() {
-    // start geode cluster, and spark context
-    val settings = new Properties()
-    settings.setProperty("cache-xml-file", "src/it/resources/test-regions.xml")
-    settings.setProperty("num-of-servers", "2")
-    val locatorPort = GeodeCluster.start(settings)
-
-    // start spark context in local mode
-    IOUtils.configTestLog4j("ERROR", "log4j.logger.org.apache.spark" -> "INFO",
-                            "log4j.logger.io.pivotal.geode.spark.connector" -> "DEBUG")
-    val conf = new SparkConf()
-      .setAppName("BasicIntegrationTest")
-      .setMaster("local[2]")
-      .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
-      .set(GeodeLocatorPropKey, s"localhost[$locatorPort]")
-      .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-      .set("spark.kryo.registrator", "io.pivotal.geode.spark.connector.GeodeKryoRegistrator")
-
-    sc = new SparkContext(conf)
-  }
-
-  override def afterAll() {
-    // stop connection, spark context, and geode cluster
-    DefaultGeodeConnectionManager.closeConnection(GeodeConnectionConf(sc.getConf))
-    sc.stop()
-    GeodeCluster.stop()
-  }
-
-  //Convert Map[Object, Object] to java.util.Properties
-  private def map2Props(map: Map[Object, Object]): java.util.Properties =
-    (new java.util.Properties /: map) {case (props, (k,v)) => props.put(k,v); props}
-
-  // ===========================================================
-  //       DefaultGeodeConnection functional tests
-  // ===========================================================
-
-  test("DefaultGeodeConnection.validateRegion()") {
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-
-    // normal exist-region
-    var regionPath: String = "str_str_region"
-    conn.validateRegion[String, String](regionPath)
-
-    // non-exist region
-    regionPath = "non_exist_region"
-    try {
-      conn.validateRegion[String, String](regionPath)
-      fail("validateRegion failed to catch non-exist region error")
-    } catch {
-      case e: RuntimeException => 
-        if (! e.getMessage.contains(s"The region named $regionPath was not found"))
-          fail("validateRegion gives wrong exception on non-exist region", e)
-      case e: Throwable => 
-        fail("validateRegion gives wrong exception on non-exist region", e)
-    }
-
-    // Note: currently, can't catch type mismatch error
-    conn.validateRegion[String, Integer]("str_str_region")
-  }
-
-  test("DefaultGeodeConnection.getRegionMetadata()") {
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-
-    // exist region
-    validateRegionMetadata(conn, "obj_obj_region", true, 113, null, null, false)
-    validateRegionMetadata(conn, "str_int_region", true, 113, "java.lang.String", "java.lang.Integer", false)
-    validateRegionMetadata(conn, "str_str_rep_region", false, 0, "java.lang.String", "java.lang.String", true)
-
-    // non-exist region
-    assert(! conn.getRegionMetadata("no_exist_region").isDefined)
-  }
-    
-  def validateRegionMetadata(
-    conn: GeodeConnection, regionPath: String, partitioned: Boolean, buckets: Int,
-    keyType: String, valueType: String, emptyMap: Boolean): Unit = {
-
-    val mdOption = conn.getRegionMetadata(regionPath)
-    val md = mdOption.get
-   
-    assert(md.getRegionPath == s"/$regionPath")
-    assert(md.isPartitioned == partitioned)
-    assert(md.getKeyTypeName == keyType)
-    assert(md.getValueTypeName == valueType)
-    assert(md.getTotalBuckets == buckets)
-    if (emptyMap) assert(md.getServerBucketMap == null) 
-    else assert(md.getServerBucketMap != null)
-  }
-
-  test("DefaultGeodeConnection.getRegionProxy()") {
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-
-    val region1 = conn.getRegionProxy[String, String]("str_str_region")
-    region1.put("1", "One")
-    assert(region1.get("1") == "One")
-    region1.remove("1")
-    assert(region1.get("1") == null)
-    
-    // getRegionProxy doesn't fail when region doesn't exist
-    val region2 = conn.getRegionProxy[String, String]("non_exist_region")
-    try {
-      region2.put("1", "One")
-      fail("getRegionProxy failed to catch non-exist region error")
-    } catch {
-      case e: Exception =>
-        if (e.getCause == null || ! e.getCause.getMessage.contains(s"Region named /non_exist_region was not found")) {
-          e.printStackTrace()
-          fail("validateRegion gives wrong exception on non-exist region", e)
-        }
-    }
-  }
-  
-  // Note: DefaultGeodeConnecton.getQuery() and getRegionData() are covered by
-  //       RetrieveRegionIntegrationTest.scala and following OQL tests.
-  
-  // ===========================================================
-  //                OQL functional tests
-  // ===========================================================
-  
-  private def initRegion(regionName: String): Unit = {
-
-    //Populate some data in the region
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-
-    //This will call the implicit conversion map2Properties in connector package object, since it is Map[String, String]
-    var position1 = new Position(Map("secId" -> "SUN", "qty" -> "34000", "mktValue" -> "24.42"))
-    var position2 = new Position(Map("secId" -> "IBM", "qty" -> "8765", "mktValue" -> "34.29"))
-    val portfolio1 = new Portfolio(map2Props(Map("id" ->"1", "type" -> "type1", "status" -> "active",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("1", portfolio1)
-
-    position1 = new Position(Map("secId" -> "YHOO", "qty" -> "9834", "mktValue" -> "12.925"))
-    position2 = new Position(Map("secId" -> "GOOG", "qty" -> "12176", "mktValue" -> "21.972"))
-    val portfolio2 = new Portfolio(map2Props(Map("id" -> "2", "type" -> "type2", "status" -> "inactive",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("2", portfolio2)
-
-    position1 = new Position(Map("secId" -> "MSFT", "qty" -> "98327", "mktValue" -> "23.32"))
-    position2 = new Position(Map("secId" -> "AOL", "qty" -> "978", "mktValue" -> "40.373"))
-    val portfolio3 = new Portfolio(map2Props(Map("id" -> "3", "type" -> "type3", "status" -> "active",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("3", portfolio3)
-
-    position1 = new Position(Map("secId" -> "APPL", "qty" -> "67", "mktValue" -> "67.356572"))
-    position2 = new Position(Map("secId" -> "ORCL", "qty" -> "376", "mktValue" -> "101.34"))
-    val portfolio4 = new Portfolio(map2Props(Map("id" -> "4", "type" -> "type1", "status" -> "inactive",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("4", portfolio4)
-
-    position1 = new Position(Map("secId" -> "SAP", "qty" -> "90", "mktValue" -> "67.356572"))
-    position2 = new Position(Map("secId" -> "DELL", "qty" -> "376", "mktValue" -> "101.34"))
-    val portfolio5 = new Portfolio(map2Props(Map("id" -> "5", "type" -> "type2", "status" -> "active",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("5", portfolio5)
-
-    position1 = new Position(Map("secId" -> "RHAT", "qty" -> "90", "mktValue" -> "67.356572"))
-    position2 = new Position(Map("secId" -> "NOVL", "qty" -> "376", "mktValue" -> "101.34"))
-    val portfolio6 = new Portfolio(map2Props(Map("id" -> "6", "type" -> "type3", "status" -> "inactive",
-      "position1" -> position1, "position2" -> position2)))
-    rgn.put("6", portfolio6)
-
-    position1 = new Position(Map("secId" -> "MSFT", "qty" -> "98327", "mktValue" -> "23.32"))
-    position2 = new Position(Map("secId" -> "AOL", "qty" -> "978", "mktValue" -> "40.373"))
-    val portfolio7 = new Portfolio(map2Props(Map("id" -> "7", "type" -> "type4", "status" -> "active",
-      "position1" -> position1, "position2" -> position2)))
-    //Not using null, due to intermittent query failure on column containing null, likely a Spark SQL bug
-    //portfolio7.setType(null)
-    rgn.put("7", portfolio7)
-  }
-
-  private def getQueryRDD[T: ClassTag](
-    query: String, connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)): QueryRDD[T] =
-      new QueryRDD[T](sc, query, connConf)
-
-  test("Run Geode OQL query and convert the returned QueryRDD to DataFrame: Partitioned Region") {
-    simpleQuery("obj_obj_region")
-  }
-
-  test("Run Geode OQL query and convert the returned QueryRDD to DataFrame: Replicated Region") {
-    simpleQuery("obj_obj_rep_region")
-  }
-
-  private def simpleQuery(regionName: String) {
-    //Populate some data in the region
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val rgn: Region[String, String] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> "one", "2" -> "two", "3" -> "three")))
-
-    //Create QueryRDD using OQL
-    val OQLResult: QueryRDD[String] = getQueryRDD[String](s"select * from /$regionName")
-
-    //verify the QueryRDD
-    val oqlRS: Array[String] = OQLResult.collect()
-    oqlRS should have length 3
-    oqlRS should contain theSameElementsAs List("one", "two", "three")
-
-    //Convert QueryRDD to DataFrame
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    // this is used to implicitly convert an RDD to a DataFrame.
-    import sqlContext.implicits._
-    val dataFrame = OQLResult.map(x => Number(x, x.length)).toDF()
-    //Register dataFrame as a table of two columns of type String and Int respectively
-    dataFrame.registerTempTable("numberTable")
-
-    //Issue SQL query against the table
-    val SQLResult = sqlContext.sql("SELECT * FROM numberTable")
-    //Verify the SQL query result, r(0) mean column 0
-    val sqlRS: Array[Any] = SQLResult.map(r => r(0)).collect()
-    sqlRS should have length 3
-    sqlRS should contain theSameElementsAs List("one", "two", "three")
-
-    //Convert QueryRDD to DataFrame using RDDConverter
-    val dataFrame2 = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
-    //Register dataFrame2 as a table of two columns of type String and Int respectively
-    dataFrame2.registerTempTable("numberTable2")
-
-    //Issue SQL query against the table
-    val SQLResult2 = sqlContext.sql("SELECT * FROM numberTable2")
-    //Verify the SQL query result, r(0) mean column 0
-    val sqlRS2: Array[Any] = SQLResult2.map(r => r(0)).collect()
-    sqlRS2 should have length 3
-    sqlRS2 should contain theSameElementsAs List("one", "two", "three")
-
-    //Remove the region entries, because other tests might use the same region as well
-    List("1", "2", "3").foreach(rgn.remove)
-  }
-
-  test("Run Geode OQL query and directly return DataFrame: Partitioned Region") {
-    simpleQueryDataFrame("obj_obj_region")
-  }
-
-  test("Run Geode OQL query and directly return DataFrame: Replicated Region") {
-    simpleQueryDataFrame("obj_obj_rep_region")
-  }
-
-  private def simpleQueryDataFrame(regionName: String) {
-    //Populate some data in the region
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-    val rgn: Region[String, String] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> "one", "2" -> "two", "3" -> "three")))
-
-    //Create DataFrame using Geode OQL
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"select * from /$regionName")
-    dataFrame.registerTempTable("numberTable")
-
-    //Issue SQL query against the table
-    val SQLResult = sqlContext.sql("SELECT * FROM numberTable")
-    //Verify the SQL query result, r(0) mean column 0
-    val sqlRS: Array[Any] = SQLResult.map(r => r(0)).collect()
-    sqlRS should have length 3
-    sqlRS should contain theSameElementsAs List("one", "two", "three")
-
-    //Remove the region entries, because other tests might use the same region as well
-    List("1", "2", "3").foreach(rgn.remove)
-  }
-
-  test("Geode OQL query with UDT: Partitioned Region") {
-    queryUDT("obj_obj_region")
-  }
-
-  test("Geode OQL query with UDT: Replicated Region") {
-    queryUDT("obj_obj_rep_region")
-  }
-
-  private def queryUDT(regionName: String) {
-
-    //Populate some data in the region
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-    val e1: Employee = new Employee("hello", 123)
-    val e2: Employee = new Employee("world", 456)
-    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> e1, "2" -> e2)))
-
-    //Create QueryRDD using OQL
-    val OQLResult: QueryRDD[Object] = getQueryRDD(s"select name, age from /$regionName")
-
-    //verify the QueryRDD
-    val oqlRS: Array[Object] = OQLResult.collect()
-    oqlRS should have length 2
-    oqlRS.map(e => e.asInstanceOf[StructImpl].getFieldValues.apply(1)) should contain theSameElementsAs List(123, 456)
-
-    //Convert QueryRDD to DataFrame
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-
-    //Convert QueryRDD to DataFrame using RDDConverter
-    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
-    dataFrame.registerTempTable("employee")
-    val SQLResult = sqlContext.sql("SELECT * FROM employee")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.map(r => r(0)).collect()
-    sqlRS should have length 2
-    sqlRS should contain theSameElementsAs List("hello", "world")
-
-    List("1", "2").foreach(rgn.remove)
-  }
-
-  test("Geode OQL query with UDT and directly return DataFrame: Partitioned Region") {
-    queryUDTDataFrame("obj_obj_region")
-  }
-
-  test("Geode OQL query with UDT and directly return DataFrame: Replicated Region") {
-    queryUDTDataFrame("obj_obj_rep_region")
-  }
-
-  private def queryUDTDataFrame(regionName: String) {
-    //Populate some data in the region
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-    val e1: Employee = new Employee("hello", 123)
-    val e2: Employee = new Employee("world", 456)
-    rgn.putAll(JavaConversions.mapAsJavaMap(Map("1" -> e1, "2" -> e2)))
-
-    //Create DataFrame using Geode OQL
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"select name, age from /$regionName")
-
-    dataFrame.registerTempTable("employee")
-    val SQLResult = sqlContext.sql("SELECT * FROM employee")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.map(r => r(0)).collect()
-    sqlRS should have length 2
-    sqlRS should contain theSameElementsAs List("hello", "world")
-
-    List("1", "2").foreach(rgn.remove)
-  }
-
-  test("Geode OQL query with more complex UDT: Partitioned Region") {
-    complexUDT("obj_obj_region")
-  }
-
-  test("Geode OQL query with more complex UDT: Replicated Region") {
-    complexUDT("obj_obj_rep_region")
-  }
-
-  private def complexUDT(regionName: String) {
-
-    initRegion(regionName)
-
-    //Create QueryRDD using OQL
-    val OQLResult: QueryRDD[Object] = getQueryRDD(s"SELECT DISTINCT * FROM /$regionName WHERE status = 'active'")
-
-    //verify the QueryRDD
-    val oqlRS: Array[Int] = OQLResult.collect().map(r => r.asInstanceOf[Portfolio].getId)
-    oqlRS should contain theSameElementsAs List(1, 3, 5, 7)
-
-    //Convert QueryRDD to DataFrame
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-
-    //Convert QueryRDD to DataFrame using RDDConverter
-    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
-
-    dataFrame.registerTempTable("Portfolio")
-
-    val SQLResult = sqlContext.sql("SELECT * FROM Portfolio")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.collect().map(r => r(0).asInstanceOf[Portfolio].getType)
-    sqlRS should contain theSameElementsAs List("type1", "type2", "type3", "type4")
-  }
-
-  test("Geode OQL query with more complex UDT and directly return DataFrame: Partitioned Region") {
-    complexUDTDataFrame("obj_obj_region")
-  }
-
-  test("Geode OQL query with more complex UDT and directly return DataFrame: Replicated Region") {
-    complexUDTDataFrame("obj_obj_rep_region")
-  }
-
-  private def complexUDTDataFrame(regionName: String) {
-
-    initRegion(regionName)
-
-    //Create DataFrame using Geode OQL
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"SELECT DISTINCT * FROM /$regionName WHERE status = 'active'")
-    dataFrame.registerTempTable("Portfolio")
-
-    val SQLResult = sqlContext.sql("SELECT * FROM Portfolio")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.collect().map(r => r(0).asInstanceOf[Portfolio].getType)
-    sqlRS should contain theSameElementsAs List("type1", "type2", "type3", "type4")
-  }
-
-  test("Geode OQL query with more complex UDT with Projection: Partitioned Region") {
-    queryComplexUDTProjection("obj_obj_region")
-  }
-
-  test("Geode OQL query with more complex UDT with Projection: Replicated Region") {
-    queryComplexUDTProjection("obj_obj_rep_region")
-  }
-
-  private def queryComplexUDTProjection(regionName: String) {
-
-    initRegion(regionName)
-
-    //Create QueryRDD using OQL
-    val OQLResult: QueryRDD[Object] = getQueryRDD[Object](s"""SELECT id, "type", positions, status FROM /$regionName WHERE status = 'active'""")
-
-    //verify the QueryRDD
-    val oqlRS: Array[Int] = OQLResult.collect().map(si => si.asInstanceOf[StructImpl].getFieldValues.apply(0).asInstanceOf[Int])
-    oqlRS should contain theSameElementsAs List(1, 3, 5, 7)
-
-    //Convert QueryRDD to DataFrame
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-
-    //Convert QueryRDD to DataFrame using RDDConverter
-    val dataFrame = RDDConverter.queryRDDToDataFrame(OQLResult, sqlContext)
-
-    dataFrame.registerTempTable("Portfolio")
-
-    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.collect().map(r => r(0))
-    sqlRS should contain theSameElementsAs List(3)
-  }
-
-  test("Geode OQL query with more complex UDT with Projection and directly return DataFrame: Partitioned Region") {
-    queryComplexUDTProjectionDataFrame("obj_obj_region")
-  }
-
-  test("Geode OQL query with more complex UDT with Projection and directly return DataFrame: Replicated Region") {
-    queryComplexUDTProjectionDataFrame("obj_obj_rep_region")
-  }
-
-  private def queryComplexUDTProjectionDataFrame(regionName: String) {
-
-    initRegion(regionName)
-
-    //Create DataFrame using Geode OQL
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"""SELECT id, "type", positions, status FROM /$regionName WHERE status = 'active'""")
-    dataFrame.registerTempTable("Portfolio")
-
-    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.collect().map(r => r(0))
-    sqlRS should contain theSameElementsAs List(3)
-  }
-
-  test("Geode OQL query with more complex UDT with nested Projection and directly return DataFrame: Partitioned Region") {
-    queryComplexUDTNestProjectionDataFrame("obj_obj_region")
-  }
-
-  test("Geode OQL query with more complex UDT with nested Projection and directly return DataFrame: Replicated Region") {
-    queryComplexUDTNestProjectionDataFrame("obj_obj_rep_region")
-  }
-
-  private def queryComplexUDTNestProjectionDataFrame(regionName: String) {
-
-    initRegion(regionName)
-
-    //Create DataFrame using Geode OQL
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"""SELECT r.id, r."type", r.positions, r.status FROM /$regionName r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'""")
-    dataFrame.registerTempTable("Portfolio")
-
-    val SQLResult = sqlContext.sql("SELECT id, type FROM Portfolio where type = 'type3'")
-
-    //Verify the SQL query result
-    val sqlRS = SQLResult.collect().map(r => r(0))
-    sqlRS should contain theSameElementsAs List(3)
-  }
-
-  test("Undefined instance deserialization: Partitioned Region") {
-    undefinedInstanceDeserialization("obj_obj_region")
-  }
-
-  test("Undefined instance deserialization: Replicated Region") {
-    undefinedInstanceDeserialization("obj_obj_rep_region")
-  }
-
-  private def undefinedInstanceDeserialization(regionName: String) {
-
-    val conn = GeodeConnectionConf(sc.getConf).getConnection
-    val rgn: Region[Object, Object] = conn.getRegionProxy(regionName)
-    rgn.removeAll(rgn.keySetOnServer())
-
-    //Put some new data
-    rgn.put("1", "one")
-
-    //Query some non-existent columns, which should return UNDEFINED
-    val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-    val dataFrame = sqlContext.geodeOQL(s"SELECT col100, col200 FROM /$regionName")
-    val col1 = dataFrame.first().apply(0)
-    val col2 = dataFrame.first().apply(1)
-    assert(col1 == QueryService.UNDEFINED)
-    assert(col2 == QueryService.UNDEFINED)
-    //Verify that col1 and col2 refer to the same Undefined object
-    assert(col1.asInstanceOf[AnyRef] eq col2.asInstanceOf[AnyRef])
-  }
-
-  test("RDD.saveToGeode") {
-    val regionName = "str_str_region"
-    // generate: Vector((1,11), (2,22), (3,33), (4,44), (5,55), (6,66))
-    val data = (1 to 6).map(_.toString).map(e=> (e, e*2))
-    val rdd = sc.parallelize(data)
-    rdd.saveToGeode(regionName)
-
-    // verify
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val region: Region[String, String] = connConf.getConnection.getRegionProxy(regionName)
-    println("region key set on server: " + region.keySetOnServer())
-    assert((1 to 6).map(_.toString).toSet == JavaConversions.asScalaSet(region.keySetOnServer()))
-    (1 to 6).map(_.toString).foreach(e => assert(e*2 == region.get(e)))
-  }
-
-  // ===========================================================
-  //        DStream.saveToGeode() functional tests
-  // ===========================================================
-
-  test("Basic DStream test") {
-    import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListener}
-    import io.pivotal.geode.spark.connector.streaming._
-    import org.apache.spark.streaming.ManualClockHelper
-
-    class TestStreamListener extends StreamingListener {
-      var count = 0
-      override def onBatchCompleted(batch: StreamingListenerBatchCompleted) = count += 1
-    }
-
-    def batchDuration = Seconds(1)
-    val ssc = new StreamingContext(sc, batchDuration)
-    val input = Seq(1 to 4, 5 to 8, 9 to 12)
-    val dstream = new TestInputDStream(ssc, input, 2)
-    dstream.saveToGeode[String, Int]("str_int_region", (e: Int) => (e.toString, e))
-    try {
-      val listener = new TestStreamListener
-      ssc.addStreamingListener(listener)
-      ssc.start()
-      ManualClockHelper.addToTime(ssc, batchDuration.milliseconds * input.length)
-      while (listener.count < input.length) ssc.awaitTerminationOrTimeout(50)
-    } catch {
-      case e: Exception => e.printStackTrace(); throw e
-//    } finally {
-//      ssc.stop()
-    }
-
-    val connConf: GeodeConnectionConf = GeodeConnectionConf(sc.getConf)
-    val conn = connConf.getConnection
-    val region: Region[String, Int] = conn.getRegionProxy("str_int_region")
-
-    // verify geode region contents
-    println("region key set on server: " + region.keySetOnServer())
-    assert((1 to 12).map(_.toString).toSet == JavaConversions.asScalaSet(region.keySetOnServer()))
-    (1 to 12).foreach(e => assert(e == region.get(e.toString)))
-  }
-}



[08/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireRunner.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireRunner.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireRunner.scala
new file mode 100644
index 0000000..b4ee572
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/GemFireRunner.scala
@@ -0,0 +1,148 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector.testkit
+
+import java.io.{IOException, File}
+import java.net.InetAddress
+import java.util.Properties
+import org.apache.commons.httpclient.HttpClient
+import org.apache.commons.io.FileUtils
+import org.apache.commons.io.filefilter.IOFileFilter
+
+/**
+* A class that manages GemFire locator and servers.  Uses gfsh to
+* start and stop the locator and servers.
+*/
+class GemFireRunner(settings: Properties) {
+  val gfshCmd = new File(getCurrentDirectory, "../../geode-assembly/build/install/apache-geode/bin/gfsh").toString
+  val cacheXMLFile = settings.get("cache-xml-file")
+  val numServers: Int = settings.get("num-of-servers").asInstanceOf[String].toInt
+  val cwd = new File(".").getAbsolutePath
+  val gemfireFunctionsTargetDir = new File("../gemfire-functions/target")
+  val testroot = "target/testgemfire"
+  val classpath = new File(cwd, "target/scala-2.10/it-classes/")
+  val locatorPort = startGemFireCluster(numServers)
+
+  def getLocatorPort: Int = locatorPort
+
+  private def getCurrentDirectory = new File( "." ).getCanonicalPath
+  
+  private def startGemFireCluster(numServers: Int): Int = {
+    //ports(0) for GemFire locator, the other ports are for GemFire servers
+    val ports: Seq[Int] = IOUtils.getRandomAvailableTCPPorts(2 + numServers)
+    startGemFireLocator(ports(0), ports(1))
+    startGemFireServers(ports(0), ports.drop(2))
+    registerFunctions(ports(1))
+    ports(0)
+  }
+
+  private def startGemFireLocator(locatorPort: Int, jmxHttpPort:Int) {
+    println(s"=== GemFireRunner: starting locator on port $locatorPort")
+    val locatorDir = new File(cwd, s"$testroot/locator")
+    if (locatorDir.exists())
+      FileUtils.deleteDirectory(locatorDir)
+    IOUtils.mkdir(locatorDir)
+    new ProcessBuilder()
+      .command(gfshCmd, "start", "locator",
+        "--name=locator",
+        s"--dir=$locatorDir",
+        s"--port=$locatorPort",
+        s"--J=-Dgemfire.jmx-manager-http-port=$jmxHttpPort")
+      .inheritIO()
+      .start()
+
+    // Wait 30 seconds for locator to start
+    println(s"=== GemFireRunner: waiting for locator on port $locatorPort")
+    if (!IOUtils.waitForPortOpen(InetAddress.getByName("localhost"), locatorPort, 30000))
+      throw new IOException("Failed to start GemFire locator.")
+    println(s"=== GemFireRunner: done waiting for locator on port $locatorPort")
+  }
+
+  private def startGemFireServers(locatorPort: Int, serverPorts: Seq[Int]) {
+    val procs = for (i <- 0 until serverPorts.length) yield {
+      println(s"=== GemFireRunner: starting server${i+1} with clientPort ${serverPorts(i)}")
+      val serverDir = new File(cwd, s"$testroot/server${i+1}")
+      if (serverDir.exists())
+        FileUtils.deleteDirectory(serverDir)
+      IOUtils.mkdir(serverDir)
+      new ProcessBuilder()
+        .command(gfshCmd, "start", "server",
+          s"--name=server${i+1}",
+          s"--locators=localhost[$locatorPort]",
+          s"--bind-address=localhost",
+          s"--server-port=${serverPorts(i)}",
+          s"--dir=$serverDir",
+          s"--cache-xml-file=$cacheXMLFile",
+          s"--classpath=$classpath")
+        .inheritIO()
+        .start()
+    }
+    procs.foreach(p => p.waitFor)
+    println(s"All $serverPorts.length servers have been started") 
+  }
+  
+  private def registerFunctions(jmxHttpPort:Int) {
+    import scala.collection.JavaConversions._
+    FileUtils.listFiles(gemfireFunctionsTargetDir, fileFilter, dirFilter).foreach{  f => registerFunction(jmxHttpPort, f)}
+  }
+  
+  def fileFilter = new IOFileFilter {
+    def accept (file: File) = file.getName.endsWith(".jar") && file.getName.startsWith("gemfire-functions")
+    def accept (dir: File, name: String) = name.endsWith(".jar") && name.startsWith("gemfire-functions")
+  }
+  
+  def dirFilter = new IOFileFilter {
+    def accept (file: File) = file.getName.startsWith("scala")
+    def accept (dir: File, name: String) = name.startsWith("scala")
+  }
+  
+  private def registerFunction(jmxHttpPort:Int, jar:File) {
+    println("Deploying:" + jar.getName)
+    import io.pivotal.gemfire.spark.connector.GemFireFunctionDeployer
+    val deployer = new GemFireFunctionDeployer(new HttpClient())
+    deployer.deploy("localhost", jmxHttpPort, jar)
+  }
+
+  def stopGemFireCluster(): Unit = {
+    stopGemFireServers(numServers)
+    stopGemFireLocator()
+    if (!IOUtils.waitForPortClose(InetAddress.getByName("localhost"), getLocatorPort, 30000))
+      throw new IOException(s"Failed to stop GemFire locator at port $getLocatorPort.")
+    println(s"Successfully stop GemFire locator at port $getLocatorPort.")
+  }
+
+  private def stopGemFireLocator() {
+    println(s"=== GemFireRunner: stop locator")
+    val p = new ProcessBuilder()
+      .inheritIO()
+      .command(gfshCmd, "stop", "locator", s"--dir=$testroot/locator")
+      .start()
+     p.waitFor()
+  }
+
+  private def stopGemFireServers(numServers: Int) {
+   val procs = for (i <-1 to numServers) yield {
+       println(s"=== GemFireRunner: stop server $i.")
+       new ProcessBuilder()
+        .inheritIO()
+        .command(gfshCmd, "stop", "server", s"--dir=$testroot/server$i")
+        .start()
+   }
+   procs.foreach(p => p.waitFor())
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/IOUtils.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/IOUtils.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/IOUtils.scala
new file mode 100644
index 0000000..28134a8
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/ittest/io/pivotal/gemfire/spark/connector/testkit/IOUtils.scala
@@ -0,0 +1,94 @@
+/*
+ * 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 ittest.io.pivotal.gemfire.spark.connector.testkit
+
+import java.io.{File, IOException}
+import java.net.{InetAddress, Socket}
+import com.gemstone.gemfire.internal.AvailablePort
+import scala.util.Try
+import org.apache.log4j.PropertyConfigurator
+import java.util.Properties
+
+object IOUtils {
+
+  /** Makes a new directory or throws an `IOException` if it cannot be made */
+  def mkdir(dir: File): File = {
+    if (!dir.mkdirs())
+      throw new IOException(s"Could not create dir $dir")
+    dir
+  }
+
+  private def socketPortProb(host: InetAddress, port: Int) = Iterator.continually {
+    Try {
+      Thread.sleep(100)
+      new Socket(host, port).close()
+    }
+  }
+  
+  /**
+   * Waits until a port at the given address is open or timeout passes.
+   * @return true if managed to connect to the port, false if timeout happened first
+   */
+  def waitForPortOpen(host: InetAddress, port: Int, timeout: Long): Boolean = {
+    val startTime = System.currentTimeMillis()
+    socketPortProb(host, port)
+      .dropWhile(p => p.isFailure && System.currentTimeMillis() - startTime < timeout)
+      .next()
+      .isSuccess
+  }
+
+  /**
+   * Waits until a port at the given address is close or timeout passes.
+   * @return true if host:port is un-connect-able, false if timeout happened first
+   */
+  def waitForPortClose(host: InetAddress, port: Int, timeout: Long): Boolean = {
+    val startTime = System.currentTimeMillis()
+    socketPortProb(host, port)
+      .dropWhile(p => p.isSuccess && System.currentTimeMillis() - startTime < timeout)
+      .next()
+      .isFailure
+  }
+
+  /**
+   * Returns array of unique randomly available tcp ports of specified count.
+   */
+  def getRandomAvailableTCPPorts(count: Int): Seq[Int] =
+    (0 until count).map(x => AvailablePort.getRandomAvailablePortKeeper(AvailablePort.SOCKET))
+      .map{x => x.release(); x.getPort}.toArray
+
+  /**
+   * config a log4j properties used for integration tests
+   */
+  def configTestLog4j(level: String, props: (String, String)*): Unit = {
+    val pro = new Properties()
+    props.foreach(p => pro.put(p._1, p._2))
+    configTestLog4j(level, pro)
+  }
+
+  def configTestLog4j(level: String, props: Properties): Unit = {
+    val pro = new Properties()
+    pro.put("log4j.rootLogger", s"$level, console")
+    pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender")
+    pro.put("log4j.appender.console.target", "System.err")
+    pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout")
+    pro.put("log4j.appender.console.layout.ConversionPattern",
+      "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n")
+    pro.putAll(props)
+    PropertyConfigurator.configure(pro)
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
new file mode 100644
index 0000000..67f9e57
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/ManualClockHelper.scala
@@ -0,0 +1,28 @@
+/*
+ * 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.spark.streaming
+
+import org.apache.spark.util.ManualClock
+
+object ManualClockHelper {
+
+  def addToTime(ssc: StreamingContext, timeToAdd: Long): Unit = {
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    clock.advance(timeToAdd)
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala b/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.scala
new file mode 100644
index 0000000..fce1e67
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/it/scala/org/apache/spark/streaming/TestInputDStream.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.spark.streaming
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.dstream.InputDStream
+
+import scala.reflect.ClassTag
+
+class TestInputDStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
+  extends InputDStream[T](ssc_) {
+
+  def start() {}
+
+  def stop() {}
+
+  def compute(validTime: Time): Option[RDD[T]] = {
+    logInfo("Computing RDD for time " + validTime)
+    val index = ((validTime - zeroTime) / slideDuration - 1).toInt
+    val selectedInput = if (index < input.size) input(index) else Seq[T]()
+
+    // lets us test cases where RDDs are not created
+    if (selectedInput == null)
+      return None
+
+    val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
+    logInfo("Created RDD " + rdd.id + " with " + selectedInput)
+    Some(rdd)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaDStreamFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaDStreamFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaDStreamFunctions.java
new file mode 100644
index 0000000..527b462
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaDStreamFunctions.java
@@ -0,0 +1,86 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.streaming.GemFireDStreamFunctions;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import java.util.Properties;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.JavaAPIHelper.*;
+
+/**
+ * A Java API wrapper over {@link org.apache.spark.streaming.api.java.JavaDStream}
+ * to provide GemFire Spark Connector functionality.
+ *
+ * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */ 
+public class GemFireJavaDStreamFunctions<T> {
+  
+  public final GemFireDStreamFunctions<T> dsf;
+
+  public GemFireJavaDStreamFunctions(JavaDStream<T> ds) {
+    this.dsf = new GemFireDStreamFunctions<T>(ds.dstream());
+  }
+
+  /**
+   * Save the JavaDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored  
+   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */
+  public <K, V> void saveToGemfire(
+    String regionPath, PairFunction<T, K, V> func, GemFireConnectionConf connConf, Properties opConf) {
+    dsf.saveToGemfire(regionPath, func, connConf, propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the JavaDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored  
+   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
+   * @param opConf the optional  parameters for this operation
+   */
+  public <K, V> void saveToGemfire(
+          String regionPath, PairFunction<T, K, V> func, Properties opConf) {
+    dsf.saveToGemfire(regionPath, func, dsf.defaultConnectionConf(), propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the JavaDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored
+   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   */
+  public <K, V> void saveToGemfire(
+          String regionPath, PairFunction<T, K, V> func, GemFireConnectionConf connConf) {
+    dsf.saveToGemfire(regionPath, func, connConf, emptyStrStrMap());
+  }
+
+  /**
+   * Save the JavaDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored
+   * @param func the PairFunction that converts elements of JavaDStream to key/value pairs
+   */
+  public <K, V> void saveToGemfire(
+          String regionPath, PairFunction<T, K, V> func) {
+    dsf.saveToGemfire(regionPath, func, dsf.defaultConnectionConf(), emptyStrStrMap());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairDStreamFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairDStreamFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairDStreamFunctions.java
new file mode 100644
index 0000000..6556462
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairDStreamFunctions.java
@@ -0,0 +1,77 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.streaming.GemFirePairDStreamFunctions;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import java.util.Properties;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.JavaAPIHelper.*;
+
+/**
+ * A Java API wrapper over {@link org.apache.spark.streaming.api.java.JavaPairDStream}
+ * to provide GemFire Spark Connector functionality.
+ *
+ * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */
+public class GemFireJavaPairDStreamFunctions<K, V> {
+  
+  public final GemFirePairDStreamFunctions<K, V> dsf;
+
+  public GemFireJavaPairDStreamFunctions(JavaPairDStream<K, V> ds) {    
+    this.dsf = new GemFirePairDStreamFunctions<K, V>(ds.dstream());
+  }
+
+  /**
+   * Save the JavaPairDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored  
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */  
+  public void saveToGemfire(String regionPath, GemFireConnectionConf connConf, Properties opConf) {
+    dsf.saveToGemfire(regionPath, connConf, propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the JavaPairDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored  
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   */
+  public void saveToGemfire(String regionPath, GemFireConnectionConf connConf) {
+    dsf.saveToGemfire(regionPath, connConf, emptyStrStrMap());
+  }
+
+  /**
+   * Save the JavaPairDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored
+   * @param opConf the optional parameters for this operation
+   */
+  public void saveToGemfire(String regionPath, Properties opConf) {
+    dsf.saveToGemfire(regionPath, dsf.defaultConnectionConf(), propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the JavaPairDStream to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored
+   */
+  public void saveToGemfire(String regionPath) {
+    dsf.saveToGemfire(regionPath, dsf.defaultConnectionConf(), emptyStrStrMap());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairRDDFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairRDDFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairRDDFunctions.java
new file mode 100644
index 0000000..72fa7a9
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaPairRDDFunctions.java
@@ -0,0 +1,238 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.GemFirePairRDDFunctions;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireJoinRDD;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireOuterJoinRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.function.Function;
+import scala.Option;
+import scala.Tuple2;
+import scala.reflect.ClassTag;
+
+import java.util.Properties;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.JavaAPIHelper.*;
+
+/**
+ * A Java API wrapper over {@link org.apache.spark.api.java.JavaPairRDD} to provide GemFire Spark
+ * Connector functionality.
+ *
+ * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */
+public class GemFireJavaPairRDDFunctions<K, V> {
+
+  public final GemFirePairRDDFunctions<K, V> rddf;
+
+  public GemFireJavaPairRDDFunctions(JavaPairRDD<K, V> rdd) {
+    this.rddf = new GemFirePairRDDFunctions<K, V>(rdd.rdd());
+  }
+
+  /**
+   * Save the pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the parameters for this operation
+   */
+  public void saveToGemfire(String regionPath, GemFireConnectionConf connConf, Properties opConf) {
+    rddf.saveToGemfire(regionPath, connConf, propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored
+   * @param opConf the parameters for this operation
+   */
+  public void saveToGemfire(String regionPath, Properties opConf) {
+    rddf.saveToGemfire(regionPath, rddf.defaultConnectionConf(), propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   */
+  public void saveToGemfire(String regionPath, GemFireConnectionConf connConf) {
+    rddf.saveToGemfire(regionPath, connConf, emptyStrStrMap());
+  }
+
+  /**
+   * Save the pair RDD to GemFire key-value store with the default GemFireConnector.
+   * @param regionPath the full path of region that the RDD is stored
+   */
+  public void saveToGemfire(String regionPath) {
+    rddf.saveToGemfire(regionPath, rddf.defaultConnectionConf(), emptyStrStrMap());
+  }
+
+  /**
+   * Return an JavaPairRDD containing all pairs of elements with matching keys in
+   * this RDD&lt;K, V> and the GemFire `Region&lt;K, V2>`. Each pair of elements
+   * will be returned as a ((k, v), v2) tuple, where (k, v) is in this RDD and
+   * (k, v2) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;&lt;K, V>, V2>
+   */  
+  public <V2> JavaPairRDD<Tuple2<K, V>, V2> joinGemfireRegion(String regionPath) {
+    return joinGemfireRegion(regionPath, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Return an JavaPairRDD containing all pairs of elements with matching keys in
+   * this RDD&lt;K, V> and the GemFire `Region&lt;K, V2>`. Each pair of elements
+   * will be returned as a ((k, v), v2) tuple, where (k, v) is in this RDD and
+   * (k, v2) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;&lt;K, V>, V2>
+   */
+  public <V2> JavaPairRDD<Tuple2<K, V>, V2> joinGemfireRegion(
+    String regionPath, GemFireConnectionConf connConf) {
+    GemFireJoinRDD<Tuple2<K, V>, K, V2> rdd = rddf.joinGemfireRegion(regionPath, connConf);
+    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
+    ClassTag<V2> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in this
+   * RDD&lt;K, V> and the GemFire `Region&lt;K2, V2>`. The join key from RDD
+   * element is generated by `func(K, V) => K2`, and the key from the GemFire
+   * region is just the key of the key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
+   * where (k, v) is in this RDD and (k2, v2) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param <K2> the key type of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, V2>
+   */
+  public <K2, V2> JavaPairRDD<Tuple2<K, V>, V2> joinGemfireRegion(
+    String regionPath, Function<Tuple2<K, V>, K2> func) {
+    return joinGemfireRegion(regionPath, func, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in this
+   * RDD&lt;K, V> and the GemFire `Region&lt;K2, V2>`. The join key from RDD 
+   * element is generated by `func(K, V) => K2`, and the key from the GemFire 
+   * region is just the key of the key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
+   * where (k, v) is in this RDD and (k2, v2) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <K2> the key type of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, V2>
+   */  
+  public <K2, V2> JavaPairRDD<Tuple2<K, V>, V2> joinGemfireRegion(
+    String regionPath, Function<Tuple2<K, V>, K2> func, GemFireConnectionConf connConf) {
+    GemFireJoinRDD<Tuple2<K, V>, K2, V2> rdd = rddf.joinGemfireRegion(regionPath, func, connConf);
+    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
+    ClassTag<V2> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;K, V> and the GemFire `Region&lt;K, V2>`.
+   * For each element (k, v) in this RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key k.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
+   */
+  public <V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGemfireRegion(String regionPath) {
+    return outerJoinGemfireRegion(regionPath, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;K, V> and the GemFire `Region&lt;K, V2>`.
+   * For each element (k, v) in this RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key k.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
+   */  
+  public <V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGemfireRegion(
+    String regionPath, GemFireConnectionConf connConf) {
+    GemFireOuterJoinRDD<Tuple2<K, V>, K, V2> rdd = rddf.outerJoinGemfireRegion(regionPath, connConf);
+    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
+    ClassTag<Option<V2>> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;K, V> and the GemFire `Region&lt;K2, V2>`.
+   * The join key from RDD element is generated by `func(K, V) => K2`, and the
+   * key from region is just the key of the key/value pair.
+   *
+   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key `func(k, v)`.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param <K2> the key type of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
+   */
+  public <K2, V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGemfireRegion(
+    String regionPath, Function<Tuple2<K, V>, K2> func) {
+    return outerJoinGemfireRegion(regionPath, func, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;K, V> and the GemFire `Region&lt;K2, V2>`.
+   * The join key from RDD element is generated by `func(K, V) => K2`, and the
+   * key from region is just the key of the key/value pair.
+   *
+   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key `func(k, v)`.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <K2> the key type of the GemFire region
+   * @param <V2> the value type of the GemFire region
+   * @return JavaPairRDD&lt;Tuple2&lt;K, V>, Option&lt;V>>
+   */
+  public <K2, V2> JavaPairRDD<Tuple2<K, V>, Option<V2>> outerJoinGemfireRegion(
+    String regionPath, Function<Tuple2<K, V>, K2> func, GemFireConnectionConf connConf) {
+    GemFireOuterJoinRDD<Tuple2<K, V>, K2, V2> rdd = rddf.outerJoinGemfireRegion(regionPath, func, connConf);
+    ClassTag<Tuple2<K, V>> kt = fakeClassTag();
+    ClassTag<Option<V2>> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRDDFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRDDFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRDDFunctions.java
new file mode 100644
index 0000000..519ba6e
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaRDDFunctions.java
@@ -0,0 +1,178 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.GemFireRDDFunctions;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireJoinRDD;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireOuterJoinRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Option;
+import scala.reflect.ClassTag;
+
+import java.util.Properties;
+
+import static io.pivotal.gemfire.spark.connector.javaapi.JavaAPIHelper.*;
+
+/**
+ * A Java API wrapper over {@link org.apache.spark.api.java.JavaRDD} to provide GemFire Spark
+ * Connector functionality.
+ *
+ * <p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */
+public class GemFireJavaRDDFunctions<T> {
+
+  public final GemFireRDDFunctions<T> rddf;
+
+  public GemFireJavaRDDFunctions(JavaRDD<T> rdd) {
+    this.rddf = new GemFireRDDFunctions<T>(rdd.rdd());
+  }
+
+  /**
+   * Save the non-pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored  
+   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the parameters for this operation
+   */  
+  public <K, V> void saveToGemfire(
+    String regionPath, PairFunction<T, K, V> func, GemFireConnectionConf connConf, Properties opConf) {
+    rddf.saveToGemfire(regionPath, func, connConf, propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the non-pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored  
+   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   */
+  public <K, V> void saveToGemfire(
+    String regionPath, PairFunction<T, K, V> func, GemFireConnectionConf connConf) {
+    rddf.saveToGemfire(regionPath, func, connConf, emptyStrStrMap());
+  }
+
+  /**
+   * Save the non-pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored
+   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
+   * @param opConf the parameters for this operation
+   */
+  public <K, V> void saveToGemfire(
+    String regionPath, PairFunction<T, K, V> func, Properties opConf) {
+    rddf.saveToGemfire(regionPath, func, rddf.defaultConnectionConf(), propertiesToScalaMap(opConf));
+  }
+
+  /**
+   * Save the non-pair RDD to GemFire key-value store with default GemFireConnector.
+   * @param regionPath the full path of region that the RDD is stored  
+   * @param func the PairFunction that converts elements of JavaRDD to key/value pairs
+   */
+  public <K, V> void saveToGemfire(String regionPath, PairFunction<T, K, V> func) {
+    rddf.saveToGemfire(regionPath, func, rddf.defaultConnectionConf(), emptyStrStrMap());
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in this
+   * RDD&lt;T> and the GemFire `Region&lt;K, V>`. The join key from RDD
+   * element is generated by `func(T) => K`, and the key from the GemFire
+   * region is just the key of the key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a (t, v2) tuple,
+   * where t is from this RDD and v is from the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element T
+   * @param <K> the key type of the GemFire region
+   * @param <V> the value type of the GemFire region
+   * @return JavaPairRDD&lt;T, V>
+   */
+  public <K, V> JavaPairRDD<T, V> joinGemfireRegion(String regionPath, Function<T, K> func) {
+    return joinGemfireRegion(regionPath, func, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in this
+   * RDD&lt;T> and the GemFire `Region&lt;K, V>`. The join key from RDD
+   * element is generated by `func(T) => K`, and the key from the GemFire
+   * region is just the key of the key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a (t, v2) tuple,
+   * where t is from this RDD and v is from the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element T
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <K> the key type of the GemFire region
+   * @param <V> the value type of the GemFire region
+   * @return JavaPairRDD&lt;T, V>
+   */
+  public <K, V> JavaPairRDD<T, V> joinGemfireRegion(
+    String regionPath, Function<T, K> func, GemFireConnectionConf connConf) {
+    GemFireJoinRDD<T, K, V> rdd = rddf.joinGemfireRegion(regionPath, func, connConf);
+    ClassTag<T> kt = fakeClassTag();
+    ClassTag<V> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;T> and the GemFire `Region&lt;K, V>`.
+   * The join key from RDD element is generated by `func(T) => K`, and the
+   * key from region is just the key of the key/value pair.
+   *
+   * For each element (t) in this RDD, the resulting RDD will either contain
+   * all pairs (t, Some(v)) for v in the GemFire region, or the pair
+   * (t, None) if no element in the GemFire region have key `func(t)`.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element T
+   * @param <K> the key type of the GemFire region
+   * @param <V> the value type of the GemFire region
+   * @return JavaPairRDD&lt;T, Option&lt;V>>
+   */
+  public <K, V> JavaPairRDD<T, Option<V>> outerJoinGemfireRegion(String regionPath, Function<T, K> func) {
+    return outerJoinGemfireRegion(regionPath, func, rddf.defaultConnectionConf());
+  }
+
+  /**
+   * Perform a left outer join of this RDD&lt;T> and the GemFire `Region&lt;K, V>`.
+   * The join key from RDD element is generated by `func(T) => K`, and the
+   * key from region is just the key of the key/value pair.
+   *
+   * For each element (t) in this RDD, the resulting RDD will either contain
+   * all pairs (t, Some(v)) for v in the GemFire region, or the pair
+   * (t, None) if no element in the GemFire region have key `func(t)`.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element T
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param <K> the key type of the GemFire region
+   * @param <V> the value type of the GemFire region
+   * @return JavaPairRDD&lt;T, Option&lt;V>>
+   */
+  public <K, V> JavaPairRDD<T, Option<V>> outerJoinGemfireRegion(
+    String regionPath, Function<T, K> func, GemFireConnectionConf connConf) {
+    GemFireOuterJoinRDD<T, K, V> rdd = rddf.outerJoinGemfireRegion(regionPath, func, connConf);
+    ClassTag<T> kt = fakeClassTag();
+    ClassTag<Option<V>> vt = fakeClassTag();
+    return new JavaPairRDD<>(rdd, kt, vt);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSQLContextFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSQLContextFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSQLContextFunctions.java
new file mode 100644
index 0000000..980c409
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSQLContextFunctions.java
@@ -0,0 +1,49 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.GemFireSQLContextFunctions;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.SQLContext;
+
+/**
+ * Java API wrapper over {@link org.apache.spark.sql.SQLContext} to provide GemFire
+ * OQL functionality.
+ *
+ * <p></p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */
+public class GemFireJavaSQLContextFunctions {
+
+  public final GemFireSQLContextFunctions scf;
+
+  public GemFireJavaSQLContextFunctions(SQLContext sqlContext) {
+    scf = new GemFireSQLContextFunctions(sqlContext);
+  }
+
+  public <T> DataFrame gemfireOQL(String query) {
+    DataFrame df = scf.gemfireOQL(query, scf.defaultConnectionConf());
+    return df;
+  }
+
+  public <T> DataFrame gemfireOQL(String query, GemFireConnectionConf connConf) {
+    DataFrame df = scf.gemfireOQL(query, connConf);
+    return df;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSparkContextFunctions.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSparkContextFunctions.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSparkContextFunctions.java
new file mode 100644
index 0000000..f8b930c
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaSparkContextFunctions.java
@@ -0,0 +1,87 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRegionRDD;
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRegionRDD$;
+import org.apache.spark.SparkContext;
+import static io.pivotal.gemfire.spark.connector.javaapi.JavaAPIHelper.*;
+
+import scala.reflect.ClassTag;
+import java.util.Properties;
+
+/**
+ * Java API wrapper over {@link org.apache.spark.SparkContext} to provide GemFire
+ * Connector functionality.
+ *
+ * <p></p>To obtain an instance of this wrapper, use one of the factory methods in {@link
+ * io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil} class.</p>
+ */
+public class GemFireJavaSparkContextFunctions {
+
+  public final SparkContext sc;
+
+  public GemFireJavaSparkContextFunctions(SparkContext sc) {
+    this.sc = sc;
+  }
+
+  /**
+   * Expose a GemFire region as a JavaPairRDD
+   * @param regionPath the full path of the region
+   * @param connConf the GemFireConnectionConf that can be used to access the region
+   * @param opConf the parameters for this operation, such as preferred partitioner.
+   */
+  public <K, V> GemFireJavaRegionRDD<K, V> gemfireRegion(
+    String regionPath, GemFireConnectionConf connConf, Properties opConf) {
+    ClassTag<K> kt = fakeClassTag();
+    ClassTag<V> vt = fakeClassTag();    
+    GemFireRegionRDD<K, V>  rdd =  GemFireRegionRDD$.MODULE$.apply(
+      sc, regionPath, connConf, propertiesToScalaMap(opConf), kt, vt);
+    return new GemFireJavaRegionRDD<>(rdd);
+  }
+
+  /**
+   * Expose a GemFire region as a JavaPairRDD with default GemFireConnector and no preferred partitioner.
+   * @param regionPath the full path of the region
+   */
+  public <K, V> GemFireJavaRegionRDD<K, V> gemfireRegion(String regionPath) {
+    GemFireConnectionConf connConf = GemFireConnectionConf.apply(sc.getConf());
+    return gemfireRegion(regionPath, connConf, new Properties());
+  }
+
+  /**
+   * Expose a GemFire region as a JavaPairRDD with no preferred partitioner.
+   * @param regionPath the full path of the region
+   * @param connConf the GemFireConnectionConf that can be used to access the region
+   */
+  public <K, V> GemFireJavaRegionRDD<K, V> gemfireRegion(String regionPath, GemFireConnectionConf connConf) {
+    return gemfireRegion(regionPath, connConf, new Properties());
+  }
+
+  /**
+   * Expose a GemFire region as a JavaPairRDD with default GemFireConnector.
+   * @param regionPath the full path of the region
+   * @param opConf the parameters for this operation, such as preferred partitioner.
+   */
+  public <K, V> GemFireJavaRegionRDD<K, V> gemfireRegion(String regionPath, Properties opConf) {
+    GemFireConnectionConf connConf = GemFireConnectionConf.apply(sc.getConf());
+    return gemfireRegion(regionPath, connConf, opConf);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaUtil.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaUtil.java b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaUtil.java
new file mode 100644
index 0000000..679f197
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/java/io/pivotal/gemfire/spark/connector/javaapi/GemFireJavaUtil.java
@@ -0,0 +1,122 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.javaapi;
+
+import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import scala.Tuple2;
+
+import io.pivotal.gemfire.spark.connector.package$;
+
+/**
+ * The main entry point to Spark GemFire Connector Java API.
+ *
+ * There are several helpful static factory methods which build useful wrappers
+ * around Spark Context, Streaming Context and RDD. There are also helper methods
+ * to convert JavaRDD<Tuple2<K, V>> to JavaPairRDD<K, V>.
+ */
+public final class GemFireJavaUtil {
+  
+  /** constants */
+  public static String GemFireLocatorPropKey = package$.MODULE$.GemFireLocatorPropKey();
+  // partitioner related keys and values
+  public static String PreferredPartitionerPropKey = package$.MODULE$.PreferredPartitionerPropKey();
+  public static String NumberPartitionsPerServerPropKey = package$.MODULE$.NumberPartitionsPerServerPropKey();
+  public static String OnePartitionPartitionerName = package$.MODULE$.OnePartitionPartitionerName();
+  public static String ServerSplitsPartitionerName = package$.MODULE$.ServerSplitsPartitionerName();
+  public static String RDDSaveBatchSizePropKey = package$.MODULE$.RDDSaveBatchSizePropKey();
+  public static int RDDSaveBatchSizeDefault = package$.MODULE$.RDDSaveBatchSizeDefault();
+  
+  /** The private constructor is used prevents user from creating instance of this class. */
+  private GemFireJavaUtil() { }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaSparkContextFunctions} based
+   * on an existing {@link SparkContext} instance.
+   */
+  public static GemFireJavaSparkContextFunctions javaFunctions(SparkContext sc) {
+    return new GemFireJavaSparkContextFunctions(sc);
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaSparkContextFunctions} based
+   * on an existing {@link JavaSparkContext} instance.
+   */
+  public static GemFireJavaSparkContextFunctions javaFunctions(JavaSparkContext jsc) {
+    return new GemFireJavaSparkContextFunctions(JavaSparkContext.toSparkContext(jsc));
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaPairRDDFunctions} based on an
+   * existing {@link org.apache.spark.api.java.JavaPairRDD} instance.
+   */
+  public static <K, V> GemFireJavaPairRDDFunctions<K, V> javaFunctions(JavaPairRDD<K, V> rdd) {
+    return new GemFireJavaPairRDDFunctions<K, V>(rdd);
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaRDDFunctions} based on an
+   * existing {@link org.apache.spark.api.java.JavaRDD} instance.
+   */
+  public static <T> GemFireJavaRDDFunctions<T> javaFunctions(JavaRDD<T> rdd) {
+    return new GemFireJavaRDDFunctions<T>(rdd);
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaPairDStreamFunctions} based on an
+   * existing {@link org.apache.spark.streaming.api.java.JavaPairDStream} instance.
+   */
+  public static <K, V> GemFireJavaPairDStreamFunctions<K, V> javaFunctions(JavaPairDStream<K, V> ds) {
+    return new GemFireJavaPairDStreamFunctions<>(ds);
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaDStreamFunctions} based on an
+   * existing {@link org.apache.spark.streaming.api.java.JavaDStream} instance.
+   */
+  public static <T> GemFireJavaDStreamFunctions<T> javaFunctions(JavaDStream<T> ds) {
+    return new GemFireJavaDStreamFunctions<>(ds);
+  }
+
+  /** Convert an instance of {@link org.apache.spark.api.java.JavaRDD}&lt;&lt;Tuple2&lt;K, V&gt;&gt;
+   * to a {@link org.apache.spark.api.java.JavaPairRDD}&lt;K, V&gt;.
+   */
+  public static <K, V> JavaPairRDD<K, V> toJavaPairRDD(JavaRDD<Tuple2<K, V>> rdd) {
+    return JavaAPIHelper.toJavaPairRDD(rdd);
+  }
+
+  /** Convert an instance of {@link org.apache.spark.streaming.api.java.JavaDStream}&lt;&lt;Tuple2&lt;K, V&gt;&gt;
+   * to a {@link org.apache.spark.streaming.api.java.JavaPairDStream}&lt;K, V&gt;.
+   */
+  public static <K, V> JavaPairDStream<K, V> toJavaPairDStream(JavaDStream<Tuple2<K, V>> ds) {
+    return JavaAPIHelper.toJavaPairDStream(ds);
+  }
+
+  /**
+   * A static factory method to create a {@link GemFireJavaSQLContextFunctions} based
+   * on an existing {@link SQLContext} instance.
+   */
+  public static GemFireJavaSQLContextFunctions javaFunctions(SQLContext sqlContext) {
+    return new GemFireJavaSQLContextFunctions(sqlContext);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnection.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnection.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnection.scala
new file mode 100644
index 0000000..39ec1c1
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnection.scala
@@ -0,0 +1,67 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import com.gemstone.gemfire.cache.execute.ResultCollector
+import com.gemstone.gemfire.cache.query.Query
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartition
+
+
+trait GemFireConnection {
+
+  /**
+   * Validate region existence and key/value type constraints, throw RuntimeException
+   * if region does not exist or key and/or value type do(es) not match.
+   * @param regionPath the full path of region
+   */
+  def validateRegion[K, V](regionPath: String): Unit
+
+  /**
+   * Get Region proxy for the given region
+   * @param regionPath the full path of region
+   */
+  def getRegionProxy[K, V](regionPath: String): Region[K, V]
+
+  /**
+   * Retrieve region meta data for the given region. 
+   * @param regionPath: the full path of the region
+   * @return Some[RegionMetadata] if region exists, None otherwise
+   */
+  def getRegionMetadata[K, V](regionPath: String): Option[RegionMetadata]
+
+  /** 
+   * Retrieve region data for the given region and bucket set 
+   * @param regionPath: the full path of the region
+   * @param whereClause: the set of bucket IDs
+   * @param split: GemFire RDD Partition instance
+   */
+  def getRegionData[K, V](regionPath: String, whereClause: Option[String], split: GemFireRDDPartition): Iterator[(K, V)]
+
+  def executeQuery(regionPath: String, bucketSet: Set[Int], queryString: String): Object
+  /** 
+   * Create a gemfire OQL query
+   * @param queryString GemFire OQL query string
+   */
+  def getQuery(queryString: String): Query
+
+  /** Close the connection */
+  def close(): Unit
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionConf.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionConf.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionConf.scala
new file mode 100644
index 0000000..ea6d246
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionConf.scala
@@ -0,0 +1,73 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import org.apache.spark.SparkConf
+import io.pivotal.gemfire.spark.connector.internal.{DefaultGemFireConnectionManager, LocatorHelper}
+
+/**
+ * Stores configuration of a connection to GemFire cluster. It is serializable and can
+ * be safely sent over network.
+ *
+ * @param locators GemFire locator host:port pairs, the default is (localhost,10334)
+ * @param gemfireProps The initial gemfire properties to be used.
+ * @param connectionManager GemFireConnectionFactory instance
+ */
+class GemFireConnectionConf(
+   val locators: Seq[(String, Int)], 
+   val gemfireProps: Map[String, String] = Map.empty,
+   connectionManager: GemFireConnectionManager = new DefaultGemFireConnectionManager
+  ) extends Serializable {
+
+  /** require at least 1 pair of (host,port) */
+  require(locators.nonEmpty)
+  
+  def getConnection: GemFireConnection = connectionManager.getConnection(this)
+  
+}
+
+object GemFireConnectionConf {
+
+  /**
+   * create GemFireConnectionConf object based on locator string and optional GemFireConnectionFactory
+   * @param locatorStr GemFire cluster locator string
+   * @param connectionManager GemFireConnection factory
+   */
+  def apply(locatorStr: String, gemfireProps: Map[String, String] = Map.empty)
+    (implicit connectionManager: GemFireConnectionManager = new DefaultGemFireConnectionManager): GemFireConnectionConf = {
+    new GemFireConnectionConf(LocatorHelper.parseLocatorsString(locatorStr), gemfireProps, connectionManager)
+  }
+
+  /**
+   * create GemFireConnectionConf object based on SparkConf. Note that implicit can
+   * be used to control what GemFireConnectionFactory instance to use if desired
+   * @param conf a SparkConf instance 
+   */
+  def apply(conf: SparkConf): GemFireConnectionConf = {
+    val locatorStr = conf.getOption(GemFireLocatorPropKey).getOrElse(
+      throw new RuntimeException(s"SparkConf does not contain property $GemFireLocatorPropKey"))
+    // SparkConf only holds properties whose key starts with "spark.", In order to
+    // put gemfire properties in SparkConf, all gemfire properties are prefixes with
+    // "spark.gemfire.". This prefix was removed before the properties were put in `gemfireProp`
+    val prefix = "spark.gemfire."
+    val gemfireProps = conf.getAll.filter {
+        case (k, v) => k.startsWith(prefix) && k != GemFireLocatorPropKey
+      }.map { case (k, v) => (k.substring(prefix.length), v) }.toMap
+    apply(locatorStr, gemfireProps)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionManager.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionManager.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionManager.scala
new file mode 100644
index 0000000..7386f5c
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireConnectionManager.scala
@@ -0,0 +1,31 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+/**
+ * GemFireConnectionFactory provide an common interface that manages GemFire
+ * connections, and it's serializable. Each factory instance will handle
+ * connection instance creation and connection pool management.
+ */
+trait GemFireConnectionManager extends Serializable {
+
+  /** get connection for the given connector */
+  def getConnection(connConf: GemFireConnectionConf): GemFireConnection
+
+  /** close the connection */
+  def closeConnection(connConf: GemFireConnectionConf): Unit
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployer.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployer.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployer.scala
new file mode 100644
index 0000000..96a7e81
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployer.scala
@@ -0,0 +1,81 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import java.io.File
+import java.net.URL
+import org.apache.commons.httpclient.methods.PostMethod
+import org.apache.commons.httpclient.methods.multipart.{FilePart, Part, MultipartRequestEntity}
+import org.apache.commons.httpclient.HttpClient
+import org.apache.spark.Logging
+
+object GemFireFunctionDeployer {
+  def main(args: Array[String]) {
+    new GemFireFunctionDeployer(new HttpClient()).commandLineRun(args)
+  }
+}
+
+class GemFireFunctionDeployer(val httpClient:HttpClient) extends Logging {
+
+  def deploy(host: String, port: Int, jarLocation: String): String =
+    deploy(host + ":" + port, jarLocation)
+  
+  def deploy(host: String, port: Int, jar:File): String =
+    deploy(host + ":" + port, jar)
+  
+  def deploy(jmxHostAndPort: String, jarLocation: String): String =
+    deploy(jmxHostAndPort, jarFileHandle(jarLocation))
+  
+  def deploy(jmxHostAndPort: String, jar: File): String = {
+    val urlString = constructURLString(jmxHostAndPort)
+    val filePost: PostMethod = new PostMethod(urlString)
+    val parts: Array[Part] = new Array[Part](1)
+    parts(0) = new FilePart("resources", jar)
+    filePost.setRequestEntity(new MultipartRequestEntity(parts, filePost.getParams))
+    val status: Int = httpClient.executeMethod(filePost)
+    "Deployed Jar with status:" + status
+  }
+
+  private[connector] def constructURLString(jmxHostAndPort: String) =
+    "http://" + jmxHostAndPort + "/gemfire/v1/deployed"
+
+  private[connector]def jarFileHandle(jarLocation: String) = {
+    val f: File = new File(jarLocation)
+    if (!f.exists()) {
+      val errorMessage: String = "Invalid jar file:" + f.getAbsolutePath
+      logInfo(errorMessage)
+      throw new RuntimeException(errorMessage)
+    }
+    f
+  }
+  
+  def commandLineRun(args: Array[String]):Unit = {
+    val (hostPort: String, jarFile: String) =
+    if (args.length < 2) {
+      logInfo("JMX Manager Host and Port (example: localhost:7070):")
+      val bufferedReader = new java.io.BufferedReader(new java.io.InputStreamReader(System.in))
+      val jmxHostAndPort = bufferedReader.readLine()
+      logInfo("Location of gemfire-functions.jar:")
+      val functionJarLocation = bufferedReader.readLine()
+      (jmxHostAndPort, functionJarLocation)
+    } else {
+      (args(0), args(1))
+    }
+    val status = deploy(hostPort, jarFile)
+    logInfo(status)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireKryoRegistrator.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireKryoRegistrator.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireKryoRegistrator.scala
new file mode 100644
index 0000000..196d991
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireKryoRegistrator.scala
@@ -0,0 +1,29 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import com.esotericsoftware.kryo.Kryo
+import io.pivotal.gemfire.spark.connector.internal.oql.UndefinedSerializer
+import org.apache.spark.serializer.KryoRegistrator
+import com.gemstone.gemfire.cache.query.internal.Undefined
+
+class GemFireKryoRegistrator extends KryoRegistrator{
+
+  override def registerClasses(kyro: Kryo): Unit = {
+    kyro.addDefaultSerializer(classOf[Undefined], classOf[UndefinedSerializer])
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
new file mode 100644
index 0000000..63583da
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
@@ -0,0 +1,140 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFireOuterJoinRDD, GemFireJoinRDD, GemFirePairRDDWriter}
+import org.apache.spark.Logging
+import org.apache.spark.api.java.function.Function
+import org.apache.spark.rdd.RDD
+
+/**
+ * Extra gemFire functions on RDDs of (key, value) pairs through an implicit conversion.
+ * Import `io.pivotal.gemfire.spark.connector._` at the top of your program to
+ * use these functions.
+ */
+class GemFirePairRDDFunctions[K, V](val rdd: RDD[(K, V)]) extends Serializable with Logging {
+
+  /**
+   * Save the RDD of pairs to GemFire key-value store without any conversion
+   * @param regionPath the full path of region that the RDD is stored
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */
+  def saveToGemfire(
+      regionPath: String, 
+      connConf: GemFireConnectionConf = defaultConnectionConf, 
+      opConf: Map[String, String] = Map.empty): Unit = {    
+    connConf.getConnection.validateRegion[K, V](regionPath)
+    if (log.isDebugEnabled)
+      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
+    else
+      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
+    val writer = new GemFirePairRDDWriter[K, V](regionPath, connConf, opConf)
+    rdd.sparkContext.runJob(rdd, writer.write _)
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in `this`
+   * RDD and the GemFire `Region[K, V2]`. Each pair of elements will be returned
+   * as a ((k, v), v2) tuple, where (k, v) is in `this` RDD and (k, v2) is in the
+   * GemFire region.
+   *
+   *@param regionPath the region path of the GemFire region
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K2 the key type of the GemFire region
+   * @tparam V2 the value type of the GemFire region
+   * @return RDD[T, V]
+   */
+  def joinGemfireRegion[K2 <: K, V2](
+    regionPath: String, connConf: GemFireConnectionConf = defaultConnectionConf): GemFireJoinRDD[(K, V), K, V2] = {
+    new GemFireJoinRDD[(K, V), K, V2](rdd, null, regionPath, connConf)
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in `this` RDD
+   * and the GemFire `Region[K2, V2]`. The join key from RDD element is generated by
+   * `func(K, V) => K2`, and the key from the GemFire region is jus the key of the
+   * key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a ((k, v), v2) tuple,
+   * where (k, v) is in `this` RDD and (k2, v2) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K2 the key type of the GemFire region
+   * @tparam V2 the value type of the GemFire region
+   * @return RDD[(K, V), V2]
+   */
+  def joinGemfireRegion[K2, V2](
+    regionPath: String, func: ((K, V)) => K2, connConf: GemFireConnectionConf = defaultConnectionConf): GemFireJoinRDD[(K, V), K2, V2] =
+    new GemFireJoinRDD[(K, V), K2, V2](rdd, func, regionPath, connConf)
+
+  /** This version of joinGemfireRegion(...) is just for Java API. */
+  private[connector] def joinGemfireRegion[K2, V2](
+    regionPath: String, func: Function[(K, V), K2], connConf: GemFireConnectionConf): GemFireJoinRDD[(K, V), K2, V2] = {
+    new GemFireJoinRDD[(K, V), K2, V2](rdd, func.call, regionPath, connConf)
+  }
+
+  /**
+   * Perform a left outer join of `this` RDD and the GemFire `Region[K, V2]`.
+   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key k.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K2 the key type of the GemFire region
+   * @tparam V2 the value type of the GemFire region
+   * @return RDD[ (K, V), Option[V] ]
+   */
+  def outerJoinGemfireRegion[K2 <: K, V2](
+    regionPath: String, connConf: GemFireConnectionConf = defaultConnectionConf): GemFireOuterJoinRDD[(K, V), K, V2] = {
+    new GemFireOuterJoinRDD[(K, V), K, V2](rdd, null, regionPath, connConf)
+  }
+
+  /**
+   * Perform a left outer join of `this` RDD and the GemFire `Region[K2, V2]`.
+   * The join key from RDD element is generated by `func(K, V) => K2`, and the
+   * key from region is jus the key of the key/value pair.
+   *
+   * For each element (k, v) in `this` RDD, the resulting RDD will either contain
+   * all pairs ((k, v), Some(v2)) for v2 in the GemFire region, or the pair
+   * ((k, v), None)) if no element in the GemFire region have key `func(k, v)`.
+   *
+   *@param regionPath the region path of the GemFire region
+   * @param func the function that generates region key from RDD element (K, V)
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K2 the key type of the GemFire region
+   * @tparam V2 the value type of the GemFire region
+   * @return RDD[ (K, V), Option[V] ]
+   */
+  def outerJoinGemfireRegion[K2, V2](
+    regionPath: String, func: ((K, V)) => K2, connConf: GemFireConnectionConf = defaultConnectionConf): GemFireOuterJoinRDD[(K, V), K2, V2] = {
+    new GemFireOuterJoinRDD[(K, V), K2, V2](rdd, func, regionPath, connConf)
+  }
+
+  /** This version of outerJoinGemfireRegion(...) is just for Java API. */
+  private[connector] def outerJoinGemfireRegion[K2, V2](
+    regionPath: String, func: Function[(K, V), K2], connConf: GemFireConnectionConf): GemFireOuterJoinRDD[(K, V), K2, V2] = {
+    new GemFireOuterJoinRDD[(K, V), K2, V2](rdd, func.call, regionPath, connConf)
+  }
+
+  private[connector] def defaultConnectionConf: GemFireConnectionConf =
+    GemFireConnectionConf(rdd.sparkContext.getConf)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
new file mode 100644
index 0000000..2039b7f
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
@@ -0,0 +1,120 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFireOuterJoinRDD, GemFireJoinRDD, GemFireRDDWriter}
+import org.apache.spark.Logging
+import org.apache.spark.api.java.function.{PairFunction, Function}
+import org.apache.spark.rdd.RDD
+
+/**
+ * Extra gemFire functions on non-Pair RDDs through an implicit conversion.
+ * Import `io.pivotal.gemfire.spark.connector._` at the top of your program to 
+ * use these functions.  
+ */
+class GemFireRDDFunctions[T](val rdd: RDD[T]) extends Serializable with Logging {
+
+  /**
+   * Save the non-pair RDD to GemFire key-value store.
+   * @param regionPath the full path of region that the RDD is stored  
+   * @param func the function that converts elements of RDD to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */
+  def saveToGemfire[K, V](
+      regionPath: String, 
+      func: T => (K, V), 
+      connConf: GemFireConnectionConf = defaultConnectionConf,
+      opConf: Map[String, String] = Map.empty): Unit = {
+    connConf.getConnection.validateRegion[K, V](regionPath)
+    if (log.isDebugEnabled)
+      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
+    else
+      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
+    val writer = new GemFireRDDWriter[T, K, V](regionPath, connConf, opConf)
+    rdd.sparkContext.runJob(rdd, writer.write(func) _)
+  }
+
+  /** This version of saveToGemfire(...) is just for Java API. */
+  private[connector] def saveToGemfire[K, V](
+      regionPath: String, 
+      func: PairFunction[T, K, V], 
+      connConf: GemFireConnectionConf, 
+      opConf: Map[String, String]): Unit = {
+    saveToGemfire[K, V](regionPath, func.call _, connConf, opConf)
+  }
+
+  /**
+   * Return an RDD containing all pairs of elements with matching keys in `this` RDD
+   * and the GemFire `Region[K, V]`. The join key from RDD element is generated by
+   * `func(T) => K`, and the key from the GemFire region is just the key of the
+   * key/value pair.
+   *
+   * Each pair of elements of result RDD will be returned as a (t, v) tuple, 
+   * where (t) is in `this` RDD and (k, v) is in the GemFire region.
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generate region key from RDD element T
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K the key type of the GemFire region
+   * @tparam V the value type of the GemFire region
+   * @return RDD[T, V]
+   */
+  def joinGemfireRegion[K, V](regionPath: String, func: T => K, 
+    connConf: GemFireConnectionConf = defaultConnectionConf): GemFireJoinRDD[T, K, V] = {
+    new GemFireJoinRDD[T, K, V](rdd, func, regionPath, connConf)    
+  }
+
+  /** This version of joinGemfireRegion(...) is just for Java API. */
+  private[connector] def joinGemfireRegion[K, V](
+    regionPath: String, func: Function[T, K], connConf: GemFireConnectionConf): GemFireJoinRDD[T, K, V] = {
+    joinGemfireRegion(regionPath, func.call _, connConf)
+  }
+
+  /**
+   * Perform a left outer join of `this` RDD and the GemFire `Region[K, V]`.
+   * The join key from RDD element is generated by `func(T) => K`, and the
+   * key from region is just the key of the key/value pair.
+   *
+   * For each element (t) in `this` RDD, the resulting RDD will either contain
+   * all pairs (t, Some(v)) for v in the GemFire region, or the pair
+   * (t, None) if no element in the GemFire region have key `func(t)`
+   *
+   * @param regionPath the region path of the GemFire region
+   * @param func the function that generate region key from RDD element T
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @tparam K the key type of the GemFire region
+   * @tparam V the value type of the GemFire region
+   * @return RDD[ T, Option[V] ]
+   */
+  def outerJoinGemfireRegion[K, V](regionPath: String, func: T => K,
+    connConf: GemFireConnectionConf = defaultConnectionConf): GemFireOuterJoinRDD[T, K, V] = {
+    new GemFireOuterJoinRDD[T, K, V](rdd, func, regionPath, connConf)
+  }
+
+  /** This version of outerJoinGemfireRegion(...) is just for Java API. */
+  private[connector] def outerJoinGemfireRegion[K, V](
+    regionPath: String, func: Function[T, K], connConf: GemFireConnectionConf): GemFireOuterJoinRDD[T, K, V] = {
+    outerJoinGemfireRegion(regionPath, func.call _, connConf)
+  }
+
+  private[connector] def defaultConnectionConf: GemFireConnectionConf =
+    GemFireConnectionConf(rdd.sparkContext.getConf)
+
+}
+
+



[06/10] incubator-geode git commit: GEODE-1244: Revert rename of package, directory, project and file rename for geode-spark-connector

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
new file mode 100644
index 0000000..aaf0fcc
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
@@ -0,0 +1,69 @@
+/*
+ * 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 io.pivotal.gemfire.spark
+
+import io.pivotal.gemfire.spark.connector.internal.rdd.{ServerSplitsPartitioner, OnePartitionPartitioner}
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SQLContext
+
+import scala.reflect.ClassTag
+
+/**
+ * The root package of Gemfire connector for Apache Spark.
+ * Provides handy implicit conversions that add gemfire-specific
+ * methods to `SparkContext` and `RDD`.
+ */
+package object connector {
+
+  /** constants */
+  final val GemFireLocatorPropKey = "spark.gemfire.locators"
+  // partitioner related keys and values
+  final val PreferredPartitionerPropKey = "preferred.partitioner"
+  final val NumberPartitionsPerServerPropKey = "number.partitions.per.server"
+  final val OnePartitionPartitionerName = OnePartitionPartitioner.name
+  final val ServerSplitsPartitionerName = ServerSplitsPartitioner.name
+
+  final val RDDSaveBatchSizePropKey = "rdd.save.batch.size"
+  final val RDDSaveBatchSizeDefault = 10000
+  
+  /** implicits */
+  
+  implicit def toSparkContextFunctions(sc: SparkContext): GemFireSparkContextFunctions =
+    new GemFireSparkContextFunctions(sc)
+
+  implicit def toSQLContextFunctions(sqlContext: SQLContext): GemFireSQLContextFunctions =
+    new GemFireSQLContextFunctions(sqlContext)
+
+  implicit def toGemfirePairRDDFunctions[K: ClassTag, V: ClassTag]
+    (self: RDD[(K, V)]): GemFirePairRDDFunctions[K, V] = new GemFirePairRDDFunctions(self)
+
+  implicit def toGemfireRDDFunctions[T: ClassTag]
+    (self: RDD[T]): GemFireRDDFunctions[T] = new GemFireRDDFunctions(self)
+
+  /** utility implicits */
+  
+  /** convert Map[String, String] to java.util.Properties */
+  implicit def map2Properties(map: Map[String,String]): java.util.Properties =
+    (new java.util.Properties /: map) {case (props, (k,v)) => props.put(k,v); props}
+
+  /** internal util methods */
+  
+  private[connector] def getRddPartitionsInfo(rdd: RDD[_], sep: String = "\n  "): String =
+    rdd.partitions.zipWithIndex.map{case (p,i) => s"$i: $p loc=${rdd.preferredLocations(p)}"}.mkString(sep)
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/GemFireDStreamFunctions.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/GemFireDStreamFunctions.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/GemFireDStreamFunctions.scala
new file mode 100644
index 0000000..6890ec0
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/GemFireDStreamFunctions.scala
@@ -0,0 +1,89 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.streaming
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFirePairRDDWriter, GemFireRDDWriter}
+import org.apache.spark.Logging
+import org.apache.spark.api.java.function.PairFunction
+import org.apache.spark.streaming.dstream.DStream
+
+/**
+ * Extra gemFire functions on DStream of non-pair elements through an implicit conversion.
+ * Import `io.pivotal.gemfire.spark.connector.streaming._` at the top of your program to
+ * use these functions.
+ */
+class GemFireDStreamFunctions[T](val dstream: DStream[T]) extends Serializable with Logging {
+
+  /**
+   * Save the DStream of non-pair elements to GemFire key-value store.
+   * @param regionPath the full path of region that the DStream is stored
+   * @param func the function that converts elements of the DStream to key/value pairs
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */
+  def saveToGemfire[K, V](
+      regionPath: String, 
+      func: T => (K, V), 
+      connConf: GemFireConnectionConf = defaultConnectionConf, 
+      opConf: Map[String, String] = Map.empty): Unit = {
+    connConf.getConnection.validateRegion[K, V](regionPath)
+    val writer = new GemFireRDDWriter[T, K, V](regionPath, connConf, opConf)
+    logInfo(s"""Save DStream region=$regionPath conn=${connConf.locators.mkString(",")}""")
+    dstream.foreachRDD(rdd => rdd.sparkContext.runJob(rdd, writer.write(func) _))
+  }
+
+  /** this version of saveToGemfire is just for Java API */
+  def saveToGemfire[K, V](
+      regionPath: String,
+      func: PairFunction[T, K, V],
+      connConf: GemFireConnectionConf,
+      opConf: Map[String, String] ): Unit = {
+    saveToGemfire[K, V](regionPath, func.call _, connConf, opConf)
+  }
+
+  private[connector] def defaultConnectionConf: GemFireConnectionConf =
+    GemFireConnectionConf(dstream.context.sparkContext.getConf)
+}
+
+
+/**
+ * Extra gemFire functions on DStream of (key, value) pairs through an implicit conversion.
+ * Import `io.pivotal.gemfire.spark.connector.streaming._` at the top of your program to
+ * use these functions.
+ */
+class GemFirePairDStreamFunctions[K, V](val dstream: DStream[(K,V)]) extends Serializable with Logging {
+
+  /**
+   * Save the DStream of pairs to GemFire key-value store without any conversion
+   * @param regionPath the full path of region that the DStream is stored
+   * @param connConf the GemFireConnectionConf object that provides connection to GemFire cluster
+   * @param opConf the optional parameters for this operation
+   */
+  def saveToGemfire(
+      regionPath: String, 
+      connConf: GemFireConnectionConf = defaultConnectionConf, 
+      opConf: Map[String, String] = Map.empty): Unit = {
+    connConf.getConnection.validateRegion[K, V](regionPath)
+    val writer = new GemFirePairRDDWriter[K, V](regionPath, connConf, opConf)
+    logInfo(s"""Save DStream region=$regionPath conn=${connConf.locators.mkString(",")}""")
+    dstream.foreachRDD(rdd => rdd.sparkContext.runJob(rdd, writer.write _))
+  }
+
+  private[connector] def defaultConnectionConf: GemFireConnectionConf =
+    GemFireConnectionConf(dstream.context.sparkContext.getConf)
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/package.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/package.scala b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/package.scala
new file mode 100644
index 0000000..b475cbb
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/streaming/package.scala
@@ -0,0 +1,32 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import org.apache.spark.streaming.dstream.DStream
+
+/**
+ * Provides handy implicit conversions that add gemfire-specific methods to `DStream`.
+ */
+package object streaming {
+
+  implicit def toGemFireDStreamFunctions[T](ds: DStream[T]): GemFireDStreamFunctions[T] =
+    new GemFireDStreamFunctions[T](ds)
+
+  implicit def toGemFirePairDStreamFunctions[K, V](ds: DStream[(K, V)]): GemFirePairDStreamFunctions[K, V] =
+    new GemFirePairDStreamFunctions[K, V](ds)
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/java/io/pivotal/gemfire/spark/connector/JavaAPITest.java
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/java/io/pivotal/gemfire/spark/connector/JavaAPITest.java b/geode-spark-connector/gemfire-spark-connector/src/test/java/io/pivotal/gemfire/spark/connector/JavaAPITest.java
new file mode 100644
index 0000000..2236b4a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/java/io/pivotal/gemfire/spark/connector/JavaAPITest.java
@@ -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 io.pivotal.gemfire.spark.connector;
+
+import io.pivotal.gemfire.spark.connector.javaapi.*;
+import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.sql.SQLContext;
+//import org.apache.spark.sql.api.java.JavaSQLContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.dstream.DStream;
+import org.junit.Test;
+import org.scalatest.junit.JUnitSuite;
+import scala.Function1;
+import scala.Function2;
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.mutable.LinkedList;
+import scala.reflect.ClassTag;
+
+import static org.junit.Assert.*;
+import static io.pivotal.gemfire.spark.connector.javaapi.GemFireJavaUtil.*;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.*;
+
+public class JavaAPITest extends JUnitSuite {
+
+  @SuppressWarnings( "unchecked" )
+  public Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> createCommonMocks() {
+    SparkContext mockSparkContext = mock(SparkContext.class);
+    GemFireConnectionConf mockConnConf = mock(GemFireConnectionConf.class);
+    GemFireConnection mockConnection = mock(GemFireConnection.class);
+    when(mockConnConf.getConnection()).thenReturn(mockConnection);
+    when(mockConnConf.locators()).thenReturn(new LinkedList());
+    return new Tuple3<>(mockSparkContext, mockConnConf, mockConnection);
+  }
+  
+  @Test
+  public void testSparkContextFunction() throws Exception {
+    Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> tuple3 = createCommonMocks();
+    GemFireJavaSparkContextFunctions wrapper = javaFunctions(tuple3._1());
+    assertTrue(tuple3._1() == wrapper.sc);
+    String regionPath = "testregion";
+    JavaPairRDD<String, String> rdd = wrapper.gemfireRegion(regionPath, tuple3._2());
+    verify(tuple3._3()).validateRegion(regionPath);
+  }
+
+  @Test
+  public void testJavaSparkContextFunctions() throws Exception {
+    SparkContext mockSparkContext = mock(SparkContext.class);
+    JavaSparkContext mockJavaSparkContext = mock(JavaSparkContext.class);
+    when(mockJavaSparkContext.sc()).thenReturn(mockSparkContext);
+    GemFireJavaSparkContextFunctions wrapper = javaFunctions(mockJavaSparkContext);
+    assertTrue(mockSparkContext == wrapper.sc);
+  }
+  
+  @Test
+  @SuppressWarnings( "unchecked" )
+  public void testJavaPairRDDFunctions() throws Exception {
+    JavaPairRDD<String, Integer> mockPairRDD = mock(JavaPairRDD.class);
+    RDD<Tuple2<String, Integer>> mockTuple2RDD = mock(RDD.class);
+    when(mockPairRDD.rdd()).thenReturn(mockTuple2RDD);
+    GemFireJavaPairRDDFunctions wrapper = javaFunctions(mockPairRDD);
+    assertTrue(mockTuple2RDD == wrapper.rddf.rdd());
+
+    Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> tuple3 = createCommonMocks();
+    when(mockTuple2RDD.sparkContext()).thenReturn(tuple3._1());
+    String regionPath = "testregion";
+    wrapper.saveToGemfire(regionPath, tuple3._2());
+    verify(mockTuple2RDD, times(1)).sparkContext();
+    verify(tuple3._1(), times(1)).runJob(eq(mockTuple2RDD), any(Function2.class), any(ClassTag.class));
+  }
+
+  @Test
+  @SuppressWarnings( "unchecked" )
+  public void testJavaRDDFunctions() throws Exception {
+    JavaRDD<String> mockJavaRDD = mock(JavaRDD.class);
+    RDD<String> mockRDD = mock(RDD.class);
+    when(mockJavaRDD.rdd()).thenReturn(mockRDD);
+    GemFireJavaRDDFunctions wrapper = javaFunctions(mockJavaRDD);
+    assertTrue(mockRDD == wrapper.rddf.rdd());
+
+    Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> tuple3 = createCommonMocks();
+    when(mockRDD.sparkContext()).thenReturn(tuple3._1());
+    PairFunction<String, String, Integer> mockPairFunc = mock(PairFunction.class);
+    String regionPath = "testregion";
+    wrapper.saveToGemfire(regionPath, mockPairFunc, tuple3._2());
+    verify(mockRDD, times(1)).sparkContext();
+    verify(tuple3._1(), times(1)).runJob(eq(mockRDD), any(Function2.class), any(ClassTag.class));
+  }
+
+  @Test
+  @SuppressWarnings( "unchecked" )
+  public void testJavaPairDStreamFunctions() throws Exception {
+    JavaPairDStream<String, String> mockJavaDStream = mock(JavaPairDStream.class);
+    DStream<Tuple2<String, String>> mockDStream = mock(DStream.class);
+    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
+    GemFireJavaPairDStreamFunctions wrapper = javaFunctions(mockJavaDStream);
+    assertTrue(mockDStream == wrapper.dsf.dstream());
+
+    Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> tuple3 = createCommonMocks();
+    String regionPath = "testregion";
+    wrapper.saveToGemfire(regionPath, tuple3._2());
+    verify(tuple3._2()).getConnection();
+    verify(tuple3._3()).validateRegion(regionPath);
+    verify(mockDStream).foreachRDD(any(Function1.class));
+  }
+
+  @Test
+  @SuppressWarnings( "unchecked" )
+  public void testJavaPairDStreamFunctionsWithTuple2DStream() throws Exception {
+    JavaDStream<Tuple2<String, String>> mockJavaDStream = mock(JavaDStream.class);
+    DStream<Tuple2<String, String>> mockDStream = mock(DStream.class);
+    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
+    GemFireJavaPairDStreamFunctions wrapper = javaFunctions(toJavaPairDStream(mockJavaDStream));
+    assertTrue(mockDStream == wrapper.dsf.dstream());
+  }
+
+  @Test
+  @SuppressWarnings( "unchecked" )
+  public void testJavaDStreamFunctions() throws Exception {
+    JavaDStream<String> mockJavaDStream = mock(JavaDStream.class);
+    DStream<String> mockDStream = mock(DStream.class);
+    when(mockJavaDStream.dstream()).thenReturn(mockDStream);
+    GemFireJavaDStreamFunctions wrapper = javaFunctions(mockJavaDStream);
+    assertTrue(mockDStream == wrapper.dsf.dstream());
+
+    Tuple3<SparkContext, GemFireConnectionConf, GemFireConnection> tuple3 = createCommonMocks();
+    PairFunction<String, String, Integer> mockPairFunc = mock(PairFunction.class);
+    String regionPath = "testregion";
+    wrapper.saveToGemfire(regionPath, mockPairFunc, tuple3._2());
+    verify(tuple3._2()).getConnection();
+    verify(tuple3._3()).validateRegion(regionPath);
+    verify(mockDStream).foreachRDD(any(Function1.class));
+  }
+
+  @Test
+  public void testSQLContextFunction() throws Exception {
+    SQLContext mockSQLContext = mock(SQLContext.class);
+    GemFireJavaSQLContextFunctions wrapper = javaFunctions(mockSQLContext);
+    assertTrue(wrapper.scf.getClass() == GemFireSQLContextFunctions.class);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployerTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployerTest.scala
new file mode 100644
index 0000000..854fd8f
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/GemFireFunctionDeployerTest.scala
@@ -0,0 +1,58 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector
+
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{FunSuite, Matchers}
+import org.apache.commons.httpclient.HttpClient
+import java.io.File
+
+
+class GemFireFunctionDeployerTest extends FunSuite with Matchers with MockitoSugar {
+  val mockHttpClient: HttpClient = mock[HttpClient]
+    
+  test("jmx url creation") {
+    val jmxHostAndPort = "localhost:7070"
+    val expectedUrlString = "http://" + jmxHostAndPort + "/gemfire/v1/deployed"
+    val gfd = new GemFireFunctionDeployer(mockHttpClient);
+    val urlString = gfd.constructURLString(jmxHostAndPort)
+    assert(urlString === expectedUrlString)
+  }
+    
+  test("missing jar file") {
+    val missingJarFileLocation = "file:///somemissingjarfilethatdoesnot.exist"
+    val gfd = new GemFireFunctionDeployer(mockHttpClient);
+    intercept[RuntimeException] { gfd.jarFileHandle(missingJarFileLocation)}
+  }
+  
+  test("deploy with missing jar") {
+    val missingJarFileLocation = "file:///somemissingjarfilethatdoesnot.exist"
+    val gfd = new GemFireFunctionDeployer(mockHttpClient);
+    intercept[RuntimeException] {(gfd.deploy("localhost:7070", missingJarFileLocation).contains("Deployed"))}
+    intercept[RuntimeException] {(gfd.deploy("localhost", 7070, missingJarFileLocation).contains("Deployed"))}
+  }
+  
+  test("successful mocked deploy") {
+    val gfd = new GemFireFunctionDeployer(mockHttpClient);
+    val jar = new File("README.md");
+    assert(gfd.deploy("localhost:7070", jar).contains("Deployed"))
+  }
+  
+
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
new file mode 100644
index 0000000..0ce9808
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnectionManagerTest.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.pivotal.gemfire.spark.connector.internal
+
+import io.pivotal.gemfire.spark.connector.GemFireConnectionConf
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{FunSuite, Matchers}
+
+class DefaultGemFireConnectionManagerTest extends FunSuite  with Matchers with MockitoSugar {
+
+  test("DefaultGemFireConnectionFactory get/closeConnection") {
+    // note: connConf 1-4 share the same set of locators
+    val connConf1 = new GemFireConnectionConf(Seq(("host1", 1234)))
+    val connConf2 = new GemFireConnectionConf(Seq(("host2", 5678)))
+    val connConf3 = new GemFireConnectionConf(Seq(("host1", 1234), ("host2", 5678)))
+    val connConf4 = new GemFireConnectionConf(Seq(("host2", 5678), ("host1", 1234)))
+    val connConf5 = new GemFireConnectionConf(Seq(("host5", 3333)))
+
+    val props: Map[String, String] = Map.empty
+    val mockConnFactory: DefaultGemFireConnectionFactory = mock[DefaultGemFireConnectionFactory]
+    val mockConn1 = mock[DefaultGemFireConnection]
+    val mockConn2 = mock[DefaultGemFireConnection]
+    when(mockConnFactory.newConnection(connConf3.locators, props)).thenReturn(mockConn1)
+    when(mockConnFactory.newConnection(connConf5.locators, props)).thenReturn(mockConn2)
+
+    assert(DefaultGemFireConnectionManager.getConnection(connConf3)(mockConnFactory) == mockConn1)
+    // note: following 3 lines do not trigger connFactory.newConnection(...)
+    assert(DefaultGemFireConnectionManager.getConnection(connConf1)(mockConnFactory) == mockConn1)
+    assert(DefaultGemFireConnectionManager.getConnection(connConf2)(mockConnFactory) == mockConn1)
+    assert(DefaultGemFireConnectionManager.getConnection(connConf4)(mockConnFactory) == mockConn1)
+    assert(DefaultGemFireConnectionManager.getConnection(connConf5)(mockConnFactory) == mockConn2)
+
+    // connFactory.newConnection(...) were invoked only twice
+    verify(mockConnFactory, times(1)).newConnection(connConf3.locators, props)
+    verify(mockConnFactory, times(1)).newConnection(connConf5.locators, props)
+    assert(DefaultGemFireConnectionManager.connections.size == 3)
+
+    DefaultGemFireConnectionManager.closeConnection(connConf1)
+    assert(DefaultGemFireConnectionManager.connections.size == 1)
+    DefaultGemFireConnectionManager.closeConnection(connConf5)
+    assert(DefaultGemFireConnectionManager.connections.isEmpty)
+  }
+  
+  test("DefaultGemFireConnectionFactory newConnection(...) throws RuntimeException") {
+    val connConf1 = new GemFireConnectionConf(Seq(("host1", 1234)))
+    val props: Map[String, String] = Map.empty
+    val mockConnFactory: DefaultGemFireConnectionFactory = mock[DefaultGemFireConnectionFactory]
+    when(mockConnFactory.newConnection(connConf1.locators, props)).thenThrow(new RuntimeException())
+    intercept[RuntimeException] { DefaultGemFireConnectionManager.getConnection(connConf1)(mockConnFactory) }
+    verify(mockConnFactory, times(1)).newConnection(connConf1.locators, props)
+  }
+
+  test("DefaultGemFireConnectionFactory close() w/ non-exist connection") {
+    val props: Map[String, String] = Map.empty
+    val mockConnFactory: DefaultGemFireConnectionFactory = mock[DefaultGemFireConnectionFactory]
+    val connConf1 = new GemFireConnectionConf(Seq(("host1", 1234)))
+    val connConf2 = new GemFireConnectionConf(Seq(("host2", 5678)))
+    val mockConn1 = mock[DefaultGemFireConnection]
+    when(mockConnFactory.newConnection(connConf1.locators, props)).thenReturn(mockConn1)
+    assert(DefaultGemFireConnectionManager.getConnection(connConf1)(mockConnFactory) == mockConn1)
+    assert(DefaultGemFireConnectionManager.connections.size == 1)
+    // connection does not exists in the connection manager
+    DefaultGemFireConnectionManager.closeConnection(connConf2)
+    assert(DefaultGemFireConnectionManager.connections.size == 1)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
new file mode 100644
index 0000000..ad2b94e
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/gemfirefunctions/StructStreamingResultSenderAndCollectorTest.scala
@@ -0,0 +1,254 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.gemfirefunctions
+
+import com.gemstone.gemfire.DataSerializer
+import com.gemstone.gemfire.cache.execute.{ResultCollector, ResultSender}
+import com.gemstone.gemfire.cache.query.internal.types.{ObjectTypeImpl, StructTypeImpl}
+import com.gemstone.gemfire.cache.query.types.ObjectType
+import com.gemstone.gemfire.internal.{Version, ByteArrayDataInput, HeapDataOutputStream}
+import com.gemstone.gemfire.internal.cache.{CachedDeserializable, CachedDeserializableFactory}
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import scala.collection.JavaConversions._
+import scala.concurrent.{Await, ExecutionContext, Future}
+import ExecutionContext.Implicits.global
+import scala.concurrent.duration._
+
+class StructStreamingResultSenderAndCollectorTest extends FunSuite with BeforeAndAfter  {
+
+  /** 
+    * A test ResultSender that connects struct ResultSender and ResultCollector 
+    * Note: this test ResultSender has to copy the data (byte array) since the
+    *       StructStreamingResultSender will reuse the byte array.
+    */
+  class LocalResultSender(collector: ResultCollector[Array[Byte], _], num: Int = 1) extends ResultSender[Object] {
+
+    var finishedNum = 0
+    
+    override def sendResult(result: Object): Unit =
+      collector.addResult(null, result.asInstanceOf[Array[Byte]].clone())
+
+    /** exception should be sent via lastResult() */
+    override def sendException(throwable: Throwable): Unit = 
+      throw new UnsupportedOperationException("sendException is not supported.")
+
+    override def lastResult(result: Object): Unit = {
+      collector.addResult(null, result.asInstanceOf[Array[Byte]].clone())
+      this.synchronized {
+        finishedNum += 1
+        if (finishedNum == num)
+          collector.endResults()
+      }
+    }
+  }
+
+  /** common variables */
+  var collector: StructStreamingResultCollector = _
+  var baseSender: LocalResultSender = _
+  /** common types */
+  val objType = new ObjectTypeImpl("java.lang.Object").asInstanceOf[ObjectType]
+  val TwoColType = new StructTypeImpl(Array("key", "value"), Array(objType, objType))
+  val OneColType = new StructTypeImpl(Array("value"), Array(objType))
+
+  before {
+    collector = new StructStreamingResultCollector
+    baseSender = new LocalResultSender(collector, 1)
+  }
+  
+  test("transfer simple data") {
+    verifySimpleTransfer(sendDataType = true)
+  }
+
+  test("transfer simple data with no type info") {
+    verifySimpleTransfer(sendDataType = false)
+  }
+
+  def verifySimpleTransfer(sendDataType: Boolean): Unit = {
+    val iter = (0 to 9).map(i => Array(i.asInstanceOf[Object], (i.toString * 5).asInstanceOf[Object])).toIterator
+    val dataType = if (sendDataType) TwoColType else null
+    new StructStreamingResultSender(baseSender, dataType , iter).send()
+    // println("type: " + collector.getResultType.toString)
+    assert(TwoColType.equals(collector.getResultType))
+    val iter2 = collector.getResult
+    (0 to 9).foreach { i =>
+      assert(iter2.hasNext)
+      val o = iter2.next()
+      assert(o.size == 2)
+      assert(o(0).asInstanceOf[Int] == i)
+      assert(o(1).asInstanceOf[String] == i.toString * 5)
+    }
+    assert(! iter2.hasNext)
+  }
+
+  
+  /**
+   * A test iterator that generate integer data
+   * @param start the 1st value
+   * @param n number of integers generated
+   * @param genExcp generate Exception if true. This is used to test exception handling.
+   */
+  def intIterator(start: Int, n: Int, genExcp: Boolean): Iterator[Array[Object]] = {
+    new Iterator[Array[Object]] {
+      val max = if (genExcp) start + n else start + n - 1
+      var index: Int = start - 1
+
+      override def hasNext: Boolean = if (index < max) true else false
+
+      override def next(): Array[Object] =
+        if (index < (start + n - 1)) {
+          index += 1
+          Array(index.asInstanceOf[Object])
+        } else throw new RuntimeException("simulated error")
+    }
+  }
+
+  test("transfer data with 0 row") {
+    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 0, genExcp = false)).send()
+    // println("type: " + collector.getResultType.toString)
+    assert(collector.getResultType == null)
+    val iter = collector.getResult
+    assert(! iter.hasNext)
+  }
+
+  test("transfer data with 10K rows") {
+    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 10000, genExcp = false)).send()
+    // println("type: " + collector.getResultType.toString)
+    assert(OneColType.equals(collector.getResultType))
+    val iter = collector.getResult
+    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
+    (1 to 10000).foreach { i =>
+      assert(iter.hasNext)
+      val o = iter.next()
+      assert(o.size == 1)
+      assert(o(0).asInstanceOf[Int] == i)
+    }
+    assert(! iter.hasNext)
+  }
+
+  test("transfer data with 10K rows with 2 sender") {
+    baseSender = new LocalResultSender(collector, 2)
+    val total = 300
+    val sender1 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(1, total/2, genExcp = false), "sender1").send()}
+    val sender2 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(total/2+1, total/2, genExcp = false), "sender2").send()}
+    Await.result(sender1, 1.seconds)
+    Await.result(sender2, 1.seconds)
+
+    // println("type: " + collector.getResultType.toString)
+    assert(OneColType.equals(collector.getResultType))
+    val iter = collector.getResult
+    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
+    val set = scala.collection.mutable.Set[Int]()
+    (1 to total).foreach { i =>
+      assert(iter.hasNext)
+      val o = iter.next()
+      assert(o.size == 1)
+      assert(! set.contains(o(0).asInstanceOf[Int]))
+      set.add(o(0).asInstanceOf[Int])
+    }
+    assert(! iter.hasNext)
+  }
+
+  test("transfer data with 10K rows with 2 sender with error") {
+    baseSender = new LocalResultSender(collector, 2)
+    val total = 1000
+    val sender1 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(1, total/2, genExcp = false), "sender1").send()}
+    val sender2 = Future { new StructStreamingResultSender(baseSender, OneColType, intIterator(total/2+1, total/2, genExcp = true), "sender2").send()}
+    Await.result(sender1, 1 seconds)
+    Await.result(sender2, 1 seconds)
+
+    // println("type: " + collector.getResultType.toString)
+    assert(OneColType.equals(collector.getResultType))
+    val iter = collector.getResult
+    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
+    val set = scala.collection.mutable.Set[Int]()
+    intercept[RuntimeException] {
+      (1 to total).foreach { i =>
+        assert(iter.hasNext)
+        val o = iter.next()
+        assert(o.size == 1)
+        assert(! set.contains(o(0).asInstanceOf[Int]))
+        set.add(o(0).asInstanceOf[Int])
+      }
+    }
+    // println(s"rows received: ${set.size}")
+  }
+
+  test("transfer data with Exception") {
+    new StructStreamingResultSender(baseSender, OneColType, intIterator(1, 200, genExcp = true)).send()
+    // println("type: " + collector.getResultType.toString)
+    val iter = collector.getResult
+    intercept[RuntimeException] ( iter.foreach(_.mkString(",")) )
+  }
+
+  def stringPairIterator(n: Int, genExcp: Boolean): Iterator[Array[Object]] =
+    intIterator(1, n, genExcp).map(x => Array(s"key-${x(0)}", s"value-${x(0)}"))
+
+  test("transfer string pair data with 200 rows") {
+    new StructStreamingResultSender(baseSender, TwoColType, stringPairIterator(1000, genExcp = false)).send()
+    // println("type: " + collector.getResultType.toString)
+    assert(TwoColType.equals(collector.getResultType))
+    val iter = collector.getResult
+    // println(iter.toList.map(list => list.mkString(",")).mkString("; "))
+    (1 to 1000).foreach { i =>
+      assert(iter.hasNext)
+      val o = iter.next()
+      assert(o.size == 2)
+      assert(o(0) == s"key-$i")
+      assert(o(1) == s"value-$i")
+    }
+    assert(! iter.hasNext)
+  }
+
+  /**
+   * Usage notes: There are 3 kinds of data to transfer:
+   * (1) object, (2) byte array of serialized object, and (3) byte array
+   * this test shows how to handle all of them.
+   */
+  test("DataSerializer usage") {
+    val outBuf = new HeapDataOutputStream(1024, null)
+    val inBuf = new ByteArrayDataInput()
+
+    // 1. a regular object
+    val hello = "Hello World!" * 30
+    // serialize the data
+    DataSerializer.writeObject(hello, outBuf)
+    val bytesHello = outBuf.toByteArray.clone()
+    // de-serialize the data
+    inBuf.initialize(bytesHello, Version.CURRENT)
+    val hello2 = DataSerializer.readObject(inBuf).asInstanceOf[Object]
+    assert(hello == hello2)
+    
+    // 2. byte array of serialized object
+    // serialize: byte array from `CachedDeserializable`
+    val cd: CachedDeserializable = CachedDeserializableFactory.create(bytesHello)
+    outBuf.reset()
+    DataSerializer.writeByteArray(cd.getSerializedValue, outBuf)
+    // de-serialize the data in 2 steps
+    inBuf.initialize(outBuf.toByteArray.clone(), Version.CURRENT)
+    val bytesHello2: Array[Byte] = DataSerializer.readByteArray(inBuf)
+    inBuf.initialize(bytesHello2, Version.CURRENT)
+    val hello3 = DataSerializer.readObject(inBuf).asInstanceOf[Object]
+    assert(hello == hello3)
+
+    // 3. byte array
+    outBuf.reset()
+    DataSerializer.writeByteArray(bytesHello, outBuf)
+    inBuf.initialize(outBuf.toByteArray.clone(), Version.CURRENT)
+    val bytesHello3: Array[Byte] = DataSerializer.readByteArray(inBuf)
+    assert(bytesHello sameElements bytesHello3)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParserTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParserTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParserTest.scala
new file mode 100644
index 0000000..e33e9e8
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/io/pivotal/gemfire/spark/connector/internal/oql/QueryParserTest.scala
@@ -0,0 +1,83 @@
+/*
+ * 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 io.pivotal.gemfire.spark.connector.internal.oql
+
+import org.scalatest.FunSuite
+
+class QueryParserTest extends FunSuite {
+
+  test("select * from /r1") {
+    val r = QueryParser.parseOQL("select * from /r1").get
+    assert(r == "List(/r1)")
+  }
+
+  test("select c2 from /r1") {
+    val r = QueryParser.parseOQL("select c2 from /r1").get
+    assert(r == "List(/r1)")
+  }
+
+  test("select key, value from /r1.entries") {
+    val r = QueryParser.parseOQL("select key, value from /r1.entries").get
+    assert(r == "List(/r1.entries)")
+  }
+
+  test("select c1, c2 from /r1 where col1 > 100 and col2 <= 120 or c3 = 2") {
+    val r = QueryParser.parseOQL("select c1, c2 from /r1 where col1 > 100 and col2 <= 120 or c3 = 2").get
+    assert(r == "List(/r1)")
+  }
+
+  test("select * from /r1/r2 where c1 >= 200") {
+    val r = QueryParser.parseOQL("select * from /r1/r2 where c1 >= 200").get
+    assert(r == "List(/r1/r2)")
+  }
+
+  test("import io.pivotal select c1, c2, c3 from /r1/r2, /r3/r4 where c1 <= 15 and c2 = 100") {
+    val r = QueryParser.parseOQL("import io.pivotal select c1, c2, c3 from /r1/r2, /r3/r4 where c1 <= 15 and c2 = 100").get
+    assert(r == "List(/r1/r2, /r3/r4)")
+  }
+
+  test("SELECT distinct f1, f2 FROM /r1/r2 WHere f = 100") {
+    val r = QueryParser.parseOQL("SELECT distinct f1, f2 FROM /r1/r2 WHere f = 100").get
+    assert(r == "List(/r1/r2)")
+  }
+
+  test("IMPORT io.pivotal.gemfire IMPORT com.mypackage SELECT key,value FROM /root/sub.entries WHERE status = 'active' ORDER BY id desc") {
+    val r = QueryParser.parseOQL("IMPORT io.pivotal.gemfire IMPORT com.mypackage SELECT key,value FROM /root/sub.entries WHERE status = 'active' ORDER BY id desc").get
+    assert(r == "List(/root/sub.entries)")
+  }
+
+  test("select distinct p.ID, p.status from /region p where p.ID > 5 order by p.status") {
+    val r = QueryParser.parseOQL("select distinct p.ID, p.status from /region p where p.ID > 5 order by p.status").get
+    assert(r == "List(/region)")
+  }
+
+  test("SELECT DISTINCT * FROM /QueryRegion1 r1,  /QueryRegion2 r2 WHERE r1.ID = r2.ID") {
+    val r = QueryParser.parseOQL("SELECT DISTINCT * FROM /QueryRegion1 r1,  /QueryRegion2 r2 WHERE r1.ID = r2.ID").get
+    assert(r == "List(/QueryRegion1, /QueryRegion2)")
+  }
+
+  test("SELECT id, \"type\", positions, status FROM /obj_obj_region WHERE status = 'active'") {
+    val r = QueryParser.parseOQL("SELECT id, \"type\", positions, status FROM /obj_obj_region WHERE status = 'active'").get
+    println("r.type=" + r.getClass.getName + " r=" + r)
+    assert(r == "List(/obj_obj_region)")
+  }
+
+  test("SELECT r.id, r.\"type\", r.positions, r.status FROM /obj_obj_region r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'") {
+    val r = QueryParser.parseOQL("SELECT r.id, r.\"type\", r.positions, r.status FROM /obj_obj_region r, r.positions.values f WHERE r.status = 'active' and f.secId = 'MSFT'").get
+    assert(r == "List(/obj_obj_region, r.positions.values)")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/ConnectorImplicitsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/ConnectorImplicitsTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/ConnectorImplicitsTest.scala
new file mode 100644
index 0000000..4032ee8
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/ConnectorImplicitsTest.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 unittest.io.pivotal.gemfire.spark.connector
+
+import io.pivotal.gemfire.spark.connector._
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.SQLContext
+import org.scalatest.FunSuite
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.Matchers
+
+class ConnectorImplicitsTest extends FunSuite with Matchers with MockitoSugar {
+
+  test("implicit map2Properties") {
+    verifyProperties(Map.empty)
+    verifyProperties(Map("One" -> "1", "Two" -> "2", "Three" ->"3"))
+  }
+  
+  def verifyProperties(map: Map[String, String]): Unit = {
+    val props: java.util.Properties = map
+    assert(props.size() == map.size)
+    map.foreach(p => assert(props.getProperty(p._1) == p._2))    
+  }
+
+  test("Test Implicit SparkContext Conversion") {
+    val mockSparkContext = mock[SparkContext]
+    val gfscf: GemFireSparkContextFunctions = mockSparkContext
+    assert(gfscf.isInstanceOf[GemFireSparkContextFunctions])
+  }
+
+  test("Test Implicit SQLContext Conversion") {
+    val mockSQLContext = mock[SQLContext]
+    val gfscf: GemFireSQLContextFunctions = mockSQLContext
+    assert(gfscf.isInstanceOf[GemFireSQLContextFunctions])
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireConnectionConfTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireConnectionConfTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireConnectionConfTest.scala
new file mode 100644
index 0000000..0e06db4
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireConnectionConfTest.scala
@@ -0,0 +1,100 @@
+/*
+ * 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 unittest.io.pivotal.gemfire.spark.connector
+
+import org.apache.spark.SparkConf
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{Matchers, FunSuite}
+import io.pivotal.gemfire.spark.connector._
+
+class GemFireConnectionConfTest extends FunSuite with Matchers with MockitoSugar {
+
+  test("apply(SparkConf) w/ GemFireLocator property and empty gemfireProps") {
+    val (host1, port1) = ("host1", 1234)
+    val (host2, port2) = ("host2", 5678)
+    val conf = new SparkConf().set(GemFireLocatorPropKey, s"$host1[$port1],$host2[$port2]")
+    val connConf = GemFireConnectionConf(conf)
+    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
+    assert(connConf.gemfireProps.isEmpty)
+  }
+  
+  test("apply(SparkConf) w/ GemFireLocator property and gemfire properties") {
+    val (host1, port1) = ("host1", 1234)
+    val (host2, port2) = ("host2", 5678)
+    val (propK1, propV1) = ("ack-severe-alert-threshold", "1")
+    val (propK2, propV2) = ("ack-wait-threshold", "10")
+    val conf = new SparkConf().set(GemFireLocatorPropKey, s"$host1[$port1],$host2[$port2]")
+                              .set(s"spark.gemfire.$propK1", propV1).set(s"spark.gemfire.$propK2", propV2)
+    val connConf = GemFireConnectionConf(conf)
+    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
+    assert(connConf.gemfireProps == Map(propK1 -> propV1, propK2 -> propV2))
+  }
+
+  test("apply(SparkConf) w/o GemFireLocator property") {
+    intercept[RuntimeException] { GemFireConnectionConf(new SparkConf()) }
+  }
+
+  test("apply(SparkConf) w/ invalid GemFireLocator property") {
+    val conf = new SparkConf().set(GemFireLocatorPropKey, "local^host:1234")
+    intercept[Exception] { GemFireConnectionConf(conf) }
+  }
+
+  test("apply(locatorStr, gemfireProps) w/ valid locatorStr and non gemfireProps") {
+    val (host1, port1) = ("host1", 1234)
+    val connConf = GemFireConnectionConf(s"$host1:$port1")
+    assert(connConf.locators == Seq((host1, port1)))
+    assert(connConf.gemfireProps.isEmpty)
+  }
+
+  test("apply(locatorStr, gemfireProps) w/ valid locatorStr and non-empty gemfireProps") {
+    val (host1, port1) = ("host1", 1234)
+    val (host2, port2) = ("host2", 5678)
+    val (propK1, propV1) = ("ack-severe-alert-threshold", "1")
+    val (propK2, propV2) = ("ack-wait-threshold", "10")
+    val props = Map(propK1 -> propV1, propK2 -> propV2)
+    val connConf = GemFireConnectionConf(s"$host1:$port1,$host2:$port2", props)
+    assert(connConf.locators == Seq((host1, port1),(host2, port2)))
+    assert(connConf.gemfireProps == props)
+  }
+
+  test("apply(locatorStr, gemfireProps) w/ invalid locatorStr") {
+    intercept[Exception] { GemFireConnectionConf("local~host:4321") }
+  }
+
+  test("constructor w/ empty (host,port) pairs") {
+    intercept[IllegalArgumentException] { new GemFireConnectionConf(Seq.empty) }
+  }
+
+  test("getConnection() normal") {
+    implicit val mockFactory = mock[GemFireConnectionManager]
+    val mockConnection = mock[GemFireConnection]
+    when(mockFactory.getConnection(org.mockito.Matchers.any[GemFireConnectionConf])).thenReturn(mockConnection)
+    val connConf = GemFireConnectionConf("localhost:1234")
+    assert(connConf.getConnection == mockConnection)
+    verify(mockFactory).getConnection(connConf)
+  }
+
+  test("getConnection() failure") {
+    implicit val mockFactory = mock[GemFireConnectionManager]
+    when(mockFactory.getConnection(org.mockito.Matchers.any[GemFireConnectionConf])).thenThrow(new RuntimeException)
+    val connConf = GemFireConnectionConf("localhost:1234")
+    intercept[RuntimeException] { connConf.getConnection }
+    verify(mockFactory).getConnection(connConf)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireDStreamFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireDStreamFunctionsTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireDStreamFunctionsTest.scala
new file mode 100644
index 0000000..4117596
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireDStreamFunctionsTest.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 unittest.io.pivotal.gemfire.spark.connector
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector.{GemFireConnection, GemFireConnectionConf}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.streaming.dstream.DStream
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{Matchers, FunSuite}
+import org.mockito.Matchers.{eq => mockEq, any => mockAny}
+
+import scala.reflect.ClassTag
+
+class GemFireDStreamFunctionsTest extends FunSuite with Matchers with MockitoSugar {
+
+  test("test GemFirePairDStreamFunctions Implicit") {
+    import io.pivotal.gemfire.spark.connector.streaming._
+    val mockDStream = mock[DStream[(Int, String)]]
+    // the implicit make the following line valid
+    val pairDStream: GemFirePairDStreamFunctions[Int, String] = mockDStream
+    pairDStream shouldBe a[GemFirePairDStreamFunctions[_, _]]
+  }
+
+  test("test GemFireDStreamFunctions Implicit") {
+    import io.pivotal.gemfire.spark.connector.streaming._
+    val mockDStream = mock[DStream[String]]
+    // the implicit make the following line valid
+    val dstream: GemFireDStreamFunctions[String] = mockDStream
+    dstream shouldBe a[GemFireDStreamFunctions[_]]
+  }
+
+  def createMocks[K, V](regionPath: String)
+    (implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]])
+    : (String, GemFireConnectionConf, GemFireConnection, Region[K, V]) = {
+    val mockConnection = mock[GemFireConnection]
+    val mockConnConf = mock[GemFireConnectionConf]
+    val mockRegion = mock[Region[K, V]]
+    when(mockConnConf.getConnection).thenReturn(mockConnection)
+    when(mockConnConf.locators).thenReturn(Seq.empty)
+    (regionPath, mockConnConf, mockConnection, mockRegion)
+  }
+
+  test("test GemFirePairDStreamFunctions.saveToGemfire()") {
+    import io.pivotal.gemfire.spark.connector.streaming._
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
+    val mockDStream = mock[DStream[(String, String)]]
+    mockDStream.saveToGemfire(regionPath, mockConnConf)
+    verify(mockConnConf).getConnection
+    verify(mockConnection).validateRegion[String, String](regionPath)
+    verify(mockDStream).foreachRDD(mockAny[(RDD[(String, String)]) => Unit])
+  }
+
+  test("test GemFireDStreamFunctions.saveToGemfire()") {
+    import io.pivotal.gemfire.spark.connector.streaming._
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, Int]("test")
+    val mockDStream = mock[DStream[String]]
+    mockDStream.saveToGemfire[String, Int](regionPath,  (s: String) => (s, s.length), mockConnConf)
+    verify(mockConnConf).getConnection
+    verify(mockConnection).validateRegion[String, String](regionPath)
+    verify(mockDStream).foreachRDD(mockAny[(RDD[String]) => Unit])
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireRDDFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireRDDFunctionsTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireRDDFunctionsTest.scala
new file mode 100644
index 0000000..f2d49cb
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/GemFireRDDFunctionsTest.scala
@@ -0,0 +1,139 @@
+/*
+ * 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 unittest.io.pivotal.gemfire.spark.connector
+
+import com.gemstone.gemfire.cache.Region
+import io.pivotal.gemfire.spark.connector._
+import io.pivotal.gemfire.spark.connector.internal.rdd.{GemFireRDDWriter, GemFirePairRDDWriter}
+import org.apache.spark.{TaskContext, SparkContext}
+import org.apache.spark.rdd.RDD
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{FunSuite, Matchers}
+import collection.JavaConversions._
+import scala.reflect.ClassTag
+import org.mockito.Matchers.{eq => mockEq, any => mockAny}
+
+class GemFireRDDFunctionsTest extends FunSuite with Matchers with MockitoSugar {
+
+  test("test PairRDDFunction Implicit") {
+    import io.pivotal.gemfire.spark.connector._
+    val mockRDD = mock[RDD[(Int, String)]]
+    // the implicit make the following line valid
+    val pairRDD: GemFirePairRDDFunctions[Int, String] = mockRDD
+    pairRDD shouldBe a [GemFirePairRDDFunctions[_, _]]
+  }
+  
+  test("test RDDFunction Implicit") {
+    import io.pivotal.gemfire.spark.connector._
+    val mockRDD = mock[RDD[String]]
+    // the implicit make the following line valid
+    val nonPairRDD: GemFireRDDFunctions[String] = mockRDD
+    nonPairRDD shouldBe a [GemFireRDDFunctions[_]]
+  }
+
+  def createMocks[K, V](regionPath: String)
+    (implicit kt: ClassTag[K], vt: ClassTag[V], m: Manifest[Region[K, V]]): (String, GemFireConnectionConf, GemFireConnection, Region[K, V]) = {
+    val mockConnection = mock[GemFireConnection]
+    val mockConnConf = mock[GemFireConnectionConf]
+    val mockRegion = mock[Region[K, V]]
+    when(mockConnConf.getConnection).thenReturn(mockConnection)
+    when(mockConnection.getRegionProxy[K, V](regionPath)).thenReturn(mockRegion)
+    // mockRegion shouldEqual mockConn.getRegionProxy[K, V](regionPath)
+    (regionPath, mockConnConf, mockConnection, mockRegion)
+  }
+
+  test("test GemFirePairRDDWriter") {
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
+    val writer = new GemFirePairRDDWriter[String, String](regionPath, mockConnConf)
+    val data = List(("1", "one"), ("2", "two"), ("3", "three"))
+    writer.write(null, data.toIterator)
+    val expectedMap: Map[String, String] = data.toMap
+    verify(mockRegion).putAll(expectedMap)
+  }
+
+  test("test GemFireNonPairRDDWriter") {
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[Int, String]("test")
+    val writer = new GemFireRDDWriter[String, Int, String](regionPath, mockConnConf)
+    val data = List("a", "ab", "abc")
+    val f: String => (Int, String) = s => (s.length, s)
+    writer.write(f)(null, data.toIterator)
+    val expectedMap: Map[Int, String] = data.map(f).toMap
+    verify(mockRegion).putAll(expectedMap)
+  }
+  
+  test("test PairRDDFunctions.saveToGemfire") {
+    verifyPairRDDFunction(useOpConf = false)
+  }
+
+  test("test PairRDDFunctions.saveToGemfire w/ opConf") {
+    verifyPairRDDFunction(useOpConf = true)
+  }
+  
+  def verifyPairRDDFunction(useOpConf: Boolean): Unit = {
+    import io.pivotal.gemfire.spark.connector._
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[String, String]("test")
+    val mockRDD = mock[RDD[(String, String)]]
+    val mockSparkContext = mock[SparkContext]
+    when(mockRDD.sparkContext).thenReturn(mockSparkContext)
+    val result = 
+      if (useOpConf) 
+        mockRDD.saveToGemfire(regionPath, mockConnConf, Map(RDDSaveBatchSizePropKey -> "5000"))
+      else
+        mockRDD.saveToGemfire(regionPath, mockConnConf)
+    verify(mockConnection, times(1)).validateRegion[String, String](regionPath)
+    result === Unit
+    verify(mockSparkContext, times(1)).runJob[(String, String), Unit](
+      mockEq(mockRDD), mockAny[(TaskContext, Iterator[(String, String)]) => Unit])(mockAny(classOf[ClassTag[Unit]]))
+
+    // Note: current implementation make following code not compilable
+    //       so not negative test for this case
+    //  val rdd: RDD[(K, V)] = ...
+    //  rdd.saveToGemfire(regionPath, s => (s.length, s))
+  }
+
+  test("test RDDFunctions.saveToGemfire") {
+    verifyRDDFunction(useOpConf = false)
+  }
+
+  test("test RDDFunctions.saveToGemfire w/ opConf") {
+    verifyRDDFunction(useOpConf = true)
+  }
+  
+  def verifyRDDFunction(useOpConf: Boolean): Unit = {
+    import io.pivotal.gemfire.spark.connector._
+    val (regionPath, mockConnConf, mockConnection, mockRegion) = createMocks[Int, String]("test")
+    val mockRDD = mock[RDD[(String)]]
+    val mockSparkContext = mock[SparkContext]
+    when(mockRDD.sparkContext).thenReturn(mockSparkContext)
+    val result = 
+      if (useOpConf)
+        mockRDD.saveToGemfire(regionPath, s => (s.length, s), mockConnConf, Map(RDDSaveBatchSizePropKey -> "5000"))
+      else
+        mockRDD.saveToGemfire(regionPath, s => (s.length, s), mockConnConf)
+    verify(mockConnection, times(1)).validateRegion[Int, String](regionPath)
+    result === Unit
+    verify(mockSparkContext, times(1)).runJob[String, Unit](
+      mockEq(mockRDD), mockAny[(TaskContext, Iterator[String]) => Unit])(mockAny(classOf[ClassTag[Unit]]))
+
+    // Note: current implementation make following code not compilable
+    //       so not negative test for this case
+    //  val rdd: RDD[T] = ...   // T is not a (K, V) tuple
+    //  rdd.saveToGemfire(regionPath)
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
new file mode 100644
index 0000000..bfb115a
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
@@ -0,0 +1,168 @@
+/*
+ * 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 unittest.io.pivotal.gemfire.spark.connector
+
+import java.net.InetAddress
+
+import io.pivotal.gemfire.spark.connector.internal.LocatorHelper
+import org.scalatest.FunSuite
+
+class LocatorHelperTest extends FunSuite {
+
+  test("locatorStr2HostPortPair hostname w/o domain") {
+    val (host, port) = ("localhost", 10334)
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:$port").get ==(host, port))
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host[$port]").get ==(host, port))
+  }
+
+  test("locatorStr2HostPortPair hostname w/ domain") {
+    val (host, port) = ("localhost", 10334)
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:$port").get ==(host, port))
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host[$port]").get ==(host, port))
+  }
+
+  test("locatorStr2HostPortPair w/ invalid host name") {
+    // empty or null locatorStr
+    assert(LocatorHelper.locatorStr2HostPortPair("").isFailure)
+    assert(LocatorHelper.locatorStr2HostPortPair(null).isFailure)
+    // host name has leading `.`
+    assert(LocatorHelper.locatorStr2HostPortPair(".localhost.1234").isFailure)
+    // host name has leading and/or tail white space
+    assert(LocatorHelper.locatorStr2HostPortPair(" localhost.1234").isFailure)
+    assert(LocatorHelper.locatorStr2HostPortPair("localhost .1234").isFailure)
+    assert(LocatorHelper.locatorStr2HostPortPair(" localhost .1234").isFailure)
+    // host name contain invalid characters
+    assert(LocatorHelper.locatorStr2HostPortPair("local%host.1234").isFailure)
+    assert(LocatorHelper.locatorStr2HostPortPair("localhost*.1234").isFailure)
+    assert(LocatorHelper.locatorStr2HostPortPair("^localhost.1234").isFailure)
+  }
+
+  test("locatorStr2HostPortPair w/ valid port") {
+    val host = "192.168.0.1"
+    // port has 2, 3, 4, 5 digits
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:20").get ==(host, 20))
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:300").get ==(host, 300))
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:4000").get ==(host, 4000))
+    assert(LocatorHelper.locatorStr2HostPortPair(s"$host:50000").get ==(host, 50000))
+  }
+  
+  test("locatorStr2HostPortPair w/ invalid port") {
+    // port number is less than 2 digits
+    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.9").isFailure)
+    // port number is more than 5 digits
+    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.100000").isFailure)
+    // port number is invalid
+    assert(LocatorHelper.locatorStr2HostPortPair("locslhost.1xx1").isFailure)
+  }
+  
+  test("parseLocatorsString with valid locator(s)") {
+    val (host1, port1) = ("localhost", 10334)
+    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1") == Seq((host1, port1)))
+    val (host2, port2) = ("localhost2", 10335)
+    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1,$host2:$port2") == Seq((host1, port1),(host2, port2)))
+    val (host3, port3) = ("localhost2", 10336)
+    assert(LocatorHelper.parseLocatorsString(s"$host1:$port1,$host2:$port2,$host3:$port3") == 
+      Seq((host1, port1),(host2, port2),(host3, port3)))
+  }
+
+  test("parseLocatorsString with invalid locator(s)") {
+    // empty and null locatorsStr
+    intercept[Exception] { LocatorHelper.parseLocatorsString("") }
+    intercept[Exception] { LocatorHelper.parseLocatorsString(null) }
+    // 1 bad locatorStr
+    intercept[Exception] { LocatorHelper.parseLocatorsString("local%host.1234") }
+    // 1 good locatorStr and 1 bad locatorStr
+    intercept[Exception] { LocatorHelper.parseLocatorsString("localhost:2345,local%host.1234") }
+    intercept[Exception] { LocatorHelper.parseLocatorsString("local^host:2345,localhost.1234") }
+  }
+
+  test("pickPreferredGemFireServers: shared servers and one gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: shared servers, one gf-server per host, un-sorted list") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv4, srv2, srv3, srv1)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: shared servers and two gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
+  }
+
+  test("pickPreferredGemFireServers: shared servers, two gf-server per host, un-sorted server list") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
+    val servers = Seq(srv1, srv4, srv3, srv2)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
+  }
+
+  test("pickPreferredGemFireServers: no shared servers and one gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host5", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host6", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host7", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host8", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: no shared servers, one gf-server per host, and less gf-server") {
+    val (srv1, srv2) = (("host1", 4001), ("host2", 4002))
+    val servers = Seq(srv1, srv2)
+    verifyPickPreferredGemFireServers(servers, "host5", "<driver>", Seq(srv1, srv2))
+    verifyPickPreferredGemFireServers(servers, "host6", "0", Seq(srv2, srv1))
+    verifyPickPreferredGemFireServers(servers, "host7", "1", Seq(srv1, srv2))
+    verifyPickPreferredGemFireServers(servers, "host8", "2", Seq(srv2, srv1))
+
+
+    println("host name: " + InetAddress.getLocalHost.getHostName)
+    println("canonical host name: " + InetAddress.getLocalHost.getCanonicalHostName)
+    println("canonical host name 2: " + InetAddress.getByName(InetAddress.getLocalHost.getHostName).getCanonicalHostName)
+  }
+
+  test("pickPreferredGemFireServers: ad-hoc") {
+    val (srv4, srv5, srv6) = (
+      ("w2-gst-pnq-04.gemstone.com", 40411), ("w2-gst-pnq-05.gemstone.com", 40411), ("w2-gst-pnq-06.gemstone.com", 40411))
+    val servers = Seq(srv6, srv5, srv4)
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-03.gemstone.com", "<driver>", Seq(srv4, srv5, srv6))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-04.gemstone.com", "1", Seq(srv4, srv5, srv6))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-05.gemstone.com", "0", Seq(srv5, srv6, srv4))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-06.gemstone.com", "2", Seq(srv6, srv4, srv5))
+  }
+  
+  def verifyPickPreferredGemFireServers(
+    servers: Seq[(String, Int)], hostName: String, executorId: String, expectation: Seq[(String, Int)]): Unit = {
+    val result = LocatorHelper.pickPreferredGemFireServers(servers, hostName, executorId)
+    assert(result == expectation, s"pick servers for $hostName:$executorId")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ddee87fe/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRDDPartitionerTest.scala
----------------------------------------------------------------------
diff --git a/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRDDPartitionerTest.scala b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRDDPartitionerTest.scala
new file mode 100644
index 0000000..f6a30c7
--- /dev/null
+++ b/geode-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/rdd/GemFireRDDPartitionerTest.scala
@@ -0,0 +1,190 @@
+/*
+ * 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 unittest.io.pivotal.gemfire.spark.connector.rdd
+
+import com.gemstone.gemfire.distributed.internal.ServerLocation
+import io.pivotal.gemfire.spark.connector.internal.RegionMetadata
+import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartitioner._
+import io.pivotal.gemfire.spark.connector.GemFireConnection
+import io.pivotal.gemfire.spark.connector.internal.rdd._
+import org.apache.spark.Partition
+import org.mockito.Mockito._
+import org.scalatest.mock.MockitoSugar
+import org.scalatest.{Matchers, FunSuite}
+
+import java.util.{HashSet => JHashSet, HashMap => JHashMap}
+
+import scala.collection.mutable
+
+class GemFireRDDPartitionerTest extends FunSuite with Matchers with MockitoSugar {
+
+  val emptyServerBucketMap: JHashMap[ServerLocation, JHashSet[Integer]] = new JHashMap()
+
+  def toJavaServerBucketMap(map: Map[(String, Int), Set[Int]]): JHashMap[ServerLocation, JHashSet[Integer]] = {
+    import scala.collection.JavaConversions._
+    val tmp = map.map {case ((host, port), set) => (new ServerLocation(host, port), set.map(Integer.valueOf))}
+    (new JHashMap[ServerLocation, JHashSet[Integer]]() /: tmp) { case (acc, (s, jset)) => acc.put(s, new JHashSet(jset)); acc }
+  }
+  
+  val map: mutable.Map[(String, Int), mutable.Set[Int]] = mutable.Map(
+    ("s0",1) -> mutable.Set.empty, ("s1",2) -> mutable.Set(0), ("s2",3) -> mutable.Set(1, 2), ("s3",4) -> mutable.Set(3, 4, 5))
+
+  
+  // update this test whenever change default setting 
+  test("default partitioned region partitioner") {
+    assert(GemFireRDDPartitioner.defaultPartitionedRegionPartitioner === ServerSplitsPartitioner)
+  }
+
+  // update this test whenever change default setting 
+  test("default replicated region partitioner") {
+    assert(GemFireRDDPartitioner.defaultReplicatedRegionPartitioner === OnePartitionPartitioner)
+  }
+  
+  test("GemFireRDDPartitioner.apply method") {
+    import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartitioner._
+    for ((name, partitioner) <- partitioners) assert(GemFireRDDPartitioner(name) == partitioner)
+    assert(GemFireRDDPartitioner("dummy") == GemFireRDDPartitioner.defaultPartitionedRegionPartitioner)
+    assert(GemFireRDDPartitioner() == GemFireRDDPartitioner.defaultPartitionedRegionPartitioner)
+  }
+  
+  test("OnePartitionPartitioner") {
+    val mockConnection = mock[GemFireConnection]
+    val partitions = OnePartitionPartitioner.partitions[String, String](mockConnection, null, Map.empty)
+    verifySinglePartition(partitions)
+  }
+
+  def verifySinglePartition(partitions: Array[Partition]): Unit = {
+    assert(1 == partitions.size)
+    assert(partitions(0).index === 0)
+    assert(partitions(0).isInstanceOf[GemFireRDDPartition])
+    assert(partitions(0).asInstanceOf[GemFireRDDPartition].bucketSet.isEmpty)
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=1 & no empty bucket") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 6, 1)
+    verifyPartitions(partitions, List(
+      (Set(0, 1, 2, 3), Seq("server1")), (Set(4, 5), Seq("server2"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=1 & 1 empty bucket") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 7, 1)
+    verifyPartitions(partitions, List(
+      (Set(0, 1, 2, 3, 6), Seq("server1")), (Set(4, 5), Seq("server2"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=1 & 2 empty bucket") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 8, 1)
+    verifyPartitions(partitions, List(
+      (Set(0, 1, 2, 3, 6), Seq("server1")), (Set(4, 5, 7), Seq("server2"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=1 & 5 empty bucket") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "server1" -> mutable.Set(3,2,1,0), "server2" -> mutable.Set(5,4))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 11, 1)
+    verifyPartitions(partitions, List(
+      (Set(0, 1, 2, 3, 6, 7, 8), Seq("server1")), (Set(4, 5, 9, 10), Seq("server2"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=1, 4 empty-bucket, non-continuous IDs") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "server1" -> mutable.Set(1, 3), "server2" -> mutable.Set(5,7))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 8, 1)
+    verifyPartitions(partitions, List(
+      (Set(0, 1, 2, 3), Seq("server1")), (Set(4, 5, 6, 7), Seq("server2"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=2, no empty buckets, 3 servers have 1, 2, and 3 buckets") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "s1" -> mutable.Set(0), "s2" -> mutable.Set(1, 2), "s3" -> mutable.Set(3, 4, 5))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 6, 2)
+    // partitions.foreach(println)
+    verifyPartitions(partitions, List(
+      (Set(0), Seq("s1")), (Set(1), Seq("s2")), (Set(2), Seq("s2")), (Set(3, 4), Seq("s3")), (Set(5), Seq("s3"))))
+  }
+
+  test("ServerSplitsPartitioner.doPartitions(): n=3, no empty buckets, 4 servers have 0, 2, 3, and 4 buckets") {
+    val map: List[(String, mutable.Set[Int])] = List(
+      "s0" -> mutable.Set.empty, "s1" -> mutable.Set(0, 1), "s2" -> mutable.Set(2, 3, 4), "s3" -> mutable.Set(5, 6, 7, 8))
+    val partitions = ServerSplitsPartitioner.doPartitions(map, 9, 3)
+    // partitions.foreach(println)
+    verifyPartitions(partitions, List(
+      (Set(0), Seq("s1")), (Set(1), Seq("s1")), (Set(2), Seq("s2")), (Set(3), Seq("s2")), (Set(4), Seq("s2")),
+      (Set(5, 6), Seq("s3")), (Set(7, 8), Seq("s3")) ))
+  }
+
+  test("ServerSplitsPartitioner.partitions(): metadata = None ") {
+    val regionPath = "test"
+    val mockConnection = mock[GemFireConnection]
+    intercept[RuntimeException] { ServerSplitsPartitioner.partitions[String, String](mockConnection, null, Map.empty) }
+  }
+
+  test("ServerSplitsPartitioner.partitions(): replicated region ") {
+    val regionPath = "test"
+    val mockConnection = mock[GemFireConnection]
+    val md = new RegionMetadata(regionPath, false, 11, null)
+    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
+    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map.empty)
+    verifySinglePartition(partitions)
+  }
+
+  test("ServerSplitsPartitioner.partitions(): partitioned region w/o data ") {
+    val regionPath = "test"
+    val mockConnection = mock[GemFireConnection]
+    val md = new RegionMetadata(regionPath, true, 6, emptyServerBucketMap)
+    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
+    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map.empty)
+    verifySinglePartition(partitions)
+  }
+
+  test("ServerSplitsPartitioner.partitions(): partitioned region w/ some data ") {
+    import io.pivotal.gemfire.spark.connector.NumberPartitionsPerServerPropKey
+    val regionPath = "test"
+    val mockConnection = mock[GemFireConnection]
+    val map: Map[(String, Int), Set[Int]] = Map(
+      ("s0",1) -> Set.empty, ("s1",2) -> Set(0), ("s2",3) -> Set(1, 2), ("s3",4) -> Set(3, 4, 5))
+    val md = new RegionMetadata(regionPath, true, 6, toJavaServerBucketMap(map))
+    when(mockConnection.getRegionMetadata(regionPath)).thenReturn(Some(md))
+    val partitions = ServerSplitsPartitioner.partitions[String, String](mockConnection, md, Map(NumberPartitionsPerServerPropKey->"2"))
+    // partitions.foreach(println)
+    verifyPartitions(partitions, List(
+      (Set(0), Seq("s1")), (Set(1), Seq("s2")), (Set(2), Seq("s2")), (Set(3, 4), Seq("s3")), (Set(5), Seq("s3"))))
+  }
+  
+  // Note: since the order of partitions is not pre-determined, we have to verify partition id
+  // and contents separately
+  def verifyPartitions(partitions: Array[Partition], expPartitions: List[(Set[Int], Seq[String])]): Unit = {
+    // 1. check size
+    assert(partitions.size == expPartitions.size)
+    // 2. check IDs are 0 to n-1
+    (0 until partitions.size).toList.zip(partitions).foreach { case (id, p) => assert(id == p.index) }
+
+    // 3. get all pairs of bucket set and its locations, and compare to the expected pairs
+    val list = partitions.map { e =>
+      val p = e.asInstanceOf[GemFireRDDPartition]
+      (p.bucketSet, p.locations)
+    }
+    expPartitions.foreach(e => assert(list.contains(e)))    
+  }
+
+}