ignite-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From agoncha...@apache.org
Subject [08/50] [abbrv] ignite git commit: IGNITE-2929 - Removed type parameters from IgniteContext - Fixes #740
Date Wed, 06 Jul 2016 18:23:07 GMT
IGNITE-2929 - Removed type parameters from IgniteContext - Fixes #740


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

Branch: refs/heads/ignite-3227
Commit: 5193fe70ba118253a88fc0120694190dc0c911c8
Parents: e0ac099
Author: MaBiao <mabiaocas@gmail.com>
Authored: Mon Jul 4 11:15:29 2016 -0700
Committer: Valentin Kulichenko <valentin.lulichenko@gmail.com>
Committed: Mon Jul 4 11:15:29 2016 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/spark/IgniteContext.scala |  8 ++---
 .../org/apache/ignite/spark/IgniteRDD.scala     | 10 +++---
 .../apache/ignite/spark/JavaIgniteContext.scala |  2 +-
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  6 ++--
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |  2 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |  2 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala | 38 +++++++++-----------
 7 files changed, 31 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index c63a370..04139d1 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -29,10 +29,8 @@ import org.apache.spark.{Logging, SparkContext}
  *
  * @param sparkContext Spark context.
  * @param cfgF Configuration factory.
- * @tparam K Key type.
- * @tparam V Value type.
  */
-class IgniteContext[K, V](
+class IgniteContext(
     @transient val sparkContext: SparkContext,
     cfgF: () ⇒ IgniteConfiguration,
     standalone: Boolean = true
@@ -105,7 +103,7 @@ class IgniteContext[K, V](
      * @param cacheName Cache name.
      * @return `IgniteRDD` instance.
      */
-    def fromCache(cacheName: String): IgniteRDD[K, V] = {
+    def fromCache[K, V](cacheName: String): IgniteRDD[K, V] = {
         new IgniteRDD[K, V](this, cacheName, null, false)
     }
 
@@ -116,7 +114,7 @@ class IgniteContext[K, V](
      * @param cacheCfg Cache configuration to use.
      * @return `IgniteRDD` instance.
      */
-    def fromCache(cacheCfg: CacheConfiguration[K, V]) = {
+    def fromCache[K, V](cacheCfg: CacheConfiguration[K, V]) = {
         new IgniteRDD[K, V](this, cacheCfg.getName, cacheCfg, false)
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index f198f0c..81509d0 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -43,7 +43,7 @@ import scala.collection.JavaConversions._
  * @tparam V Value type.
  */
 class IgniteRDD[K, V] (
-    val ic: IgniteContext[K, V],
+    val ic: IgniteContext,
     val cacheName: String,
     val cacheCfg: CacheConfiguration[K, V],
     val keepBinary: Boolean
@@ -188,7 +188,7 @@ class IgniteRDD[K, V] (
      * @param rdd RDD instance to save values from.
      * @param f Transformation function.
      */
-    def saveValues[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) ⇒ V) = {
+    def saveValues[T](rdd: RDD[T], f: (T, IgniteContext) ⇒ V) = {
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
@@ -252,7 +252,7 @@ class IgniteRDD[K, V] (
      * @param overwrite Boolean flag indicating whether the call on this method should overwrite
existing
      *      values in Ignite cache.
      */
-    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) ⇒ (K, V), overwrite: Boolean)
= {
+    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext) ⇒ (K, V), overwrite: Boolean) =
{
         rdd.foreachPartition(it ⇒ {
             val ig = ic.ignite()
 
@@ -282,7 +282,7 @@ class IgniteRDD[K, V] (
      * @param rdd RDD instance to save values from.
      * @param f Transformation function.
      */
-    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext[K, V]) ⇒ (K, V)): Unit = {
+    def savePairs[T](rdd: RDD[T], f: (T, IgniteContext) ⇒ (K, V)): Unit = {
         savePairs(rdd, f, overwrite = false)
     }
 
@@ -301,7 +301,7 @@ class IgniteRDD[K, V] (
      */
     def withKeepBinary[K1, V1](): IgniteRDD[K1, V1] = {
         new IgniteRDD[K1, V1](
-            ic.asInstanceOf[IgniteContext[K1, V1]],
+            ic,
             cacheName,
             cacheCfg.asInstanceOf[CacheConfiguration[K1, V1]],
             true)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
index 25184e7..689a22d 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
@@ -39,7 +39,7 @@ class JavaIgniteContext[K, V](
     standalone: Boolean = true
     ) extends Serializable {
 
-    @transient val ic: IgniteContext[K, V] = new IgniteContext[K, V](sc.sc, () => cfgF.apply(),
standalone)
+    @transient val ic: IgniteContext = new IgniteContext(sc.sc, () => cfgF.apply(), standalone)
 
     def this(sc: JavaSparkContext, cfgF: IgniteOutClosure[IgniteConfiguration]) {
         this(sc, cfgF, true)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
index 8a6c6b8..a44cb51 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
@@ -80,7 +80,7 @@ class JavaIgniteRDD[K, V](override val rdd: IgniteRDD[K, V])
 
     def saveValues(jrdd: JavaRDD[V]) = rdd.saveValues(JavaRDD.toRDD(jrdd))
 
-    def saveValues[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) ⇒ V) = rdd.saveValues(JavaRDD.toRDD(jrdd),
f)
+    def saveValues[T](jrdd: JavaRDD[T], f: (T, IgniteContext) ⇒ V) = rdd.saveValues(JavaRDD.toRDD(jrdd),
f)
 
     def savePairs(jrdd: JavaPairRDD[K, V], overwrite: Boolean) = {
         val rrdd: RDD[(K, V)] = JavaPairRDD.toRDD(jrdd)
@@ -90,11 +90,11 @@ class JavaIgniteRDD[K, V](override val rdd: IgniteRDD[K, V])
 
     def savePairs(jrdd: JavaPairRDD[K, V]) : Unit = savePairs(jrdd, overwrite = false)
 
-    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) ⇒ (K, V), overwrite:
Boolean = false) = {
+    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext) ⇒ (K, V), overwrite: Boolean
= false) = {
         rdd.savePairs(JavaRDD.toRDD(jrdd), f, overwrite)
     }
 
-    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext[K, V]) ⇒ (K, V)): Unit =
+    def savePairs[T](jrdd: JavaRDD[T], f: (T, IgniteContext) ⇒ (K, V)): Unit =
         savePairs(jrdd, f, overwrite = false)
 
     def clear(): Unit = rdd.clear()

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
index 9d5171c..fd43a33 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
@@ -25,7 +25,7 @@ import org.apache.spark.rdd.RDD
 import scala.reflect.ClassTag
 
 abstract class IgniteAbstractRDD[R:ClassTag, K, V] (
-    ic: IgniteContext[K, V],
+    ic: IgniteContext,
     cacheName: String,
     cacheCfg: CacheConfiguration[K, V],
     keepBinary: Boolean

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
index b4579aa..f074572 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
@@ -25,7 +25,7 @@ import org.apache.spark.{TaskContext, Partition}
 import scala.reflect.ClassTag
 
 class IgniteSqlRDD[R: ClassTag, T, K, V](
-    ic: IgniteContext[K, V],
+    ic: IgniteContext,
     cacheName: String,
     cacheCfg: CacheConfiguration[K, V],
     qry: Query[T],

http://git-wip-us.apache.org/repos/asf/ignite/blob/5193fe70/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
index dff82f4..51559ab 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
@@ -40,11 +40,11 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, String](sc,
+                val ic = new IgniteContext(sc,
                     () ⇒ configuration("client", client = true))
 
                 // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
-                ic.fromCache(PARTITIONED_CACHE_NAME).savePairs(sc.parallelize(0 to 10000,
2).map(i ⇒ (String.valueOf(i), "val" + i)))
+                ic.fromCache[String, String](PARTITIONED_CACHE_NAME).savePairs(sc.parallelize(0
to 10000, 2).map(i ⇒ (String.valueOf(i), "val" + i)))
 
                 // Check cache contents.
                 val ignite = Ignition.ignite("grid-0")
@@ -65,8 +65,7 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, String](sc,
-                    () ⇒ configuration("client", client = true))
+                val ic = new IgniteContext(sc, () ⇒ configuration("client", client = true))
 
                 // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
                 ic.fromCache(PARTITIONED_CACHE_NAME).savePairs(
@@ -91,8 +90,7 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, String](sc,
-                    () ⇒ configuration("client", client = true))
+                val ic = new IgniteContext(sc, () ⇒ configuration("client", client = true))
 
                 // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
                 ic.fromCache(PARTITIONED_CACHE_NAME).saveValues(
@@ -115,8 +113,7 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, String](sc,
-                    () ⇒ configuration("client", client = true))
+                val ic = new IgniteContext(sc, () ⇒ configuration("client", client = true))
 
                 // Save pairs ("0", "val0"), ("1", "val1"), ... to Ignite cache.
                 ic.fromCache(PARTITIONED_CACHE_NAME).saveValues(
@@ -147,10 +144,10 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
                     cache.put(String.valueOf(i), i)
                 }
 
-                val ic = new IgniteContext[String, Int](sc,
+                val ic = new IgniteContext(sc,
                     () ⇒ configuration("client", client = true))
 
-                val res = ic.fromCache(PARTITIONED_CACHE_NAME).map(_._2).sum()
+                val res = ic.fromCache[String, Int](PARTITIONED_CACHE_NAME).map(_._2).sum()
 
                 assert(res == (0 to num).sum)
             }
@@ -163,10 +160,10 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, Entity](sc,
+                val ic = new IgniteContext(sc,
                     () ⇒ configuration("client", client = true))
 
-                val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
+                val cache: IgniteRDD[String, Entity] = ic.fromCache[String, Entity](PARTITIONED_CACHE_NAME)
 
                 cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (String.valueOf(i),
new Entity(i, "name" + i, i * 100))))
 
@@ -188,7 +185,7 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, Entity](sc,
+                val ic = new IgniteContext(sc,
                     () ⇒ configuration("client", client = true))
 
                 val cache: IgniteRDD[String, Entity] = ic.fromCache(PARTITIONED_CACHE_NAME)
@@ -228,10 +225,10 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, String](sc,
+                val ic = new IgniteContext(sc,
                     "modules/core/src/test/config/spark/spark-config.xml")
 
-                val cache: IgniteRDD[String, String] = ic.fromCache(PARTITIONED_CACHE_NAME)
+                val cache: IgniteRDD[String, String] = ic.fromCache[String, String](PARTITIONED_CACHE_NAME)
 
                 cache.savePairs(sc.parallelize(1 to 1000, 2).map(i ⇒ (String.valueOf(i),
"val" + i)))
 
@@ -248,10 +245,10 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[Integer, WithObjectField](sc,
+                val ic = new IgniteContext(sc,
                     () ⇒ configuration("client", client = true))
 
-                val cache: IgniteRDD[Integer, WithObjectField] = ic.fromCache(PARTITIONED_CACHE_NAME)
+                val cache: IgniteRDD[Integer, WithObjectField] = ic.fromCache[Integer, WithObjectField](PARTITIONED_CACHE_NAME)
 
                 cache.savePairs(sc.parallelize(0 to 1000, 2).map(i ⇒ (i:java.lang.Integer,
new WithObjectField(i, new Entity(i, "", i)))))
 
@@ -273,8 +270,7 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[Integer, WithObjectField](sc,
-                    () ⇒ configuration("client", client = true))
+                val ic = new IgniteContext(sc, () ⇒ configuration("client", client = true))
 
                 val cache: IgniteRDD[Integer, WithObjectField] = ic.fromCache(PARTITIONED_CACHE_NAME)
 
@@ -300,9 +296,9 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll
with Be
             val sc = new SparkContext("local[*]", "test")
 
             try {
-                val ic = new IgniteContext[String, Entity](sc, () ⇒ configuration("client",
client = true))
+                val ic = new IgniteContext(sc, () ⇒ configuration("client", client = true))
 
-                val cache = ic.fromCache(PARTITIONED_CACHE_NAME)
+                val cache = ic.fromCache[String, Entity](PARTITIONED_CACHE_NAME)
 
                 cache.savePairs(sc.parallelize(0 until 10, 2).map(i ⇒ (String.valueOf(i),
                     new Entity(i, "name" + i, i * 100))))


Mime
View raw message