From 03cad516cd79fe925a6169075907e6f75bab199c Mon Sep 17 00:00:00 2001 From: Isaac Abraham Date: Thu, 21 Apr 2016 00:35:44 +0100 Subject: [PATCH 01/20] Initial port from KVP to Tuple --- .../Core/Accumulator.cs | 14 +- .../Core/OrderedRDDFunctions.cs | 18 +- .../Core/PairRDDFunctions.cs | 272 +++++++++--------- .../Microsoft.Spark.CSharp/Core/RDD.cs | 96 +++---- .../Core/SparkContext.cs | 8 +- .../Interop/Ipc/JvmBridgeUtils.cs | 13 +- .../Microsoft.Spark.CSharp/Proxy/IRDDProxy.cs | 10 +- .../Proxy/ISparkContextProxy.cs | 8 +- .../Proxy/IStreamingContextProxy.cs | 8 +- .../Proxy/Ipc/RDDIpcProxy.cs | 10 +- .../Proxy/Ipc/SparkContextIpcProxy.cs | 8 +- .../Proxy/Ipc/StreamingContextIpcProxy.cs | 20 +- .../Streaming/DStream.cs | 14 +- .../Streaming/EventHubsUtils.cs | 2 +- .../Microsoft.Spark.CSharp/Streaming/Kafka.cs | 26 +- .../Streaming/MapWithStateDStream.cs | 30 +- .../Streaming/PairDStreamFunctions.cs | 130 ++++----- .../Microsoft.Spark.CSharp.Adapter.Doc.XML | 231 ++++++++------- .../documentation/Mobius_API_Documentation.md | 22 +- csharp/AdapterTest/AccumulatorTest.cs | 8 +- csharp/AdapterTest/DStreamTest.cs | 106 +++---- csharp/AdapterTest/DoubleRDDTest.cs | 2 +- csharp/AdapterTest/EventHubsUtilsTest.cs | 4 +- csharp/AdapterTest/Mocks/MockRddProxy.cs | 10 +- .../Mocks/MockSparkContextProxy.cs | 8 +- .../Mocks/MockStreamingContextProxy.cs | 8 +- csharp/AdapterTest/PairRDDTest.cs | 64 ++--- csharp/AdapterTest/RDDTest.cs | 8 +- csharp/AdapterTest/SparkContextTest.cs | 12 +- csharp/AdapterTest/StreamingContextTest.cs | 6 +- csharp/AdapterTest/TestWithMoqDemo.cs | 6 +- .../FreebaseDeletionsBenchmark.cs | 8 +- .../Microsoft.Spark.CSharp/DStreamSamples.cs | 4 +- .../DStreamStateSample.cs | 14 +- .../Microsoft.Spark.CSharp/PairRDDSamples.cs | 203 ++++++------- .../Microsoft.Spark.CSharp/RDDSamples.cs | 74 ++--- 36 files changed, 757 insertions(+), 728 deletions(-) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/Accumulator.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/Accumulator.cs index 816002ef..b07b138a 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/Accumulator.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/Accumulator.cs @@ -215,19 +215,19 @@ internal int StartUpdateServer() for (int i = 0; i < numUpdates; i++) { var ms = new MemoryStream(SerDe.ReadBytes(ns)); - KeyValuePair update = (KeyValuePair)formatter.Deserialize(ms); + var update = (Tuple)formatter.Deserialize(ms); - if (Accumulator.accumulatorRegistry.ContainsKey(update.Key)) + if (Accumulator.accumulatorRegistry.ContainsKey(update.Item1)) { - Accumulator accumulator = Accumulator.accumulatorRegistry[update.Key]; - accumulator.GetType().GetMethod("Add").Invoke(accumulator, new object[] { update.Value }); + Accumulator accumulator = Accumulator.accumulatorRegistry[update.Item1]; + accumulator.GetType().GetMethod("Add").Invoke(accumulator, new object[] { update.Item2 }); } else { - Console.Error.WriteLine("WARN: cann't find update.Key: {0} for accumulator, will create a new one", update.Key); + Console.Error.WriteLine("WARN: cann't find update.Key: {0} for accumulator, will create a new one", update.Item1); var genericAccumulatorType = typeof(Accumulator<>); - var specificAccumulatorType = genericAccumulatorType.MakeGenericType(update.Value.GetType()); - Activator.CreateInstance(specificAccumulatorType, new object[] { update.Key, update.Value }); + var specificAccumulatorType = genericAccumulatorType.MakeGenericType(update.Item2.GetType()); + Activator.CreateInstance(specificAccumulatorType, new object[] { update.Item1, update.Item2 }); } } ns.WriteByte((byte)1); // acknowledge byte other than -1 diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs index bb514604..23509e56 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs @@ -26,7 +26,7 @@ public static class OrderedRDDFunctions /// /// /// - public static RDD> SortByKey(this RDD> self, + public static RDD> SortByKey(this RDD> self, bool ascending = true, int? numPartitions = null) { return SortByKey(self, ascending, numPartitions, new DefaultSortKeyFuncHelper().Execute); @@ -42,7 +42,7 @@ public static RDD> SortByKey(this RDDNumber of partitions. Each partition of the sorted RDD contains a sorted range of the elements. /// RDD will sort by keyFunc(key) for every key in KeyValuePair. Must not be null. /// - public static RDD> SortByKey(this RDD> self, + public static RDD> SortByKey(this RDD> self, bool ascending, int? numPartitions, Func keyFunc) { if (keyFunc == null) @@ -73,7 +73,7 @@ public static RDD> SortByKey(this RDD kv.Key).Collect().Select(k => keyFunc(k)).ToArray(); + U[] samples = self.Sample(false, fraction, 1).Map(kv => kv.Item1).Collect().Select(k => keyFunc(k)).ToArray(); Array.Sort(samples, StringComparer.Ordinal); // case sensitive if key type is string List bounds = new List(); @@ -123,22 +123,22 @@ public SortByKeyHelper(Func f, bool ascending = true) this.ascending = ascending; } - public IEnumerable> Execute(int pid, IEnumerable> kvs) + public IEnumerable> Execute(int pid, IEnumerable> kvs) { - IEnumerable> ordered; + IEnumerable> ordered; if (ascending) { if (typeof(K) == typeof(string)) - ordered = kvs.OrderBy(k => func(k.Key).ToString(), StringComparer.Ordinal); + ordered = kvs.OrderBy(k => func(k.Item1).ToString(), StringComparer.Ordinal); else - ordered = kvs.OrderBy(k => func(k.Key)); + ordered = kvs.OrderBy(k => func(k.Item1)); } else { if (typeof(K) == typeof(string)) - ordered = kvs.OrderByDescending(k => func(k.Key).ToString(), StringComparer.Ordinal); + ordered = kvs.OrderByDescending(k => func(k.Item1).ToString(), StringComparer.Ordinal); else - ordered = kvs.OrderByDescending(k => func(k.Key)); + ordered = kvs.OrderByDescending(k => func(k.Item1)); } return ordered; } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs index deb5eee0..a40c6197 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs @@ -13,7 +13,7 @@ namespace Microsoft.Spark.CSharp.Core { /// - /// operations only available to KeyValuePair RDD + /// operations only available to Tuple RDD /// /// See also http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions /// @@ -22,7 +22,7 @@ public static class PairRDDFunctions /// /// Return the key-value pairs in this RDD to the master as a dictionary. /// - /// var m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).CollectAsMap() + /// var m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).CollectAsMap() /// m[1] /// 2 /// m[3] @@ -33,30 +33,30 @@ public static class PairRDDFunctions /// /// /// - public static Dictionary CollectAsMap(this RDD> self) + public static IDictionary CollectAsMap(this RDD> self) { - return self.Collect().ToDictionary(kv => kv.Key, kv => kv.Value); + return self.Collect().ToDictionary(kv => kv.Item1, kv => kv.Item2); } /// /// Return an RDD with the keys of each tuple. /// - /// >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Keys().Collect() + /// >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Keys().Collect() /// [1, 3] /// /// /// /// /// - public static RDD Keys(this RDD> self) + public static RDD Keys(this RDD> self) { - return self.Map(kv => kv.Key); + return self.Map(kv => kv.Item1); } /// /// Return an RDD with the values of each tuple. /// - /// >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Values().Collect() + /// >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Values().Collect() /// [2, 4] /// /// @@ -64,9 +64,9 @@ public static RDD Keys(this RDD> self) /// /// /// - public static RDD Values(this RDD> self) + public static RDD Values(this RDD> self) { - return self.Map(kv => kv.Value); + return self.Map(kv => kv.Item2); } /// @@ -80,9 +80,9 @@ public static RDD Values(this RDD> self) /// /// sc.Parallelize(new[] /// { - /// new KeyValuePair<string, int>("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .ReduceByKey((x, y) => x + y).Collect() /// @@ -95,7 +95,7 @@ public static RDD Values(this RDD> self) /// /// /// - public static RDD> ReduceByKey(this RDD> self, Func reduceFunc, int numPartitions = 0) + public static RDD> ReduceByKey(this RDD> self, Func reduceFunc, int numPartitions = 0) { return CombineByKey(self, () => default(V), reduceFunc, reduceFunc, numPartitions); } @@ -109,9 +109,9 @@ public static RDD> ReduceByKey(this RDD("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .ReduceByKeyLocally((x, y) => x + y).Collect() /// @@ -123,7 +123,7 @@ public static RDD> ReduceByKey(this RDD /// /// - public static Dictionary ReduceByKeyLocally(this RDD> self, Func reduceFunc) + public static IDictionary ReduceByKeyLocally(this RDD> self, Func reduceFunc) { return ReduceByKey(self, reduceFunc).CollectAsMap(); } @@ -133,9 +133,9 @@ public static Dictionary ReduceByKeyLocally(this RDD("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .CountByKey((x, y) => x + y).Collect() /// @@ -146,7 +146,7 @@ public static Dictionary ReduceByKeyLocally(this RDD /// /// - public static Dictionary CountByKey(this RDD> self) + public static IDictionary CountByKey(this RDD> self) { return self.MapValues(v => 1L).ReduceByKey((a, b) => a + b).CollectAsMap(); } @@ -159,9 +159,9 @@ public static Dictionary CountByKey(this RDD> /// Performs a hash join across the cluster. /// /// var l = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + /// new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); /// var r = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("a", 3) }, 1); + /// new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("a", 3) }, 1); /// var m = l.Join(r, 2).Collect(); /// /// [('a', (1, 2)), ('a', (1, 3))] @@ -174,9 +174,9 @@ public static Dictionary CountByKey(this RDD> /// /// /// - public static RDD>> Join( - this RDD> self, - RDD> other, + public static RDD>> Join( + this RDD> self, + RDD> other, int numPartitions = 0) { return self.GroupWith(other, numPartitions).FlatMapValues( @@ -194,9 +194,9 @@ public static RDD>> Join( /// Hash-partitions the resulting RDD into the given number of partitions. /// /// var l = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + /// new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); /// var r = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 2) }, 1); + /// new[] { new Tuple<string, int>("a", 2) }, 1); /// var m = l.LeftOuterJoin(r).Collect(); /// /// [('a', (1, 2)), ('b', (4, Option))] @@ -209,9 +209,9 @@ public static RDD>> Join( /// /// /// - public static RDD>>> LeftOuterJoin( - this RDD> self, - RDD> other, + public static RDD>>> LeftOuterJoin( + this RDD> self, + RDD> other, int numPartitions = 0) { return self.GroupWith(other, numPartitions).FlatMapValues( @@ -228,9 +228,9 @@ public static RDD>>> LeftOuterJoin( /// Hash-partitions the resulting RDD into the given number of partitions. /// /// var l = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 2) }, 1); + /// new[] { new Tuple<string, int>("a", 2) }, 1); /// var r = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + /// new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); /// var m = l.RightOuterJoin(r).Collect(); /// /// [('a', (2, 1)), ('b', (Option, 4))] @@ -243,9 +243,9 @@ public static RDD>>> LeftOuterJoin( /// /// /// - public static RDD, W>>> RightOuterJoin( - this RDD> self, - RDD> other, + public static RDD, W>>> RightOuterJoin( + this RDD> self, + RDD> other, int numPartitions = 0) { return self.GroupWith(other, numPartitions).FlatMapValues( @@ -267,9 +267,9 @@ public static RDD, W>>> RightOuterJoin( /// Hash-partitions the resulting RDD into the given number of partitions. /// /// var l = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 1), KeyValuePair<string, int>("b", 4) }, 1); + /// new[] { new Tuple<string, int>("a", 1), Tuple<string, int>("b", 4) }, 1); /// var r = sc.Parallelize( - /// new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("c", 8) }, 1); + /// new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("c", 8) }, 1); /// var m = l.FullOuterJoin(r).Collect(); /// /// [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))] @@ -282,9 +282,9 @@ public static RDD, W>>> RightOuterJoin( /// /// /// - public static RDD, Option>>> FullOuterJoin( - this RDD> self, - RDD> other, + public static RDD, Option>>> FullOuterJoin( + this RDD> self, + RDD> other, int numPartitions = 0) { return self.GroupWith(other, numPartitions).FlatMapValues( @@ -295,13 +295,13 @@ public static RDD, Option>>> FullOuterJoin /// Return a copy of the RDD partitioned using the specified partitioner. /// - /// sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new KeyValuePair<int, int>(x, x)).PartitionBy(3).Glom().Collect() + /// sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new Tuple<int, int>(x, x)).PartitionBy(3).Glom().Collect() /// /// /// /// /// - public static RDD> PartitionBy(this RDD> self, int numPartitions = 0, + public static RDD> PartitionBy(this RDD> self, int numPartitions = 0, Func partitionFunc = null) { if (numPartitions == 0) @@ -318,7 +318,7 @@ public static RDD> PartitionBy(this RDD>(self.sparkContext.SparkContextProxy.CreatePairwiseRDD(keyed.RddProxy, numPartitions, + var rdd = new RDD>(self.sparkContext.SparkContextProxy.CreatePairwiseRDD(keyed.RddProxy, numPartitions, GenerateObjectId(partitionFunc)), self.sparkContext); rdd.partitioner = partitioner; @@ -346,9 +346,9 @@ public static RDD> PartitionBy(this RDD("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() /// @@ -363,8 +363,8 @@ public static RDD> PartitionBy(this RDD /// /// - public static RDD> CombineByKey( - this RDD> self, + public static RDD> CombineByKey( + this RDD> self, Func createCombiner, Func mergeValue, Func mergeCombiners, @@ -389,9 +389,9 @@ public static RDD> CombineByKey( /// sc.Parallelize( /// new[] /// { - /// new KeyValuePair<string, int>("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() /// @@ -406,8 +406,8 @@ public static RDD> CombineByKey( /// /// /// - public static RDD> AggregateByKey( - this RDD> self, + public static RDD> AggregateByKey( + this RDD> self, Func zeroValue, Func seqOp, Func combOp, @@ -425,9 +425,9 @@ public static RDD> AggregateByKey( /// sc.Parallelize( /// new[] /// { - /// new KeyValuePair<string, int>("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() /// @@ -440,8 +440,8 @@ public static RDD> AggregateByKey( /// /// /// - public static RDD> FoldByKey( - this RDD> self, + public static RDD> FoldByKey( + this RDD> self, Func zeroValue, Func func, int numPartitions = 0) @@ -460,9 +460,9 @@ public static RDD> FoldByKey( /// sc.Parallelize( /// new[] /// { - /// new KeyValuePair<string, int>("a", 1), - /// new KeyValuePair<string, int>("b", 1), - /// new KeyValuePair<string, int>("a", 1) + /// new Tuple<string, int>("a", 1), + /// new Tuple<string, int>("b", 1), + /// new Tuple<string, int>("a", 1) /// }, 2) /// .GroupByKey().MapValues(l => string.Join(" ", l)).Collect() /// @@ -474,7 +474,7 @@ public static RDD> FoldByKey( /// /// /// - public static RDD>> GroupByKey(this RDD> self, int numPartitions = 0) + public static RDD>> GroupByKey(this RDD> self, int numPartitions = 0) { return CombineByKey(self, () => new List(), @@ -490,8 +490,8 @@ public static RDD>> GroupByKey(this RDD("a", new[]{"apple", "banana", "lemon"}), - /// new KeyValuePair<string, string[]>("b", new[]{"grapes"}) + /// new Tuple<string, string[]>("a", new[]{"apple", "banana", "lemon"}), + /// new Tuple<string, string[]>("b", new[]{"grapes"}) /// }, 2) /// .MapValues(x => x.Length).Collect() /// @@ -504,7 +504,7 @@ public static RDD>> GroupByKey(this RDD /// /// - public static RDD> MapValues(this RDD> self, Func func) + public static RDD> MapValues(this RDD> self, Func func) { return self.Map(new MapValuesHelper(func).Execute, true); } @@ -516,8 +516,8 @@ public static RDD> MapValues(this RDD("a", new[]{"x", "y", "z"}), - /// new KeyValuePair<string, string[]>("b", new[]{"p", "r"}) + /// new Tuple<string, string[]>("a", new[]{"x", "y", "z"}), + /// new Tuple<string, string[]>("b", new[]{"p", "r"}) /// }, 2) /// .FlatMapValues(x => x).Collect() /// @@ -530,13 +530,13 @@ public static RDD> MapValues(this RDD /// /// - public static RDD> FlatMapValues(this RDD> self, Func> func) + public static RDD> FlatMapValues(this RDD> self, Func> func) { return self.FlatMap(new FlatMapValuesHelper(func).Execute, true); } /// - /// explicitly convert KeyValuePair<K, V> to KeyValuePair<K, dynamic> + /// explicitly convert Tuple<K, V> to Tuple<K, dynamic> /// since they are incompatibles types unlike V to dynamic /// /// @@ -546,10 +546,10 @@ public static RDD> FlatMapValues(this RDD /// /// - private static RDD> MapPartitionsWithIndex(this RDD> self) + private static RDD> MapPartitionsWithIndex(this RDD> self) { CSharpWorkerFunc csharpWorkerFunc = new CSharpWorkerFunc(new DynamicTypingWrapper().Execute); - var pipelinedRDD = new PipelinedRDD> + var pipelinedRDD = new PipelinedRDD> { workerFunc = csharpWorkerFunc, preservesPartitioning = true, @@ -568,8 +568,8 @@ private static RDD> MapPartitionsWithIndex, return a resulting RDD that /// contains a tuple with the list of values for that key in this RDD as well as . /// - /// var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - /// var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); + /// var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + /// var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); /// x.GroupWith(y).Collect(); /// /// [('a', ([1], [2])), ('b', ([4], []))] @@ -582,16 +582,16 @@ private static RDD> MapPartitionsWithIndex /// /// - public static RDD, List>>> GroupWith( - this RDD> self, - RDD> other, + public static RDD, List>>> GroupWith( + this RDD> self, + RDD> other, int numPartitions = 0) { // MapValues, which introduces extra CSharpRDD, is not necessary when union different RDD types if (typeof(V) != typeof(W)) { - return self.ConvertTo>() - .Union(other.ConvertTo>()) + return self.ConvertTo>() + .Union(other.ConvertTo>()) .MapPartitionsWithIndex() .CombineByKey( () => new Tuple, List>(new List(), new List()), @@ -610,9 +610,9 @@ public static RDD, List>>> GroupWith( } /// - /// var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); - /// var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - /// var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); + /// var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); + /// var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + /// var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); /// x.GroupWith(y, z).Collect(); /// /// @@ -624,18 +624,18 @@ public static RDD, List>>> GroupWith( /// /// /// - public static RDD, List, List>>> GroupWith( - this RDD> self, - RDD> other1, - RDD> other2, + public static RDD, List, List>>> GroupWith( + this RDD> self, + RDD> other1, + RDD> other2, int numPartitions = 0) { // MapValues, which introduces extra CSharpRDD, is not necessary when union different RDD types if (!(typeof(V) == typeof(W1) && typeof(V) == typeof(W2))) { - return self.ConvertTo>() - .Union(other1.ConvertTo>()) - .Union(other2.ConvertTo>()) + return self.ConvertTo>() + .Union(other1.ConvertTo>()) + .Union(other2.ConvertTo>()) .MapPartitionsWithIndex() .CombineByKey( () => new Tuple, List, List>(new List(), new List(), new List()), @@ -655,10 +655,10 @@ public static RDD, List, List>>> GroupWith } /// - /// var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); - /// var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - /// var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); - /// var w = sc.Parallelize(new[] { new KeyValuePair<string, int>("b", 42) }, 1); + /// var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); + /// var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + /// var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); + /// var w = sc.Parallelize(new[] { new Tuple<string, int>("b", 42) }, 1); /// var m = x.GroupWith(y, z, w).MapValues(l => string.Join(" ", l.Item1) + " : " + string.Join(" ", l.Item2) + " : " + string.Join(" ", l.Item3) + " : " + string.Join(" ", l.Item4)).Collect(); /// /// @@ -672,20 +672,20 @@ public static RDD, List, List>>> GroupWith /// /// /// - public static RDD, List, List, List>>> GroupWith( - this RDD> self, - RDD> other1, - RDD> other2, - RDD> other3, + public static RDD, List, List, List>>> GroupWith( + this RDD> self, + RDD> other1, + RDD> other2, + RDD> other3, int numPartitions = 0) { // MapValues, which introduces extra CSharpRDD, is not necessary when union different RDD types if (!(typeof(V) == typeof(W1) && typeof(V) == typeof(W2))) { - return self.ConvertTo>() - .Union(other1.ConvertTo>()) - .Union(other2.ConvertTo>()) - .Union(other3.ConvertTo>()) + return self.ConvertTo>() + .Union(other1.ConvertTo>()) + .Union(other2.ConvertTo>()) + .Union(other3.ConvertTo>()) .MapPartitionsWithIndex() .CombineByKey( () => new Tuple, List, List, List>(new List(), new List(), new List(), new List()), @@ -713,7 +713,7 @@ public static RDD, List, List, List>>> // /// // /// var fractions = new { { "a", 0.2 }, { "b", 0.1 } }; // /// var rdd = sc.Parallelize(fractions.Keys.ToArray(), 2).Cartesian(sc.Parallelize(Enumerable.Range(0, 1000), 2)); - // /// var sample = rdd.Map(t => new KeyValuePair<string, int>(t.Item1, t.Item2)).SampleByKey(false, fractions, 2).GroupByKey().Collect(); + // /// var sample = rdd.Map(t => new Tuple<string, int>(t.Item1, t.Item2)).SampleByKey(false, fractions, 2).GroupByKey().Collect(); // /// // /// 100 < sample["a"].Length < 300 and 50 < sample["b"].Length < 150 // /// true @@ -730,8 +730,8 @@ public static RDD, List, List, List>>> // /// // /// // /// - //public static RDD> SampleByKey( - // this RDD> self, + //public static RDD> SampleByKey( + // this RDD> self, // bool withReplacement, // Dictionary fractions, // long seed) @@ -739,14 +739,14 @@ public static RDD, List, List, List>>> // if (fractions.Any(f => f.Value < 0.0)) // throw new ArgumentException(string.Format("Negative fraction value found in: {0}", string.Join(",", fractions.Values.ToArray()))); - // return new RDD>(self.RddProxy.SampleByKey(withReplacement, fractions, seed), self.sparkContext); + // return new RDD>(self.RddProxy.SampleByKey(withReplacement, fractions, seed), self.sparkContext); //} /// /// Return each (key, value) pair in this RDD that has no pair with matching key in . /// - /// var x = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 1), new KeyValuePair<string, int?>("b", 4), new KeyValuePair<string, int?>("b", 5), new KeyValuePair<string, int?>("a", 2) }, 2); - /// var y = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 3), new KeyValuePair<string, int?>("c", null) }, 2); + /// var x = sc.Parallelize(new[] { new Tuple<string, int?>("a", 1), new Tuple<string, int?>("b", 4), new Tuple<string, int?>("b", 5), new Tuple<string, int?>("a", 2) }, 2); + /// var y = sc.Parallelize(new[] { new Tuple<string, int?>("a", 3), new Tuple<string, int?>("c", null) }, 2); /// x.SubtractByKey(y).Collect(); /// /// [('b', 4), ('b', 5)] @@ -759,7 +759,7 @@ public static RDD, List, List, List>>> /// /// /// - public static RDD> SubtractByKey(this RDD> self, RDD> other, int numPartitions = 0) + public static RDD> SubtractByKey(this RDD> self, RDD> other, int numPartitions = 0) { return self.GroupWith(other, numPartitions).FlatMapValues(t => t.Item1.Where(v => t.Item2.Count == 0)); } @@ -770,7 +770,7 @@ public static RDD> SubtractByKey(this RDD>> l = range(1000) - /// >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new KeyValuePair<int, int>(x, y)), 10) + /// >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new Tuple<int, int>(x, y)), 10) /// >>> rdd.lookup(42) /// [42] /// @@ -780,7 +780,7 @@ public static RDD> SubtractByKey(this RDD /// /// - public static V[] Lookup(this RDD> self, K key) + public static V[] Lookup(this RDD> self, K key) { return self.Filter(new LookupHelper(key).Execute).Values().Collect(); } @@ -795,7 +795,7 @@ public static V[] Lookup(this RDD> self, K key) /// /// /// Hadoop job configuration, passed in as a dict - public static void SaveAsNewAPIHadoopDataset(this RDD> self, IEnumerable> conf) + public static void SaveAsNewAPIHadoopDataset(this RDD> self, IEnumerable> conf) { self.RddProxy.SaveAsNewAPIHadoopDataset(conf); } @@ -811,7 +811,7 @@ public static void SaveAsNewAPIHadoopDataset(this RDD> /// fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.IntWritable", None by default) /// fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.Text", None by default) /// Hadoop job configuration, passed in as a dict (None by default) - public static void SaveAsNewAPIHadoopFile(this RDD> self, string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) + public static void SaveAsNewAPIHadoopFile(this RDD> self, string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) { self.RddProxy.SaveAsNewAPIHadoopFile(path, outputFormatClass, keyClass, valueClass, conf); } @@ -826,7 +826,7 @@ public static void SaveAsNewAPIHadoopFile(this RDD> sel /// /// /// Hadoop job configuration, passed in as a dict - public static void SaveAsHadoopDataset(this RDD> self, IEnumerable> conf) + public static void SaveAsHadoopDataset(this RDD> self, IEnumerable> conf) { self.RddProxy.SaveAsHadoopDataset(conf); } @@ -848,7 +848,7 @@ public static void SaveAsHadoopDataset(this RDD> self, /// fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.Text", None by default) /// (None by default) /// (None by default) - public static void SaveAsHadoopFile(this RDD> self, string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) + public static void SaveAsHadoopFile(this RDD> self, string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) { self.RddProxy.SaveAsHadoopFile(path, outputFormatClass, keyClass, valueClass, conf, compressionCodecClass); } @@ -867,7 +867,7 @@ public static void SaveAsHadoopFile(this RDD> self, str /// /// path to sequence file /// (None by default) - public static void SaveAsSequenceFile(this RDD> self, string path, string compressionCodecClass) + public static void SaveAsSequenceFile(this RDD> self, string path, string compressionCodecClass) { self.RddProxy.SaveAsSequenceFile(path, compressionCodecClass); } @@ -887,12 +887,12 @@ public GroupByMergeHelper(Func mc) mergeCombiners = mc; } - public IEnumerable> Execute(int pid, IEnumerable> input) + public IEnumerable> Execute(int pid, IEnumerable> input) { return input.GroupBy( - kvp => kvp.Key, - kvp => kvp.Value, - (k, v) => new KeyValuePair(k, v.Aggregate(mergeCombiners)) + kvp => kvp.Item1, + kvp => kvp.Item2, + (k, v) => new Tuple(k, v.Aggregate(mergeCombiners)) ); } } @@ -908,12 +908,12 @@ public GroupByCombineHelper(Func createCombiner, Func mergeValue) this.mergeValue = mergeValue; } - public IEnumerable> Execute(int pid, IEnumerable> input) + public IEnumerable> Execute(int pid, IEnumerable> input) { return input.GroupBy( - kvp => kvp.Key, - kvp => kvp.Value, - (k, v) => new KeyValuePair(k, v.Aggregate(createCombiner(), mergeValue)) + kvp => kvp.Item1, + kvp => kvp.Item2, + (k, v) => new Tuple(k, v.Aggregate(createCombiner(), mergeValue)) ); } } @@ -932,7 +932,7 @@ public AddShuffleKeyHelper(int numPartitions, Func partitionFunc = this.partitionFunc = partitionFunc; } - public IEnumerable Execute(int split, IEnumerable> input) + public IEnumerable Execute(int split, IEnumerable> input) { // make sure that md5 is not null even if it is deseriazed in C# worker if (md5 == null) @@ -945,12 +945,12 @@ public IEnumerable Execute(int split, IEnumerable> in var ms = new MemoryStream(); if (partitionFunc == null) { - formatter.Serialize(ms, kv.Key); + formatter.Serialize(ms, kv.Item1); yield return md5.ComputeHash(ms.ToArray()).Take(8).ToArray(); } else { - long pid = (long)(partitionFunc(kv.Key) % numPartitions); + long pid = (long)(partitionFunc(kv.Item1) % numPartitions); yield return SerDe.ToBytes(pid); } ms = new MemoryStream(); @@ -969,12 +969,12 @@ public MapValuesHelper(Func f) func = f; } - public KeyValuePair Execute(KeyValuePair kvp) + public Tuple Execute(Tuple kvp) { - return new KeyValuePair + return new Tuple ( - kvp.Key, - func(kvp.Value) + kvp.Item1, + func(kvp.Item2) ); } } @@ -988,9 +988,9 @@ public FlatMapValuesHelper(Func> f) func = f; } - public IEnumerable> Execute(KeyValuePair kvp) + public IEnumerable> Execute(Tuple kvp) { - return func(kvp.Value).Select(v => new KeyValuePair(kvp.Key, v)); + return func(kvp.Item2).Select(v => new Tuple(kvp.Item1, v)); } } [Serializable] @@ -1001,9 +1001,9 @@ internal LookupHelper(K key) { this.key = key; } - internal bool Execute(KeyValuePair input) + internal bool Execute(Tuple input) { - return input.Key.ToString() == key.ToString(); + return input.Item1.ToString() == key.ToString(); } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs index c7e75ee7..345672f7 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs @@ -189,7 +189,7 @@ public int GetNumPartitions() /// /// Return a new RDD by applying a function to each element of this RDD. /// - /// sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new KeyValuePair<string, int>(x, 1)).Collect() + /// sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new Tuple<string, int>(x, 1)).Collect() /// [('a', 1), ('b', 1), ('c', 1)] /// /// @@ -288,7 +288,7 @@ public RDD Filter(Func f) /// public RDD Distinct(int numPartitions = 0) { - return Map(x => new KeyValuePair(x, 0)).ReduceByKey((x, y) => x, numPartitions).Map(x => x.Key); + return Map(x => new Tuple(x, 0)).ReduceByKey((x, y) => x, numPartitions).Map(x => x.Item1); } /// @@ -461,9 +461,9 @@ public RDD Union(RDD other) /// public RDD Intersection(RDD other) { - return Map(v => new KeyValuePair(v, 0)) - .GroupWith(other.Map(v => new KeyValuePair(v, 0))) - .Filter(kv => kv.Value.Item1.Count > 0 && kv.Value.Item2.Count > 0) + return Map(v => new Tuple(v, 0)) + .GroupWith(other.Map(v => new Tuple(v, 0))) + .Filter(kv => kv.Item2.Item1.Count > 0 && kv.Item2.Item2.Count > 0) .Keys(); } @@ -533,7 +533,7 @@ public RDD> Cartesian(RDD other) /// /// /// - public RDD>> GroupBy(Func f, int numPartitions = 0) + public RDD>> GroupBy(Func f, int numPartitions = 0) { return KeyBy(f).GroupByKey(numPartitions); } @@ -639,14 +639,14 @@ public T TreeReduce(Func f, int depth = 2) if (depth < 1) throw new ArgumentException(string.Format("Depth cannot be smaller than 1 but got {0}.", depth)); - var zeroValue = new KeyValuePair(default(T), true); // Use the second entry to indicate whether this is a dummy value. + var zeroValue = new Tuple(default(T), true); // Use the second entry to indicate whether this is a dummy value. - Func, KeyValuePair, KeyValuePair> op = new TreeReduceHelper(f).Execute; + Func, Tuple, Tuple> op = new TreeReduceHelper(f).Execute; - var reduced = Map>(x => new KeyValuePair(x, false)).TreeAggregate(zeroValue, op, op, depth); - if (reduced.Value) + var reduced = Map>(x => new Tuple(x, false)).TreeAggregate(zeroValue, op, op, depth); + if (reduced.Item2) throw new ArgumentException("Cannot reduce empty RDD."); - return reduced.Key; + return reduced.Item1; } /// @@ -736,7 +736,7 @@ public U TreeAggregate(U zeroValue, Func seqOp, Func combOp numPartitions /= scale; partiallyAggregated = partiallyAggregated - .MapPartitionsWithIndex>(new TreeAggregateHelper(numPartitions).Execute) + .MapPartitionsWithIndex>(new TreeAggregateHelper(numPartitions).Execute) .ReduceByKey(combOp, numPartitions) .Values(); } @@ -762,9 +762,9 @@ public long Count() /// /// /// - public Dictionary CountByValue() + public IDictionary CountByValue() { - return Map>(v => new KeyValuePair(v, default(T))).CountByKey(); + return Map>(v => new Tuple(v, default(T))).CountByKey(); } /// @@ -872,9 +872,9 @@ public bool IsEmpty() /// public RDD Subtract(RDD other, int numPartitions = 0) { - return Map>(v => new KeyValuePair(v, default(T))).SubtractByKey + return Map>(v => new Tuple(v, default(T))).SubtractByKey ( - other.Map>(v => new KeyValuePair(v, default(T))), + other.Map>(v => new Tuple(v, default(T))), numPartitions ) .Keys(); @@ -890,9 +890,9 @@ public RDD Subtract(RDD other, int numPartitions = 0) /// /// /// - public RDD> KeyBy(Func f) + public RDD> KeyBy(Func f) { - return Map>(new KeyByHelper(f).Execute); + return Map>(new KeyByHelper(f).Execute); } /// @@ -950,9 +950,9 @@ public RDD Coalesce(int numPartitions, bool shuffle = false) /// /// /// - public RDD> Zip(RDD other) + public RDD> Zip(RDD other) { - return new RDD>(RddProxy.Zip(other.RddProxy), sparkContext, SerializedMode.Pair); + return new RDD>(RddProxy.Zip(other.RddProxy), sparkContext, SerializedMode.Pair); } /// @@ -971,7 +971,7 @@ public RDD> Zip(RDD other) /// /// /// - public RDD> ZipWithIndex() + public RDD> ZipWithIndex() { int num = GetNumPartitions(); int[] starts = new int[num]; @@ -981,7 +981,7 @@ public RDD> ZipWithIndex() for (int i = 0; i < nums.Length - 1; i++) starts[i + 1] = starts[i] + nums[i]; } - return MapPartitionsWithIndex>(new ZipWithIndexHelper(starts).Execute); + return MapPartitionsWithIndex>(new ZipWithIndexHelper(starts).Execute); } /// @@ -996,10 +996,10 @@ public RDD> ZipWithIndex() /// /// /// - public RDD> ZipWithUniqueId() + public RDD> ZipWithUniqueId() { int num = GetNumPartitions(); - return MapPartitionsWithIndex>(new ZipWithUniqueIdHelper(num).Execute); + return MapPartitionsWithIndex>(new ZipWithUniqueIdHelper(num).Execute); } /// @@ -1217,27 +1217,27 @@ internal IEnumerable Execute(int val, IEnumerable inputValues) { K key; dynamic value; - if (x is KeyValuePair) + if (x is Tuple) { - key = ((KeyValuePair)x).Key; - value = ((KeyValuePair)x).Value; + key = ((Tuple)x).Item1; + value = ((Tuple)x).Item2; } - else if (x is KeyValuePair) + else if (x is Tuple) { - key = ((KeyValuePair)x).Key; - value = ((KeyValuePair)x).Value; + key = ((Tuple)x).Item1; + value = ((Tuple)x).Item2; } - else if (x is KeyValuePair) + else if (x is Tuple) { - key = ((KeyValuePair)x).Key; - value = ((KeyValuePair)x).Value; + key = ((Tuple)x).Item1; + value = ((Tuple)x).Item2; } else { - key = ((KeyValuePair)x).Key; - value = ((KeyValuePair)x).Value; + key = ((Tuple)x).Item1; + value = ((Tuple)x).Item2; } - return new KeyValuePair(key, value); + return new Tuple(key, value); }) .Cast(); } @@ -1397,9 +1397,9 @@ internal KeyByHelper(Func f) func = f; } - internal KeyValuePair Execute(T input) + internal Tuple Execute(T input) { - return new KeyValuePair(func(input), input); + return new Tuple(func(input), input); } } [Serializable] @@ -1426,9 +1426,9 @@ internal TreeAggregateHelper(int numPartitions) { this.numPartitions = numPartitions; } - internal IEnumerable> Execute(int pid, IEnumerable input) + internal IEnumerable> Execute(int pid, IEnumerable input) { - return input.Select(x => new KeyValuePair(pid % numPartitions, x)); + return input.Select(x => new Tuple(pid % numPartitions, x)); } } [Serializable] @@ -1439,14 +1439,14 @@ internal TreeReduceHelper(Func func) { this.func = func; } - internal KeyValuePair Execute(KeyValuePair x, KeyValuePair y) + internal Tuple Execute(Tuple x, Tuple y) { - if (x.Value) + if (x.Item2) return y; - else if (y.Value) + else if (y.Item2) return x; else - return new KeyValuePair(func(x.Key, y.Key), false); + return new Tuple(func(x.Item1, y.Item1), false); } } [Serializable] @@ -1498,12 +1498,12 @@ internal ZipWithUniqueIdHelper(int num) { this.num = num; } - internal IEnumerable> Execute(int pid, IEnumerable input) + internal IEnumerable> Execute(int pid, IEnumerable input) { long l = 0; foreach (var item in input) { - yield return new KeyValuePair(item, (l++) * num + pid); + yield return new Tuple(item, (l++) * num + pid); } } } @@ -1515,12 +1515,12 @@ internal ZipWithIndexHelper(int[] starts) { this.starts = starts; } - internal IEnumerable> Execute(int pid, IEnumerable input) + internal IEnumerable> Execute(int pid, IEnumerable input) { long l = 0; foreach (var item in input) { - yield return new KeyValuePair(item, (l++) + starts[pid]); + yield return new Tuple(item, (l++) + starts[pid]); } } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs index 442a11d4..f8a85865 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs @@ -297,7 +297,7 @@ public RDD SequenceFile(string filePath, string keyClass, string valueCl /// (None by default) /// Hadoop configuration, passed in as a dict (None by default) /// - public RDD NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) + public RDD NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) { return new RDD(SparkContextProxy.NewAPIHadoopFile(filePath, inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf, 1), this, SerializedMode.None); } @@ -316,7 +316,7 @@ public RDD NewAPIHadoopFile(string filePath, string inputFormatClass, st /// (None by default) /// Hadoop configuration, passed in as a dict (None by default) /// - public RDD NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) + public RDD NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) { return new RDD(SparkContextProxy.NewAPIHadoopRDD(inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf, 1), this, SerializedMode.None); } @@ -337,7 +337,7 @@ public RDD NewAPIHadoopRDD(string inputFormatClass, string keyClass, str /// (None by default) /// Hadoop configuration, passed in as a dict (None by default) /// - public RDD HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) + public RDD HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) { return new RDD(SparkContextProxy.HadoopFile(filePath, inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf, 1), this, SerializedMode.None); } @@ -356,7 +356,7 @@ public RDD HadoopFile(string filePath, string inputFormatClass, string k /// (None by default) /// Hadoop configuration, passed in as a dict (None by default) /// - public RDD HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) + public RDD HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass = null, string valueConverterClass = null, IEnumerable> conf = null) { return new RDD(SparkContextProxy.HadoopRDD(inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf, 1), this, SerializedMode.None); } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JvmBridgeUtils.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JvmBridgeUtils.cs index cb250fee..d4e0002e 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JvmBridgeUtils.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JvmBridgeUtils.cs @@ -7,6 +7,7 @@ using System.Text; using System.Threading.Tasks; using Microsoft.Spark.CSharp.Proxy.Ipc; +using System; namespace Microsoft.Spark.CSharp.Interop.Ipc { @@ -16,31 +17,31 @@ namespace Microsoft.Spark.CSharp.Interop.Ipc [ExcludeFromCodeCoverage] //IPC calls to JVM validated using validation-enabled samples - unit test coverage not reqiured internal static class JvmBridgeUtils { - public static JvmObjectReference GetJavaMap(IEnumerable> enumerable) + public static JvmObjectReference GetJavaMap(IEnumerable> enumerable) { var jmap = SparkCLRIpcProxy.JvmBridge.CallConstructor("java.util.Hashtable", new object[] { }); if (enumerable != null) { foreach (var item in enumerable) - SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jmap, "put", new object[] { item.Key, item.Value }); + SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jmap, "put", new object[] { item.Item1, item.Item2 }); } return jmap; } - public static JvmObjectReference GetJavaHashMap(IEnumerable> enumerable) + public static JvmObjectReference GetJavaHashMap(IEnumerable> enumerable) { var jmap = SparkCLRIpcProxy.JvmBridge.CallConstructor("java.util.HashMap", new object[] { }); if (enumerable != null) { foreach (var item in enumerable) - SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jmap, "put", new object[] { item.Key, item.Value }); + SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jmap, "put", new object[] { item.Item1, item.Item2 }); } return jmap; } - public static JvmObjectReference GetScalaMutableMap(Dictionary mapValues) + public static JvmObjectReference GetScalaMutableMap(IEnumerable> mapValues) { - var hashMapReference = GetJavaHashMap(mapValues.Select(kvp => kvp)); + var hashMapReference = GetJavaHashMap(mapValues); return new JvmObjectReference(SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.sql.api.csharp.JvmBridgeUtils", "toMutableMap", new object[] { hashMapReference }).ToString()); } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IRDDProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IRDDProxy.cs index ae8a18fa..e323cf47 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IRDDProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IRDDProxy.cs @@ -31,13 +31,13 @@ internal interface IRDDProxy string Name { get; } void SetName(string name); IRDDProxy RandomSampleWithRange(double lb, double ub, long seed); - IRDDProxy SampleByKey(bool withReplacement, Dictionary fractions, long seed); + IRDDProxy SampleByKey(bool withReplacement, IEnumerable> fractions, long seed); IRDDProxy Zip(IRDDProxy other); string ToDebugString(); - void SaveAsNewAPIHadoopDataset(IEnumerable> conf); - void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf); - void SaveAsHadoopDataset(IEnumerable> conf); - void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass); + void SaveAsNewAPIHadoopDataset(IEnumerable> conf); + void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf); + void SaveAsHadoopDataset(IEnumerable> conf); + void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass); void SaveAsSequenceFile(string path, string compressionCodecClass); void SaveAsTextFile(string path, string compressionCodecClass); long Count(); diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkContextProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkContextProxy.cs index d5b4ac0b..4764322e 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkContextProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkContextProxy.cs @@ -33,10 +33,10 @@ internal interface ISparkContextProxy IRDDProxy WholeTextFiles(string filePath, int minPartitions); IRDDProxy BinaryFiles(string filePath, int minPartitions); IRDDProxy SequenceFile(string filePath, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, int minSplits, int batchSize); - IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); - IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); - IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); - IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); + IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); + IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); + IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); + IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize); IRDDProxy CheckpointFile(string filePath); IRDDProxy Union(IEnumerable rdds); void AddFile(string path); diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IStreamingContextProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IStreamingContextProxy.cs index 8af21d61..1e23f758 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IStreamingContextProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/IStreamingContextProxy.cs @@ -20,9 +20,9 @@ internal interface IStreamingContextProxy void Checkpoint(string directory); IDStreamProxy TextFileStream(string directory); IDStreamProxy SocketTextStream(string hostname, int port, StorageLevelType storageLevelType); - IDStreamProxy KafkaStream(Dictionary topics, Dictionary kafkaParams, StorageLevelType storageLevelType); - IDStreamProxy DirectKafkaStream(List topics, Dictionary kafkaParams, Dictionary fromOffsets); - IDStreamProxy DirectKafkaStreamWithRepartition(List topics, Dictionary kafkaParams, Dictionary fromOffsets, int numPartitions); + IDStreamProxy KafkaStream(IEnumerable> topics, IEnumerable> kafkaParams, StorageLevelType storageLevelType); + IDStreamProxy DirectKafkaStream(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets); + IDStreamProxy DirectKafkaStreamWithRepartition(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets, int numPartitions); IDStreamProxy Union(IDStreamProxy firstDStreams, IDStreamProxy[] otherDStreams); void AwaitTermination(); void AwaitTermination(int timeout); @@ -31,7 +31,7 @@ internal interface IStreamingContextProxy IDStreamProxy CreateCSharpReducedWindowedDStream(IDStreamProxy jdstream, byte[] func, byte[] invFunc, int windowSeconds, int slideSeconds, string serializationMode); IDStreamProxy CreateCSharpStateDStream(IDStreamProxy jdstream, byte[] func, string className, string serializationMode, string serializationMode2); IDStreamProxy CreateConstantInputDStream(IRDDProxy rddProxy); - IDStreamProxy EventHubsUnionStream(Dictionary eventHubsParams, StorageLevelType storageLevelType); + IDStreamProxy EventHubsUnionStream(IEnumerable> eventHubsParams, StorageLevelType storageLevelType); } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/RDDIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/RDDIpcProxy.cs index 1433ac9d..9377c079 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/RDDIpcProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/RDDIpcProxy.cs @@ -158,7 +158,7 @@ public void SetName(string name) SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jvmRddReference, "setName", new object[] { name }); } - public IRDDProxy SampleByKey(bool withReplacement, Dictionary fractions, long seed) + public IRDDProxy SampleByKey(bool withReplacement, IEnumerable> fractions, long seed) { var jfractions = JvmBridgeUtils.GetJavaMap(fractions) as JvmObjectReference; return new RDDIpcProxy(new JvmObjectReference((string) SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jvmRddReference, "sampleByKey", new object[] { withReplacement, jfractions, seed }))); @@ -176,25 +176,25 @@ public IRDDProxy Zip(IRDDProxy other) return new RDDIpcProxy(new JvmObjectReference((string) SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jvmRddReference, "zip", new object[] { (other as RDDIpcProxy).jvmRddReference }))); } - public void SaveAsNewAPIHadoopDataset(IEnumerable> conf) + public void SaveAsNewAPIHadoopDataset(IEnumerable> conf) { var jconf = JvmBridgeUtils.GetJavaMap(conf); SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "saveAsHadoopDataset", new object[] { jvmRddReference, false, jconf, null, null, true }); } - public void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) + public void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) { var jconf = JvmBridgeUtils.GetJavaMap(conf); SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "saveAsNewAPIHadoopFile", new object[] { jvmRddReference, false, path, outputFormatClass, keyClass, valueClass, null, null, jconf }); } - public void SaveAsHadoopDataset(IEnumerable> conf) + public void SaveAsHadoopDataset(IEnumerable> conf) { var jconf = JvmBridgeUtils.GetJavaMap(conf); SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "saveAsHadoopDataset", new object[] { jvmRddReference, false, jconf, null, null, false }); } - public void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) + public void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) { var jconf = JvmBridgeUtils.GetJavaMap(conf); SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "saveAsHadoopFile", new object[] { jvmRddReference, false, path, outputFormatClass, keyClass, valueClass, null, null, jconf, compressionCodecClass }); diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs index c74df12d..bfdd686b 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs @@ -157,7 +157,7 @@ public IRDDProxy SequenceFile(string filePath, string keyClass, string valueClas return new RDDIpcProxy(jvmRddReference); } - public IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { var jconf = JvmBridgeUtils.GetJavaHashMap(conf); var jvmRddReference = new JvmObjectReference((string)SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "newAPIHadoopFile", @@ -165,7 +165,7 @@ public IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, stri return new RDDIpcProxy(jvmRddReference); } - public IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { var jconf = JvmBridgeUtils.GetJavaHashMap(conf); var jvmRddReference = new JvmObjectReference((string)SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "newAPIHadoopRDD", @@ -173,7 +173,7 @@ public IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, strin return new RDDIpcProxy(jvmRddReference); } - public IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { var jconf = JvmBridgeUtils.GetJavaHashMap(conf); var jvmRddReference = new JvmObjectReference((string)SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "hadoopFile", @@ -181,7 +181,7 @@ public IRDDProxy HadoopFile(string filePath, string inputFormatClass, string key return new RDDIpcProxy(jvmRddReference); } - public IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { var jconf = JvmBridgeUtils.GetJavaHashMap(conf); var jvmRddReference = new JvmObjectReference((string)SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.python.PythonRDD", "hadoopRDD", diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/StreamingContextIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/StreamingContextIpcProxy.cs index 91bae5e5..047169e3 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/StreamingContextIpcProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/StreamingContextIpcProxy.cs @@ -173,7 +173,7 @@ public IDStreamProxy SocketTextStream(string hostname, int port, StorageLevelTyp return new DStreamIpcProxy(jstream); } - public IDStreamProxy KafkaStream(Dictionary topics, Dictionary kafkaParams, StorageLevelType storageLevelType) + public IDStreamProxy KafkaStream(IEnumerable> topics, IEnumerable> kafkaParams, StorageLevelType storageLevelType) { JvmObjectReference jtopics = JvmBridgeUtils.GetJavaMap(topics); JvmObjectReference jkafkaParams = JvmBridgeUtils.GetJavaMap(kafkaParams); @@ -184,16 +184,16 @@ public IDStreamProxy KafkaStream(Dictionary topics, Dictionary topics, Dictionary kafkaParams, Dictionary fromOffsets) + public IDStreamProxy DirectKafkaStream(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets) { JvmObjectReference jtopics = JvmBridgeUtils.GetJavaSet(topics); JvmObjectReference jkafkaParams = JvmBridgeUtils.GetJavaMap(kafkaParams); var jTopicAndPartitions = fromOffsets.Select(x => - new KeyValuePair + new Tuple ( - SparkCLRIpcProxy.JvmBridge.CallConstructor("kafka.common.TopicAndPartition", new object[] { x.Key.Split(':')[0], int.Parse(x.Key.Split(':')[1]) }), - x.Value + SparkCLRIpcProxy.JvmBridge.CallConstructor("kafka.common.TopicAndPartition", new object[] { x.Item1.Split(':')[0], int.Parse(x.Item1.Split(':')[1]) }), + x.Item2 ) ); @@ -204,16 +204,16 @@ public IDStreamProxy DirectKafkaStream(List topics, Dictionary topics, Dictionary kafkaParams, Dictionary fromOffsets, int numPartitions) + public IDStreamProxy DirectKafkaStreamWithRepartition(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets, int numPartitions) { JvmObjectReference jtopics = JvmBridgeUtils.GetJavaSet(topics); JvmObjectReference jkafkaParams = JvmBridgeUtils.GetJavaMap(kafkaParams); var jTopicAndPartitions = fromOffsets.Select(x => - new KeyValuePair + new Tuple ( - SparkCLRIpcProxy.JvmBridge.CallConstructor("kafka.common.TopicAndPartition", new object[] { x.Key.Split(':')[0], int.Parse(x.Key.Split(':')[1]) }), - x.Value + SparkCLRIpcProxy.JvmBridge.CallConstructor("kafka.common.TopicAndPartition", new object[] { x.Item1.Split(':')[0], int.Parse(x.Item1.Split(':')[1]) }), + x.Item2 ) ); @@ -224,7 +224,7 @@ public IDStreamProxy DirectKafkaStreamWithRepartition(List topics, Dicti return new DStreamIpcProxy(jstream); } - public IDStreamProxy EventHubsUnionStream(Dictionary eventHubsParams, StorageLevelType storageLevelType) + public IDStreamProxy EventHubsUnionStream(IEnumerable> eventHubsParams, StorageLevelType storageLevelType) { JvmObjectReference eventHubsParamsReference = JvmBridgeUtils.GetScalaMutableMap(eventHubsParams); JvmObjectReference storageLevelTypeReference = SparkContextIpcProxy.GetJavaStorageLevel(storageLevelType); diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/DStream.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/DStream.cs index 6cb6c49e..ed0634f6 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/DStream.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/DStream.cs @@ -153,7 +153,7 @@ public DStream MapPartitionsWithIndex(Func, IEnumerabl /// public DStream Reduce(Func f) { - return Map>(x => new KeyValuePair(string.Empty, x)).ReduceByKey(f, 1).Map(kvp => kvp.Value); + return Map>(x => new Tuple(string.Empty, x)).ReduceByKey(f, 1).Map(kvp => kvp.Item2); } /// @@ -235,9 +235,9 @@ public DStream Checkpoint(long intervalMs) /// distinct value in each RDD of this DStream. /// /// - public DStream> CountByValue(int numPartitions = 0) + public DStream> CountByValue(int numPartitions = 0) { - return Map(v => new KeyValuePair(v, 1L)).ReduceByKey((x, y) => x + y, numPartitions); + return Map(v => new Tuple(v, 1L)).ReduceByKey((x, y) => x + y, numPartitions); } /// @@ -427,9 +427,9 @@ public DStream Window(int windowSeconds, int slideSeconds) /// public DStream ReduceByWindow(Func reduceFunc, Func invReduceFunc, int windowSeconds, int slideSeconds = 0) { - var keyed = Map(v => new KeyValuePair(1, v)); + var keyed = Map(v => new Tuple(1, v)); var reduced = keyed.ReduceByKeyAndWindow(reduceFunc, invReduceFunc, windowSeconds, slideSeconds, 1); - return reduced.Map(kv => (T)kv.Value); + return reduced.Map(kv => (T)kv.Item2); } /// @@ -462,9 +462,9 @@ public DStream CountByWindow(int windowSeconds, int slideSeconds = 0) /// public DStream CountByValueAndWindow(int windowSeconds, int slideSeconds, int numPartitions = 0) { - var keyed = Map(v => new KeyValuePair(v, 1)); + var keyed = Map(v => new Tuple(v, 1)); var counted = keyed.ReduceByKeyAndWindow((x, y) => x + y, (x, y) => x - y, windowSeconds, slideSeconds, numPartitions); - return counted.Filter(kv => kv.Value > 0).Count(); + return counted.Filter(kv => kv.Item2 > 0).Count(); } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/EventHubsUtils.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/EventHubsUtils.cs index 53a46de0..44b959ec 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/EventHubsUtils.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/EventHubsUtils.cs @@ -35,7 +35,7 @@ public class EventHubsUtils /// /// Storage level, by default it is MEMORY_ONLY /// DStream with byte[] representing events from EventHub - public static DStream CreateUnionStream(StreamingContext ssc, Dictionary eventhubsParams, StorageLevelType storageLevelType = StorageLevelType.MEMORY_ONLY) + public static DStream CreateUnionStream(StreamingContext ssc, IEnumerable> eventhubsParams, StorageLevelType storageLevelType = StorageLevelType.MEMORY_ONLY) { return new DStream(ssc.streamingContextProxy.EventHubsUnionStream(eventhubsParams, storageLevelType), ssc, SerializedMode.None); } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/Kafka.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/Kafka.cs index 6f517810..c204f69d 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/Kafka.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/Kafka.cs @@ -25,7 +25,7 @@ public class KafkaUtils /// Dict of (topic_name -> numPartitions) to consume. Each partition is consumed in its own thread. /// Additional params for Kafka /// A DStream object - public static DStream> CreateStream(StreamingContext ssc, string zkQuorum, string groupId, Dictionary topics, Dictionary kafkaParams) + public static DStream> CreateStream(StreamingContext ssc, string zkQuorum, string groupId, IEnumerable> topics, IEnumerable> kafkaParams) { return CreateStream(ssc, zkQuorum, groupId, topics, kafkaParams, StorageLevelType.MEMORY_AND_DISK_SER_2); } @@ -40,19 +40,21 @@ public static DStream> CreateStream(StreamingContex /// Additional params for Kafka /// RDD storage level. /// A DStream object - public static DStream> CreateStream(StreamingContext ssc, string zkQuorum, string groupId, Dictionary topics, Dictionary kafkaParams, StorageLevelType storageLevelType) + public static DStream> CreateStream(StreamingContext ssc, string zkQuorum, string groupId, IEnumerable> topics, IEnumerable> kafkaParams, StorageLevelType storageLevelType) { if (kafkaParams == null) - kafkaParams = new Dictionary(); + kafkaParams = new List>(); + + var kafkaParamsMap = kafkaParams.ToDictionary(x => x.Item1, x => x.Item2); if (!string.IsNullOrEmpty(zkQuorum)) - kafkaParams["zookeeper.connect"] = zkQuorum; + kafkaParamsMap["zookeeper.connect"] = zkQuorum; if (groupId != null) - kafkaParams["group.id"] = groupId; - if (kafkaParams.ContainsKey("zookeeper.connection.timeout.ms")) - kafkaParams["zookeeper.connection.timeout.ms"] = "10000"; + kafkaParamsMap["group.id"] = groupId; + if (kafkaParamsMap.ContainsKey("zookeeper.connection.timeout.ms")) + kafkaParamsMap["zookeeper.connection.timeout.ms"] = "10000"; - return new DStream>(ssc.streamingContextProxy.KafkaStream(topics, kafkaParams, storageLevelType), ssc); + return new DStream>(ssc.streamingContextProxy.KafkaStream(topics, kafkaParamsMap.Select(x => Tuple.Create(x.Key, x.Value)), storageLevelType), ssc); } /// @@ -79,9 +81,9 @@ public static DStream> CreateStream(StreamingContex /// /// Per-topic/partition Kafka offsets defining the (inclusive) starting point of the stream. /// A DStream object - public static DStream> CreateDirectStream(StreamingContext ssc, List topics, Dictionary kafkaParams, Dictionary fromOffsets) + public static DStream> CreateDirectStream(StreamingContext ssc, List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets) { - return new DStream>(ssc.streamingContextProxy.DirectKafkaStream(topics, kafkaParams, fromOffsets), ssc, SerializedMode.Pair); + return new DStream>(ssc.streamingContextProxy.DirectKafkaStream(topics, kafkaParams, fromOffsets), ssc, SerializedMode.Pair); } /// @@ -116,9 +118,9 @@ public static DStream> CreateDirectStream(Streaming /// If numPartitions > 0, repartition using this parameter /// /// A DStream object - public static DStream> CreateDirectStreamWithRepartition(StreamingContext ssc, List topics, Dictionary kafkaParams, Dictionary fromOffsets, int numPartitions = -1) + public static DStream> CreateDirectStreamWithRepartition(StreamingContext ssc, List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets, int numPartitions = -1) { - return new DStream>(ssc.streamingContextProxy.DirectKafkaStreamWithRepartition(topics, kafkaParams, fromOffsets, numPartitions), ssc, SerializedMode.Pair); + return new DStream>(ssc.streamingContextProxy.DirectKafkaStreamWithRepartition(topics, kafkaParams, fromOffsets, numPartitions), ssc, SerializedMode.Pair); } } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/MapWithStateDStream.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/MapWithStateDStream.cs index d0907743..fe3de008 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/MapWithStateDStream.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/MapWithStateDStream.cs @@ -29,9 +29,9 @@ namespace Microsoft.Spark.CSharp.Streaming [Serializable] public class MapWithStateDStream : DStream { - internal DStream> snapshotsDStream; + internal DStream> snapshotsDStream; - internal MapWithStateDStream(DStream mappedDataDStream, DStream> snapshotsDStream) + internal MapWithStateDStream(DStream mappedDataDStream, DStream> snapshotsDStream) : base(mappedDataDStream.DStreamProxy, mappedDataDStream.streamingContext) { this.snapshotsDStream = snapshotsDStream; @@ -40,7 +40,7 @@ internal MapWithStateDStream(DStream mappedDataDStream, DStream /// Return a pair DStream where each RDD is the snapshot of the state of all the keys. /// - public DStream> StateSnapshots() + public DStream> StateSnapshots() { return snapshotsDStream; } @@ -87,11 +87,11 @@ public MapWithStateRDDRecord() { } - public MapWithStateRDDRecord(long t, IEnumerable> iter) + public MapWithStateRDDRecord(long t, IEnumerable> iter) { foreach (var p in iter) { - stateMap[p.Key] = new KeyedState(p.Value, t); + stateMap[p.Item1] = new KeyedState(p.Item2, t); } } } @@ -131,14 +131,14 @@ internal IEnumerable Execute(int pid, IEnumerable iter) while (enumerator.MoveNext()) { - KeyValuePair kv = enumerator.Current; + Tuple kv = enumerator.Current; KeyedState keyedState; - State wrappedState = stateRddRecord.stateMap.TryGetValue(kv.Key, out keyedState) ? new State(keyedState.state) : new State(default(S)); + State wrappedState = stateRddRecord.stateMap.TryGetValue(kv.Item1, out keyedState) ? new State(keyedState.state) : new State(default(S)); var mappedData = default(M); try { - mappedData = f(kv.Key, kv.Value, wrappedState); + mappedData = f(kv.Item1, kv.Item2, wrappedState); } catch (Exception e) { @@ -149,11 +149,11 @@ internal IEnumerable Execute(int pid, IEnumerable iter) if (wrappedState.removed) { - stateRddRecord.stateMap.Remove(kv.Key); + stateRddRecord.stateMap.Remove(kv.Item1); } else if (wrappedState.updated || wrappedState.defined) { - stateRddRecord.stateMap[kv.Key] = new KeyedState(wrappedState.state, ticks); + stateRddRecord.stateMap[kv.Item1] = new KeyedState(wrappedState.state, ticks); } } @@ -223,7 +223,7 @@ internal RDD Execute(double t, RDD stateRDD, RDD valu valuesRDD = prevFunc(t, valuesRDD); } - var values = valuesRDD.ConvertTo>().PartitionBy(stateSpec.numPartitions); + var values = valuesRDD.ConvertTo>().PartitionBy(stateSpec.numPartitions); if (stateRDD == null) { @@ -259,12 +259,12 @@ internal MapWithStateMapPartitionHelper(long ticks) this.ticks = ticks; } - internal IEnumerable> Execute(IEnumerable> iter) + internal IEnumerable> Execute(IEnumerable> iter) { return new[] {new MapWithStateRDDRecord(ticks, iter)}; } - internal IEnumerable> ExecuteWithoutInitialState(IEnumerable> iter) + internal IEnumerable> ExecuteWithoutInitialState(IEnumerable> iter) { return new[] { new MapWithStateRDDRecord() }; } @@ -283,7 +283,7 @@ public class StateSpec internal Func, M> mappingFunction; internal int numPartitions; internal TimeSpan idleDuration = TimeSpan.FromTicks(0); - internal RDD> initialState = null; + internal RDD> initialState = null; /// /// Create a StateSpec for setting all the specifications of the `mapWithState` operation on a pair DStream. @@ -318,7 +318,7 @@ public StateSpec Timeout(TimeSpan idleDuration) return this; } - public StateSpec InitialState(RDD> initialState) + public StateSpec InitialState(RDD> initialState) { this.initialState = initialState; return this; diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/PairDStreamFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/PairDStreamFunctions.cs index 409a8b47..8a9244d2 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/PairDStreamFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Streaming/PairDStreamFunctions.cs @@ -15,7 +15,7 @@ namespace Microsoft.Spark.CSharp.Streaming { /// - /// operations only available to KeyValuePair RDD + /// operations only available to Tuple RDD /// public static class PairDStreamFunctions { @@ -28,7 +28,7 @@ public static class PairDStreamFunctions /// /// /// - public static DStream> ReduceByKey(this DStream> self, Func reduceFunc, int numPartitions = 0) + public static DStream> ReduceByKey(this DStream> self, Func reduceFunc, int numPartitions = 0) { return self.CombineByKey(() => default(V), reduceFunc, reduceFunc, numPartitions); } @@ -45,8 +45,8 @@ public static DStream> ReduceByKey(this DStream /// /// - public static DStream> CombineByKey( - this DStream> self, + public static DStream> CombineByKey( + this DStream> self, Func createCombiner, Func mergeValue, Func mergeCombiners, @@ -55,7 +55,7 @@ public static DStream> CombineByKey( if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.Transform>(new CombineByKeyHelper(createCombiner, mergeValue, mergeCombiners, numPartitions).Execute); + return self.Transform>(new CombineByKeyHelper(createCombiner, mergeValue, mergeCombiners, numPartitions).Execute); } /// @@ -66,12 +66,12 @@ public static DStream> CombineByKey( /// /// /// - public static DStream> PartitionBy(this DStream> self, int numPartitions = 0) + public static DStream> PartitionBy(this DStream> self, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.Transform>(new PartitionByHelper(numPartitions).Execute); + return self.Transform>(new PartitionByHelper(numPartitions).Execute); } /// @@ -84,7 +84,7 @@ public static DStream> PartitionBy(this DStream /// /// - public static DStream> MapValues(this DStream> self, Func func) + public static DStream> MapValues(this DStream> self, Func func) { return self.Map(new MapValuesHelper(func).Execute, true); } @@ -99,7 +99,7 @@ public static DStream> MapValues(this DStream /// /// - public static DStream> FlatMapValues(this DStream> self, Func> func) + public static DStream> FlatMapValues(this DStream> self, Func> func) { return self.FlatMap(new FlatMapValuesHelper(func).Execute, true); } @@ -112,9 +112,9 @@ public static DStream> FlatMapValues(this DStream /// /// - public static DStream>> GroupByKey(this DStream> self, int numPartitions = 0) + public static DStream>> GroupByKey(this DStream> self, int numPartitions = 0) { - return self.Transform>>(new GroupByKeyHelper(numPartitions).Execute); + return self.Transform>>(new GroupByKeyHelper(numPartitions).Execute); } /// @@ -128,12 +128,12 @@ public static DStream>> GroupByKey(this DStream /// /// - public static DStream, List>>> GroupWith(this DStream> self, DStream> other, int numPartitions = 0) + public static DStream, List>>> GroupWith(this DStream> self, DStream> other, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.TransformWith, KeyValuePair, List>>>(new GroupWithHelper(numPartitions).Execute, other); + return self.TransformWith, Tuple, List>>>(new GroupWithHelper(numPartitions).Execute, other); } /// @@ -147,12 +147,12 @@ public static DStream, List>>> GroupWith /// /// - public static DStream>> Join(this DStream> self, DStream> other, int numPartitions = 0) + public static DStream>> Join(this DStream> self, DStream> other, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.TransformWith, KeyValuePair>>(new JoinHelper(numPartitions).Execute, other); + return self.TransformWith, Tuple>>(new JoinHelper(numPartitions).Execute, other); } /// @@ -166,12 +166,12 @@ public static DStream>> Join(this DStream /// /// - public static DStream>>> LeftOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) + public static DStream>>> LeftOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.TransformWith, KeyValuePair>>>(new LeftOuterJoinHelper(numPartitions).Execute, other); + return self.TransformWith, Tuple>>>(new LeftOuterJoinHelper(numPartitions).Execute, other); } /// @@ -185,12 +185,12 @@ public static DStream>>> LeftOuterJoin /// /// - public static DStream, W>>> RightOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) + public static DStream, W>>> RightOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.TransformWith, KeyValuePair, W>>>(new RightOuterJoinHelper(numPartitions).Execute, other); + return self.TransformWith, Tuple, W>>>(new RightOuterJoinHelper(numPartitions).Execute, other); } /// @@ -204,12 +204,12 @@ public static DStream, W>>> RightOuterJoin /// /// - public static DStream, Option>>> FullOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) + public static DStream, Option>>> FullOuterJoin(this DStream> self, DStream> other, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - return self.TransformWith, KeyValuePair, Option>>>(new FullOuterJoinHelper(numPartitions).Execute, other); + return self.TransformWith, Tuple, Option>>>(new FullOuterJoinHelper(numPartitions).Execute, other); } /// @@ -227,7 +227,7 @@ public static DStream, Option>>> FullOuterJoi /// /// Number of partitions of each RDD in the new DStream. /// - public static DStream>> GroupByKeyAndWindow(this DStream> self, + public static DStream>> GroupByKeyAndWindow(this DStream> self, int windowSeconds, int slideSeconds, int numPartitions = 0) { var ls = self.MapValues(x => new List { x }); @@ -259,13 +259,13 @@ public static DStream>> GroupByKeyAndWindow /// number of partitions of each RDD in the new DStream. /// function to filter expired key-value pairs; only pairs that satisfy the function are retained set this to null if you do not want to filter /// - public static DStream> ReduceByKeyAndWindow(this DStream> self, + public static DStream> ReduceByKeyAndWindow(this DStream> self, Func reduceFunc, Func invReduceFunc, int windowSeconds, int slideSeconds = 0, int numPartitions = 0, - Func, bool> filterFunc = null) + Func, bool> filterFunc = null) { self.ValidateWindowParam(windowSeconds, slideSeconds); @@ -275,7 +275,7 @@ public static DStream> ReduceByKeyAndWindow(this DStrea // dstream to be transformed by substracting old RDDs and adding new RDDs based on the window var reduced = self.ReduceByKey(reduceFunc, numPartitions); - Func, RDD> prevFunc = reduced.Piplinable ? (reduced as TransformedDStream>).func : null; + Func, RDD> prevFunc = reduced.Piplinable ? (reduced as TransformedDStream>).func : null; var helper = new ReduceByKeyAndWindowHelper(reduceFunc, invReduceFunc, numPartitions, filterFunc, prevFunc); // function to reduce the new values that entered the window (e.g., adding new counts) @@ -295,7 +295,7 @@ public static DStream> ReduceByKeyAndWindow(this DStrea formatter.Serialize(stream, invReduceF); } - return new DStream>( + return new DStream>( SparkCLREnvironment.SparkCLRProxy.StreamingContextProxy.CreateCSharpReducedWindowedDStream( reduced.Piplinable ? reduced.prevDStreamProxy : reduced.DStreamProxy, stream.ToArray(), @@ -321,7 +321,7 @@ public static DStream> ReduceByKeyAndWindow(this DStrea /// /// /// - public static DStream> UpdateStateByKey(this DStream> self, + public static DStream> UpdateStateByKey(this DStream> self, Func, S, S> updateFunc, int numPartitions = 0) { @@ -339,11 +339,11 @@ public static DStream> UpdateStateByKey(this DStream /// State update function - IEnumerable[K, [newValues, oldState]] => IEnumerable[K, newState] /// /// - public static DStream> UpdateStateByKey(this DStream> self, - Func, S>>>, IEnumerable>> updateFunc, + public static DStream> UpdateStateByKey(this DStream> self, + Func, S>>>, IEnumerable>> updateFunc, int numPartitions = 0) { - return UpdateStateByKey(self, new MapPartitionsHelper, S>>, KeyValuePair>(updateFunc).Execute, numPartitions); + return UpdateStateByKey(self, new MapPartitionsHelper, S>>, Tuple>(updateFunc).Execute, numPartitions); } /// @@ -357,14 +357,14 @@ public static DStream> UpdateStateByKey(this DStream /// State update function - (pid, IEnumerable[K, [newValues, oldState]]) => IEnumerable[K, newState] /// /// - public static DStream> UpdateStateByKey(this DStream> self, - Func, S>>>, IEnumerable>> updateFunc, + public static DStream> UpdateStateByKey(this DStream> self, + Func, S>>>, IEnumerable>> updateFunc, int numPartitions = 0) { if (numPartitions <= 0) numPartitions = self.streamingContext.SparkContext.DefaultParallelism; - Func, RDD> prevFunc = self.Piplinable ? (self as TransformedDStream>).func : null; + Func, RDD> prevFunc = self.Piplinable ? (self as TransformedDStream>).func : null; Func, RDD, RDD> func = new UpdateStateByKeysHelper(updateFunc, prevFunc, numPartitions).Execute; @@ -372,7 +372,7 @@ public static DStream> UpdateStateByKey(this DStream var stream = new MemoryStream(); formatter.Serialize(stream, func); - return new DStream>(SparkCLREnvironment.SparkCLRProxy.StreamingContextProxy.CreateCSharpStateDStream( + return new DStream>(SparkCLREnvironment.SparkCLRProxy.StreamingContextProxy.CreateCSharpStateDStream( self.Piplinable ? self.prevDStreamProxy : self.DStreamProxy, stream.ToArray(), "CSharpStateDStream", @@ -385,14 +385,14 @@ public static DStream> UpdateStateByKey(this DStream /// Return a new "state" DStream where the state for each key is updated by applying /// the given function on the previous state of the key and the new values of the key. /// - public static MapWithStateDStream MapWithState(this DStream> self, StateSpec stateSpec) + public static MapWithStateDStream MapWithState(this DStream> self, StateSpec stateSpec) { if (stateSpec.numPartitions <= 0) { stateSpec = stateSpec.NumPartitions(self.streamingContext.SparkContext.DefaultParallelism); } - Func, RDD> prevFunc = self.Piplinable ? (self as TransformedDStream>).func : null; + Func, RDD> prevFunc = self.Piplinable ? (self as TransformedDStream>).func : null; Func, RDD, RDD> func = new MapWithStateHelper(prevFunc, stateSpec).Execute; @@ -409,8 +409,8 @@ public static MapWithStateDStream MapWithState(this DStr self.streamingContext); DStream mappedDataDStream = mapWithStateDStream.FlatMap(r => r.mappedData); - DStream> snapshotsDStream = mapWithStateDStream.FlatMap( - r => r.stateMap.Select(entry => new KeyValuePair(entry.Key, entry.Value.state))); + DStream> snapshotsDStream = mapWithStateDStream.FlatMap( + r => r.stateMap.Select(entry => new Tuple(entry.Key, entry.Value.state))); return new MapWithStateDStream(mappedDataDStream, snapshotsDStream); } @@ -438,7 +438,7 @@ internal CombineByKeyHelper(Func createCombiner, Func mergeValue, Fu this.numPartitions = numPartitions; } - internal RDD> Execute(RDD> rdd) + internal RDD> Execute(RDD> rdd) { return rdd.CombineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions); } @@ -453,7 +453,7 @@ internal PartitionByHelper(int numPartitions = 0) this.numPartitions = numPartitions; } - internal RDD> Execute(RDD> rdd) + internal RDD> Execute(RDD> rdd) { return rdd.PartitionBy(numPartitions); } @@ -468,9 +468,9 @@ internal MapValuesHelper(Func f) func = f; } - internal KeyValuePair Execute(KeyValuePair kvp) + internal Tuple Execute(Tuple kvp) { - return new KeyValuePair(kvp.Key, func(kvp.Value)); + return new Tuple(kvp.Item1, func(kvp.Item2)); } } @@ -483,9 +483,9 @@ internal FlatMapValuesHelper(Func> f) func = f; } - internal IEnumerable> Execute(KeyValuePair kvp) + internal IEnumerable> Execute(Tuple kvp) { - return func(kvp.Value).Select(v => new KeyValuePair(kvp.Key, v)); + return func(kvp.Item2).Select(v => new Tuple(kvp.Item1, v)); } } @@ -498,7 +498,7 @@ internal GroupByKeyHelper(int numPartitions = 0) this.numPartitions = numPartitions; } - internal RDD>> Execute(RDD> rdd) + internal RDD>> Execute(RDD> rdd) { return rdd.GroupByKey(numPartitions); } @@ -513,7 +513,7 @@ internal GroupWithHelper(int numPartitions) this.numPartitions = numPartitions; } - internal RDD, List>>> Execute(RDD> l, RDD> r) + internal RDD, List>>> Execute(RDD> l, RDD> r) { return l.GroupWith(r, numPartitions); } @@ -528,7 +528,7 @@ internal JoinHelper(int numPartitions) this.numPartitions = numPartitions; } - internal RDD>> Execute(RDD> l, RDD> r) + internal RDD>> Execute(RDD> l, RDD> r) { return l.Join(r, numPartitions); } @@ -543,7 +543,7 @@ internal LeftOuterJoinHelper(int numPartitions) this.numPartitions = numPartitions; } - internal RDD>>> Execute(RDD> l, RDD> r) + internal RDD>>> Execute(RDD> l, RDD> r) { return l.LeftOuterJoin(r, numPartitions); } @@ -558,7 +558,7 @@ internal RightOuterJoinHelper(int numPartitions) this.numPartitions = numPartitions; } - internal RDD, W>>> Execute(RDD> l, RDD> r) + internal RDD, W>>> Execute(RDD> l, RDD> r) { return l.RightOuterJoin(r, numPartitions); } @@ -573,7 +573,7 @@ internal FullOuterJoinHelper(int numPartitions) this.numPartitions = numPartitions; } - internal RDD, Option>>> Execute(RDD> l, RDD> r) + internal RDD, Option>>> Execute(RDD> l, RDD> r) { return l.FullOuterJoin(r, numPartitions); } @@ -585,13 +585,13 @@ internal class ReduceByKeyAndWindowHelper private readonly Func reduceFunc; private readonly Func invReduceFunc; private readonly int numPartitions; - private readonly Func, bool> filterFunc; + private readonly Func, bool> filterFunc; private readonly Func, RDD> prevFunc; internal ReduceByKeyAndWindowHelper(Func reduceF, Func invReduceF, int numPartitions, - Func, bool> filterF, + Func, bool> filterF, Func, RDD> prevF) { reduceFunc = reduceF; @@ -606,13 +606,13 @@ internal RDD Reduce(double t, RDD a, RDD b) if (prevFunc != null) b = prevFunc(t, b); - var r = b.ConvertTo>().ReduceByKey(reduceFunc); + var r = b.ConvertTo>().ReduceByKey(reduceFunc); if (a != null) { if (prevFunc != null) a = prevFunc(t, a); - r = a.ConvertTo>().Union(r).ReduceByKey(reduceFunc); + r = a.ConvertTo>().Union(r).ReduceByKey(reduceFunc); } if (filterFunc != null) r.Filter(filterFunc); @@ -627,8 +627,8 @@ internal RDD InvReduce(double t, RDD a, RDD b) b = prevFunc(t, b); } - var rddb = b.ConvertTo>().ReduceByKey(reduceFunc); - var rdda = a.ConvertTo>(); + var rddb = b.ConvertTo>().ReduceByKey(reduceFunc); + var rdda = a.ConvertTo>(); var joined = rdda.Join(rddb, numPartitions); var r = joined.MapValues, V>(kv => kv.Item2 != null ? invReduceFunc(kv.Item1, kv.Item2) : kv.Item1); return r.ConvertTo(); @@ -645,20 +645,20 @@ internal UpdateStateByKeyHelper(Func, S, S> f) func = f; } - internal IEnumerable> Execute(IEnumerable, S>>> input) + internal IEnumerable> Execute(IEnumerable, S>>> input) { - return input.Select(x => new KeyValuePair(x.Key, func(x.Value.Item1, x.Value.Item2))); + return input.Select(x => new Tuple(x.Item1, func(x.Item2.Item1, x.Item2.Item2))); } } [Serializable] internal class UpdateStateByKeysHelper { - private readonly Func, S>>>, IEnumerable>> func; + private readonly Func, S>>>, IEnumerable>> func; private readonly Func, RDD> prevFunc; private readonly int numPartitions; internal UpdateStateByKeysHelper( - Func, S>>>, IEnumerable>> f, + Func, S>>>, IEnumerable>> f, Func, RDD> prevF, int numPartitions) { func = f; @@ -668,13 +668,13 @@ internal UpdateStateByKeysHelper( internal RDD Execute(double t, RDD stateRDD, RDD valuesRDD) { - RDD> state = null; - RDD, S>>> g = null; + RDD> state = null; + RDD, S>>> g = null; if (prevFunc != null) valuesRDD = prevFunc(t, valuesRDD); - var values = valuesRDD.ConvertTo>(); + var values = valuesRDD.ConvertTo>(); if (stateRDD == null) { @@ -682,13 +682,13 @@ internal RDD Execute(double t, RDD stateRDD, RDD valu } else { - state = stateRDD.ConvertTo>(); + state = stateRDD.ConvertTo>(); values = values.PartitionBy(numPartitions); state.partitioner = values.partitioner; g = state.GroupWith(values, numPartitions).MapValues(x => new Tuple, S>(new List(x.Item2), x.Item1.Count > 0 ? x.Item1[0] : default(S))); } - state = g.MapPartitionsWithIndex((pid, iter) => func(pid, iter), true).Filter(x => x.Value != null); + state = g.MapPartitionsWithIndex((pid, iter) => func(pid, iter), true).Filter(x => x.Item2 != null); return state.ConvertTo(); } diff --git a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML index 4ea048c2..a68abc7e 100644 --- a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML +++ b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML @@ -407,7 +407,7 @@ a function to sort the key. - + Sorts this RDD, which is assumed to consist of KeyValuePair pairs. @@ -418,7 +418,7 @@ - + Sorts this RDD, which is assumed to consist of KeyValuePairs. If key is type of string, case is sensitive. @@ -449,16 +449,16 @@ - operations only available to KeyValuePair RDD + operations only available to Tuple RDD See also http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions - + Return the key-value pairs in this RDD to the master as a dictionary. - var m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).CollectAsMap() + var m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).CollectAsMap() m[1] 2 m[3] @@ -470,11 +470,11 @@ - + Return an RDD with the keys of each tuple. - >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Keys().Collect() + >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Keys().Collect() [1, 3] @@ -482,11 +482,11 @@ - + Return an RDD with the values of each tuple. - >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Values().Collect() + >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Values().Collect() [2, 4] @@ -495,7 +495,7 @@ - + Merge the values for each key using an associative reduce function. @@ -507,9 +507,9 @@ sc.Parallelize(new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .ReduceByKey((x, y) => x + y).Collect() @@ -523,7 +523,7 @@ - + Merge the values for each key using an associative reduce function, but return the results immediately to the master as a dictionary. @@ -533,9 +533,9 @@ sc.Parallelize(new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .ReduceByKeyLocally((x, y) => x + y).Collect() @@ -548,15 +548,15 @@ - + Count the number of elements for each key, and return the result to the master as a dictionary. sc.Parallelize(new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .CountByKey((x, y) => x + y).Collect() @@ -568,7 +568,7 @@ - + Return an RDD containing all pairs of elements with matching keys in this RDD and . @@ -577,9 +577,9 @@ Performs a hash join across the cluster. var l = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("a", 3) }, 1); + new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("a", 3) }, 1); var m = l.Join(r, 2).Collect(); [('a', (1, 2)), ('a', (1, 3))] @@ -593,7 +593,7 @@ - + Perform a left outer join of this RDD and . @@ -604,9 +604,9 @@ Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 2) }, 1); + new[] { new Tuple<string, int>("a", 2) }, 1); var m = l.LeftOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, Option))] @@ -620,7 +620,7 @@ - + Perform a right outer join of this RDD and . @@ -631,9 +631,9 @@ Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 2) }, 1); + new[] { new Tuple<string, int>("a", 2) }, 1); var r = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); + new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var m = l.RightOuterJoin(r).Collect(); [('a', (2, 1)), ('b', (Option, 4))] @@ -647,7 +647,7 @@ - + Perform a full outer join of this RDD and . @@ -662,9 +662,9 @@ Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 1), KeyValuePair<string, int>("b", 4) }, 1); + new[] { new Tuple<string, int>("a", 1), Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( - new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("c", 8) }, 1); + new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("c", 8) }, 1); var m = l.FullOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))] @@ -678,18 +678,18 @@ - + Return a copy of the RDD partitioned using the specified partitioner. - sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new KeyValuePair<int, int>(x, x)).PartitionBy(3).Glom().Collect() + sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new Tuple<int, int>(x, x)).PartitionBy(3).Glom().Collect() - + # TODO: add control over map-side aggregation Generic function to combine the elements for each key using a custom @@ -711,9 +711,9 @@ sc.Parallelize( new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() @@ -729,7 +729,7 @@ - + Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type @@ -742,9 +742,9 @@ sc.Parallelize( new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() @@ -760,7 +760,7 @@ - + Merge the values for each key using an associative function "func" and a neutral "zeroValue" which may be added to the result an @@ -770,9 +770,9 @@ sc.Parallelize( new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() @@ -786,7 +786,7 @@ - + Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with numPartitions partitions. @@ -798,9 +798,9 @@ sc.Parallelize( new[] { - new KeyValuePair<string, int>("a", 1), - new KeyValuePair<string, int>("b", 1), - new KeyValuePair<string, int>("a", 1) + new Tuple<string, int>("a", 1), + new Tuple<string, int>("b", 1), + new Tuple<string, int>("a", 1) }, 2) .GroupByKey().MapValues(l => string.Join(" ", l)).Collect() @@ -813,7 +813,7 @@ - + Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning. @@ -821,8 +821,8 @@ sc.Parallelize( new[] { - new KeyValuePair<string, string[]>("a", new[]{"apple", "banana", "lemon"}), - new KeyValuePair<string, string[]>("b", new[]{"grapes"}) + new Tuple<string, string[]>("a", new[]{"apple", "banana", "lemon"}), + new Tuple<string, string[]>("b", new[]{"grapes"}) }, 2) .MapValues(x => x.Length).Collect() @@ -836,7 +836,7 @@ - + Pass each value in the key-value pair RDD through a flatMap function without changing the keys; this also retains the original RDD's partitioning. @@ -844,8 +844,8 @@ x = sc.Parallelize( new[] { - new KeyValuePair<string, string[]>("a", new[]{"x", "y", "z"}), - new KeyValuePair<string, string[]>("b", new[]{"p", "r"}) + new Tuple<string, string[]>("a", new[]{"x", "y", "z"}), + new Tuple<string, string[]>("b", new[]{"p", "r"}) }, 2) .FlatMapValues(x => x).Collect() @@ -859,9 +859,9 @@ - + - explicitly convert KeyValuePair<K, V> to KeyValuePair<K, dynamic> + explicitly convert Tuple<K, V> to Tuple<K, dynamic> since they are incompatibles types unlike V to dynamic @@ -872,13 +872,13 @@ - + For each key k in this RDD or , return a resulting RDD that contains a tuple with the list of values for that key in this RDD as well as . - var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); + var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); x.GroupWith(y).Collect(); [('a', ([1], [2])), ('b', ([4], []))] @@ -892,11 +892,11 @@ - + - var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); - var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); + var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); + var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); x.GroupWith(y, z).Collect(); @@ -909,12 +909,12 @@ - + - var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); - var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); - var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); - var w = sc.Parallelize(new[] { new KeyValuePair<string, int>("b", 42) }, 1); + var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); + var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); + var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); + var w = sc.Parallelize(new[] { new Tuple<string, int>("b", 42) }, 1); var m = x.GroupWith(y, z, w).MapValues(l => string.Join(" ", l.Item1) + " : " + string.Join(" ", l.Item2) + " : " + string.Join(" ", l.Item3) + " : " + string.Join(" ", l.Item4)).Collect(); @@ -929,12 +929,12 @@ - + Return each (key, value) pair in this RDD that has no pair with matching key in . - var x = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 1), new KeyValuePair<string, int?>("b", 4), new KeyValuePair<string, int?>("b", 5), new KeyValuePair<string, int?>("a", 2) }, 2); - var y = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 3), new KeyValuePair<string, int?>("c", null) }, 2); + var x = sc.Parallelize(new[] { new Tuple<string, int?>("a", 1), new Tuple<string, int?>("b", 4), new Tuple<string, int?>("b", 5), new Tuple<string, int?>("a", 2) }, 2); + var y = sc.Parallelize(new[] { new Tuple<string, int?>("a", 3), new Tuple<string, int?>("c", null) }, 2); x.SubtractByKey(y).Collect(); [('b', 4), ('b', 5)] @@ -948,14 +948,14 @@ - + Return the list of values in the RDD for key `key`. This operation is done efficiently if the RDD has a known partitioner by only searching the partition that the key maps to. >>> l = range(1000) - >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new KeyValuePair<int, int>(x, y)), 10) + >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new Tuple<int, int>(x, y)), 10) >>> rdd.lookup(42) [42] @@ -966,7 +966,7 @@ - + Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are @@ -978,7 +978,7 @@ Hadoop job configuration, passed in as a dict - + @@ -991,7 +991,7 @@ fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.Text", None by default) Hadoop job configuration, passed in as a dict (None by default) - + Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Keys/values are @@ -1003,7 +1003,7 @@ Hadoop job configuration, passed in as a dict - + Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Key and value types @@ -1022,7 +1022,7 @@ (None by default) (None by default) - + Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the org.apache.hadoop.io.Writable types that we convert from the @@ -1166,7 +1166,7 @@ Return a new RDD by applying a function to each element of this RDD. - sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new KeyValuePair<string, int>(x, 1)).Collect() + sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new Tuple<string, int>(x, 1)).Collect() [('a', 1), ('b', 1), ('c', 1)] @@ -1772,7 +1772,7 @@ - + Get the N elements from a RDD ordered in ascending order or as specified by the optional key function. @@ -1784,6 +1784,7 @@ + @@ -2109,7 +2110,7 @@ minimum splits in dataset (default min(2, sc.defaultParallelism)) - + Read a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -2127,7 +2128,7 @@ Hadoop configuration, passed in as a dict (None by default) - + Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -2143,7 +2144,7 @@ Hadoop configuration, passed in as a dict (None by default) - + Read an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -2161,7 +2162,7 @@ Hadoop configuration, passed in as a dict (None by default) - + Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -4841,6 +4842,16 @@ The Json object used to construct a StructType A new StructType instance + + + An input stream that always returns the same RDD on each timestep. Useful for testing. + + + + + Construct a ConstantInputDStream instance. + + A Discretized Stream (DStream), the basic abstraction in Spark Streaming, @@ -5137,7 +5148,7 @@ Utility for creating streams from - + Create a unioned EventHubs stream that receives data from Microsoft Azure Eventhubs The unioned stream will receive message from all partitions of the EventHubs @@ -5167,7 +5178,7 @@ Utils for Kafka input stream. - + Create an input stream that pulls messages from a Kafka Broker. @@ -5178,7 +5189,7 @@ Additional params for Kafka A DStream object - + Create an input stream that pulls messages from a Kafka Broker. @@ -5190,7 +5201,7 @@ RDD storage level. A DStream object - + Create an input stream that directly pulls messages from a Kafka Broker and specific offset. @@ -5216,7 +5227,7 @@ Per-topic/partition Kafka offsets defining the (inclusive) starting point of the stream. A DStream object - + Create an input stream that directly pulls messages from a Kafka Broker and specific offset. @@ -5331,10 +5342,10 @@ - operations only available to KeyValuePair RDD + operations only available to Tuple RDD - + Return a new DStream by applying ReduceByKey to each RDD. @@ -5345,7 +5356,7 @@ - + Return a new DStream by applying combineByKey to each RDD. @@ -5359,7 +5370,7 @@ - + Return a new DStream in which each RDD are partitioned by numPartitions. @@ -5369,7 +5380,7 @@ - + Return a new DStream by applying a map function to the value of each key-value pairs in this DStream without changing the key. @@ -5381,7 +5392,7 @@ - + Return a new DStream by applying a flatmap function to the value of each key-value pairs in this DStream without changing the key. @@ -5393,7 +5404,7 @@ - + Return a new DStream by applying groupByKey on each RDD. @@ -5403,7 +5414,7 @@ - + Return a new DStream by applying 'cogroup' between RDDs of this DStream and `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions. @@ -5416,7 +5427,7 @@ - + Return a new DStream by applying 'join' between RDDs of this DStream and `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions. @@ -5429,7 +5440,7 @@ - + Return a new DStream by applying 'left outer join' between RDDs of this DStream and `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions. @@ -5442,7 +5453,7 @@ - + Return a new DStream by applying 'right outer join' between RDDs of this DStream and `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions. @@ -5455,7 +5466,7 @@ - + Return a new DStream by applying 'full outer join' between RDDs of this DStream and `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions. @@ -5468,7 +5479,7 @@ - + Return a new DStream by applying `GroupByKey` over a sliding window. Similar to `DStream.GroupByKey()`, but applies it over a sliding window. @@ -5485,7 +5496,7 @@ Number of partitions of each RDD in the new DStream. - + Return a new DStream by applying incremental `reduceByKey` over a sliding window. @@ -5506,7 +5517,7 @@ function to filter expired key-value pairs; only pairs that satisfy the function are retained set this to null if you do not want to filter - + Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. @@ -5522,7 +5533,7 @@ - + Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. @@ -5535,7 +5546,7 @@ - + Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. @@ -5548,7 +5559,7 @@ - + Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of the key. diff --git a/csharp/Adapter/documentation/Mobius_API_Documentation.md b/csharp/Adapter/documentation/Mobius_API_Documentation.md index 71bfc5d1..5d02396d 100644 --- a/csharp/Adapter/documentation/Mobius_API_Documentation.md +++ b/csharp/Adapter/documentation/Mobius_API_Documentation.md @@ -182,14 +182,14 @@ ####Summary - operations only available to KeyValuePair RDD + operations only available to Tuple RDD See also http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions ####Methods -
NameDescription
CollectAsMap``2Return the key-value pairs in this RDD to the master as a dictionary. var m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).CollectAsMap() m[1] 2 m[3] 4
Keys``2Return an RDD with the keys of each tuple. >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Keys().Collect() [1, 3]
Values``2Return an RDD with the values of each tuple. >>> m = sc.Parallelize(new[] { new KeyValuePair<int, int>(1, 2), new KeyValuePair<int, int>(3, 4) }, 1).Values().Collect() [2, 4]
ReduceByKey``2Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. Output will be hash-partitioned with partitions, or the default parallelism level if is not specified. sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .ReduceByKey((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
ReduceByKeyLocally``2Merge the values for each key using an associative reduce function, but return the results immediately to the master as a dictionary. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .ReduceByKeyLocally((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
CountByKey``2Count the number of elements for each key, and return the result to the master as a dictionary. sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .CountByKey((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
Join``3Return an RDD containing all pairs of elements with matching keys in this RDD and . Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in this RDD and (k, v2) is in . Performs a hash join across the cluster. var l = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("a", 3) }, 1); var m = l.Join(r, 2).Collect(); [('a', (1, 2)), ('a', (1, 3))]
LeftOuterJoin``3Perform a left outer join of this RDD and . For each element (k, v) in this RDD, the resulting RDD will either contain all pairs (k, (v, Option)) for w in , where Option.IsDefined is TRUE, or the pair (k, (v, Option)) if no elements in have key k, where Option.IsDefined is FALSE. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 2) }, 1); var m = l.LeftOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, Option))] * Option.IsDefined = FALSE
RightOuterJoin``3Perform a right outer join of this RDD and . For each element (k, w) in , the resulting RDD will either contain all pairs (k, (Option, w)) for v in this, where Option.IsDefined is TRUE, or the pair (k, (Option, w)) if no elements in this RDD have key k, where Option.IsDefined is FALSE. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 2) }, 1); var r = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 1); var m = l.RightOuterJoin(r).Collect(); [('a', (2, 1)), ('b', (Option, 4))] * Option.IsDefined = FALSE
FullOuterJoin``3Perform a full outer join of this RDD and . For each element (k, v) in this RDD, the resulting RDD will either contain all pairs (k, (v, w)) for w in , or the pair (k, (v, None)) if no elements in have key k. Similarly, for each element (k, w) in , the resulting RDD will either contain all pairs (k, (v, w)) for v in this RDD, or the pair (k, (None, w)) if no elements in this RDD have key k. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), KeyValuePair<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 2), new KeyValuePair<string, int>("c", 8) }, 1); var m = l.FullOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))]
PartitionBy``2Return a copy of the RDD partitioned using the specified partitioner. sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new KeyValuePair<int, int>(x, x)).PartitionBy(3).Glom().Collect()
CombineByKey``3# TODO: add control over map-side aggregation Generic function to combine the elements for each key using a custom set of aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C. Note that V and C can be different -- for example, one might group an RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three functions: - , which turns a V into a C (e.g., creates a one-element list) - , to merge a V into a C (e.g., adds it to the end of a list) - , to combine two C's into a single one. In addition, users can control the partitioning of the output RDD. sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', '11'), ('b', '1')]
AggregateByKey``3Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of the values in this RDD, V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, The former operation is used for merging values within a partition, and the latter is used for merging values between partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U. sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', 2), ('b', 1)]
FoldByKey``2Merge the values for each key using an associative function "func" and a neutral "zeroValue" which may be added to the result an arbitrary number of times, and must not change the result (e.g., 0 for addition, or 1 for multiplication.). sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', 2), ('b', 1)]
GroupByKey``2Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with numPartitions partitions. Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. sc.Parallelize( new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 1), new KeyValuePair<string, int>("a", 1) }, 2) .GroupByKey().MapValues(l => string.Join(" ", l)).Collect() [('a', [1, 1]), ('b', [1])]
MapValues``3Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning. sc.Parallelize( new[] { new KeyValuePair<string, string[]>("a", new[]{"apple", "banana", "lemon"}), new KeyValuePair<string, string[]>("b", new[]{"grapes"}) }, 2) .MapValues(x => x.Length).Collect() [('a', 3), ('b', 1)]
FlatMapValues``3Pass each value in the key-value pair RDD through a flatMap function without changing the keys; this also retains the original RDD's partitioning. x = sc.Parallelize( new[] { new KeyValuePair<string, string[]>("a", new[]{"x", "y", "z"}), new KeyValuePair<string, string[]>("b", new[]{"p", "r"}) }, 2) .FlatMapValues(x => x).Collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')]
MapPartitionsWithIndex``5explicitly convert KeyValuePair<K, V> to KeyValuePair<K, dynamic> since they are incompatibles types unlike V to dynamic
GroupWith``3For each key k in this RDD or , return a resulting RDD that contains a tuple with the list of values for that key in this RDD as well as . var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); x.GroupWith(y).Collect(); [('a', ([1], [2])), ('b', ([4], []))]
GroupWith``4var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); x.GroupWith(y, z).Collect();
GroupWith``5var x = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 5), new KeyValuePair<string, int>("b", 6) }, 2); var y = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 1), new KeyValuePair<string, int>("b", 4) }, 2); var z = sc.Parallelize(new[] { new KeyValuePair<string, int>("a", 2) }, 1); var w = sc.Parallelize(new[] { new KeyValuePair<string, int>("b", 42) }, 1); var m = x.GroupWith(y, z, w).MapValues(l => string.Join(" ", l.Item1) + " : " + string.Join(" ", l.Item2) + " : " + string.Join(" ", l.Item3) + " : " + string.Join(" ", l.Item4)).Collect();
SubtractByKey``3Return each (key, value) pair in this RDD that has no pair with matching key in . var x = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 1), new KeyValuePair<string, int?>("b", 4), new KeyValuePair<string, int?>("b", 5), new KeyValuePair<string, int?>("a", 2) }, 2); var y = sc.Parallelize(new[] { new KeyValuePair<string, int?>("a", 3), new KeyValuePair<string, int?>("c", null) }, 2); x.SubtractByKey(y).Collect(); [('b', 4), ('b', 5)]
Lookup``2Return the list of values in the RDD for key `key`. This operation is done efficiently if the RDD has a known partitioner by only searching the partition that the key maps to. >>> l = range(1000) >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new KeyValuePair<int, int>(x, y)), 10) >>> rdd.lookup(42) [42]
SaveAsNewAPIHadoopDataset``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are converted for output using either user specified converters or, by default, org.apache.spark.api.python.JavaToWritableConverter.
SaveAsNewAPIHadoopFile``2
SaveAsHadoopDataset``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Keys/values are converted for output using either user specified converters or, by default, org.apache.spark.api.python.JavaToWritableConverter.
SaveAsHadoopFile``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Key and value types will be inferred if not specified. Keys and values are converted for output using either user specified converters or org.apache.spark.api.python.JavaToWritableConverter. The is applied on top of the base Hadoop conf associated with the SparkContext of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
SaveAsSequenceFile``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the org.apache.hadoop.io.Writable types that we convert from the RDD's key and value types. The mechanism is as follows: 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. 2. Keys and values of this Java RDD are converted to Writables and written out.
NullIfEmpty``1Converts a collection to a list where the element type is Option(T) type. If the collection is empty, just returns the empty list.
+
NameDescription
CollectAsMap``2Return the key-value pairs in this RDD to the master as a dictionary. var m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).CollectAsMap() m[1] 2 m[3] 4
Keys``2Return an RDD with the keys of each tuple. >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Keys().Collect() [1, 3]
Values``2Return an RDD with the values of each tuple. >>> m = sc.Parallelize(new[] { new Tuple<int, int>(1, 2), new Tuple<int, int>(3, 4) }, 1).Values().Collect() [2, 4]
ReduceByKey``2Merge the values for each key using an associative reduce function. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. Output will be hash-partitioned with partitions, or the default parallelism level if is not specified. sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .ReduceByKey((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
ReduceByKeyLocally``2Merge the values for each key using an associative reduce function, but return the results immediately to the master as a dictionary. This will also perform the merging locally on each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .ReduceByKeyLocally((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
CountByKey``2Count the number of elements for each key, and return the result to the master as a dictionary. sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .CountByKey((x, y) => x + y).Collect() [('a', 2), ('b', 1)]
Join``3Return an RDD containing all pairs of elements with matching keys in this RDD and . Each pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in this RDD and (k, v2) is in . Performs a hash join across the cluster. var l = sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("a", 3) }, 1); var m = l.Join(r, 2).Collect(); [('a', (1, 2)), ('a', (1, 3))]
LeftOuterJoin``3Perform a left outer join of this RDD and . For each element (k, v) in this RDD, the resulting RDD will either contain all pairs (k, (v, Option)) for w in , where Option.IsDefined is TRUE, or the pair (k, (v, Option)) if no elements in have key k, where Option.IsDefined is FALSE. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new Tuple<string, int>("a", 2) }, 1); var m = l.LeftOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, Option))] * Option.IsDefined = FALSE
RightOuterJoin``3Perform a right outer join of this RDD and . For each element (k, w) in , the resulting RDD will either contain all pairs (k, (Option, w)) for v in this, where Option.IsDefined is TRUE, or the pair (k, (Option, w)) if no elements in this RDD have key k, where Option.IsDefined is FALSE. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new Tuple<string, int>("a", 2) }, 1); var r = sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 1); var m = l.RightOuterJoin(r).Collect(); [('a', (2, 1)), ('b', (Option, 4))] * Option.IsDefined = FALSE
FullOuterJoin``3Perform a full outer join of this RDD and . For each element (k, v) in this RDD, the resulting RDD will either contain all pairs (k, (v, w)) for w in , or the pair (k, (v, None)) if no elements in have key k. Similarly, for each element (k, w) in , the resulting RDD will either contain all pairs (k, (v, w)) for v in this RDD, or the pair (k, (None, w)) if no elements in this RDD have key k. Hash-partitions the resulting RDD into the given number of partitions. var l = sc.Parallelize( new[] { new Tuple<string, int>("a", 1), Tuple<string, int>("b", 4) }, 1); var r = sc.Parallelize( new[] { new Tuple<string, int>("a", 2), new Tuple<string, int>("c", 8) }, 1); var m = l.FullOuterJoin(r).Collect(); [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))]
PartitionBy``2Return a copy of the RDD partitioned using the specified partitioner. sc.Parallelize(new[] { 1, 2, 3, 4, 2, 4, 1 }, 1).Map(x => new Tuple<int, int>(x, x)).PartitionBy(3).Glom().Collect()
CombineByKey``3# TODO: add control over map-side aggregation Generic function to combine the elements for each key using a custom set of aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C. Note that V and C can be different -- for example, one might group an RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three functions: - , which turns a V into a C (e.g., creates a one-element list) - , to merge a V into a C (e.g., adds it to the end of a list) - , to combine two C's into a single one. In addition, users can control the partitioning of the output RDD. sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', '11'), ('b', '1')]
AggregateByKey``3Aggregate the values of each key, using given combine functions and a neutral "zero value". This function can return a different result type, U, than the type of the values in this RDD, V. Thus, we need one operation for merging a V into a U and one operation for merging two U's, The former operation is used for merging values within a partition, and the latter is used for merging values between partitions. To avoid memory allocation, both of these functions are allowed to modify and return their first argument instead of creating a new U. sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', 2), ('b', 1)]
FoldByKey``2Merge the values for each key using an associative function "func" and a neutral "zeroValue" which may be added to the result an arbitrary number of times, and must not change the result (e.g., 0 for addition, or 1 for multiplication.). sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(), (x, y) => x + y).Collect() [('a', 2), ('b', 1)]
GroupByKey``2Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with numPartitions partitions. Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. sc.Parallelize( new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 1), new Tuple<string, int>("a", 1) }, 2) .GroupByKey().MapValues(l => string.Join(" ", l)).Collect() [('a', [1, 1]), ('b', [1])]
MapValues``3Pass each value in the key-value pair RDD through a map function without changing the keys; this also retains the original RDD's partitioning. sc.Parallelize( new[] { new Tuple<string, string[]>("a", new[]{"apple", "banana", "lemon"}), new Tuple<string, string[]>("b", new[]{"grapes"}) }, 2) .MapValues(x => x.Length).Collect() [('a', 3), ('b', 1)]
FlatMapValues``3Pass each value in the key-value pair RDD through a flatMap function without changing the keys; this also retains the original RDD's partitioning. x = sc.Parallelize( new[] { new Tuple<string, string[]>("a", new[]{"x", "y", "z"}), new Tuple<string, string[]>("b", new[]{"p", "r"}) }, 2) .FlatMapValues(x => x).Collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')]
MapPartitionsWithIndex``5explicitly convert Tuple<K, V> to Tuple<K, dynamic> since they are incompatibles types unlike V to dynamic
GroupWith``3For each key k in this RDD or , return a resulting RDD that contains a tuple with the list of values for that key in this RDD as well as . var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); x.GroupWith(y).Collect(); [('a', ([1], [2])), ('b', ([4], []))]
GroupWith``4var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); x.GroupWith(y, z).Collect();
GroupWith``5var x = sc.Parallelize(new[] { new Tuple<string, int>("a", 5), new Tuple<string, int>("b", 6) }, 2); var y = sc.Parallelize(new[] { new Tuple<string, int>("a", 1), new Tuple<string, int>("b", 4) }, 2); var z = sc.Parallelize(new[] { new Tuple<string, int>("a", 2) }, 1); var w = sc.Parallelize(new[] { new Tuple<string, int>("b", 42) }, 1); var m = x.GroupWith(y, z, w).MapValues(l => string.Join(" ", l.Item1) + " : " + string.Join(" ", l.Item2) + " : " + string.Join(" ", l.Item3) + " : " + string.Join(" ", l.Item4)).Collect();
SubtractByKey``3Return each (key, value) pair in this RDD that has no pair with matching key in . var x = sc.Parallelize(new[] { new Tuple<string, int?>("a", 1), new Tuple<string, int?>("b", 4), new Tuple<string, int?>("b", 5), new Tuple<string, int?>("a", 2) }, 2); var y = sc.Parallelize(new[] { new Tuple<string, int?>("a", 3), new Tuple<string, int?>("c", null) }, 2); x.SubtractByKey(y).Collect(); [('b', 4), ('b', 5)]
Lookup``2Return the list of values in the RDD for key `key`. This operation is done efficiently if the RDD has a known partitioner by only searching the partition that the key maps to. >>> l = range(1000) >>> rdd = sc.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new Tuple<int, int>(x, y)), 10) >>> rdd.lookup(42) [42]
SaveAsNewAPIHadoopDataset``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are converted for output using either user specified converters or, by default, org.apache.spark.api.python.JavaToWritableConverter.
SaveAsNewAPIHadoopFile``2
SaveAsHadoopDataset``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Keys/values are converted for output using either user specified converters or, by default, org.apache.spark.api.python.JavaToWritableConverter.
SaveAsHadoopFile``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the old Hadoop OutputFormat API (mapred package). Key and value types will be inferred if not specified. Keys and values are converted for output using either user specified converters or org.apache.spark.api.python.JavaToWritableConverter. The is applied on top of the base Hadoop conf associated with the SparkContext of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.
SaveAsSequenceFile``2Output a Python RDD of key-value pairs (of form RDD[(K, V)]) to any Hadoop file system, using the org.apache.hadoop.io.Writable types that we convert from the RDD's key and value types. The mechanism is as follows: 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. 2. Keys and values of this Java RDD are converted to Writables and written out.
NullIfEmpty``1Converts a collection to a list where the element type is Option(T) type. If the collection is empty, just returns the empty list.
--- @@ -230,7 +230,7 @@ ####Methods -
NameDescription
CachePersist this RDD with the default storage level .
PersistSet this RDD's storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. If no storage level is specified defaults to . sc.Parallelize(new string[] {"b", "a", "c").Persist().isCached True
UnpersistMark the RDD as non-persistent, and remove all blocks for it from memory and disk.
CheckpointMark this RDD for checkpointing. It will be saved to a file inside the checkpoint directory set with ) and all references to its parent RDDs will be removed. This function must be called before any job has been executed on this RDD. It is strongly recommended that this RDD is persisted in memory, otherwise saving it on a file will require recomputation.
GetNumPartitionsReturns the number of partitions of this RDD.
Map``1Return a new RDD by applying a function to each element of this RDD. sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new KeyValuePair<string, int>(x, 1)).Collect() [('a', 1), ('b', 1), ('c', 1)]
FlatMap``1Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results. sc.Parallelize(new int[] {2, 3, 4}, 1).FlatMap(x => Enumerable.Range(1, x - 1)).Collect() [1, 1, 1, 2, 2, 3]
MapPartitions``1Return a new RDD by applying a function to each partition of this RDD. sc.Parallelize(new int[] {1, 2, 3, 4}, 2).MapPartitions(iter => new[]{iter.Sum(x => (x as decimal?))}).Collect() [3, 7]
MapPartitionsWithIndex``1Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition. sc.Parallelize(new int[]{1, 2, 3, 4}, 4).MapPartitionsWithIndex<double>((pid, iter) => (double)pid).Sum() 6
FilterReturn a new RDD containing only the elements that satisfy a predicate. sc.Parallelize(new int[]{1, 2, 3, 4, 5}, 1).Filter(x => x % 2 == 0).Collect() [2, 4]
DistinctReturn a new RDD containing the distinct elements in this RDD. >>> sc.Parallelize(new int[] {1, 1, 2, 3}, 1).Distinct().Collect() [1, 2, 3]
SampleReturn a sampled subset of this RDD. var rdd = sc.Parallelize(Enumerable.Range(0, 100), 4) 6 <= rdd.Sample(False, 0.1, 81).count() <= 14 true
RandomSplitRandomly splits this RDD with the provided weights. var rdd = sc.Parallelize(Enumerable.Range(0, 500), 1) var rdds = rdd.RandomSplit(new double[] {2, 3}, 17) 150 < rdds[0].Count() < 250 250 < rdds[1].Count() < 350
TakeSampleReturn a fixed-size sampled subset of this RDD. var rdd = sc.Parallelize(Enumerable.Range(0, 10), 2) rdd.TakeSample(true, 20, 1).Length 20 rdd.TakeSample(false, 5, 2).Length 5 rdd.TakeSample(false, 15, 3).Length 10
ComputeFractionForSampleSizeReturns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of the time. How the sampling rate is determined: Let p = num / total, where num is the sample size and total is the total number of data points in the RDD. We're trying to compute q > p such that - when sampling with replacement, we're drawing each data point with prob_i ~ Pois(q), where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total), i.e. the failure rate of not having a sufficiently large sample < 0.0001. Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for num > 12, but we need a slightly larger q (9 empirically determined). - when sampling without replacement, we're drawing each data point with prob_i ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success rate, where success rate is defined the same as in sampling with replacement.
UnionReturn the union of this RDD and another one. var rdd = sc.Parallelize(new int[] { 1, 1, 2, 3 }, 1) rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3]
IntersectionReturn the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did. Note that this method performs a shuffle internally. var rdd1 = sc.Parallelize(new int[] { 1, 10, 2, 3, 4, 5 }, 1) var rdd2 = sc.Parallelize(new int[] { 1, 6, 2, 3, 7, 8 }, 1) var rdd1.Intersection(rdd2).Collect() [1, 2, 3]
GlomReturn an RDD created by coalescing all elements within each partition into a list. var rdd = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 2) rdd.Glom().Collect() [[1, 2], [3, 4]]
Cartesian``1Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of elements (a, b) where a is in self and b is in other. rdd = sc.Parallelize(new int[] { 1, 2 }, 1) rdd.Cartesian(rdd).Collect() [(1, 1), (1, 2), (2, 1), (2, 2)]
GroupBy``1Return an RDD of grouped items. Each group consists of a key and a sequence of elements mapping to that key. The ordering of elements within each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. Note: This operation may be very expensive. If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] or [[PairRDDFunctions.reduceByKey]] will provide much better performance. >>> rdd = sc.Parallelize(new int[] { 1, 1, 2, 3, 5, 8 }, 1) >>> result = rdd.GroupBy(lambda x: x % 2).Collect() [(0, [2, 8]), (1, [1, 1, 3, 5])]
PipeReturn an RDD created by piping elements to a forked external process. >>> sc.Parallelize(new char[] { '1', '2', '3', '4' }, 1).Pipe("cat").Collect() [u'1', u'2', u'3', u'4']
ForeachApplies a function to all elements of this RDD. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).Foreach(x => Console.Write(x))
ForeachPartitionApplies a function to each partition of this RDD. sc.parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).ForeachPartition(iter => { foreach (var x in iter) Console.Write(x + " "); })
CollectReturn a list that contains all of the elements in this RDD.
ReduceReduces the elements of this RDD using the specified commutative and associative binary operator. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).Reduce((x, y) => x + y) 15
TreeReduceReduces the elements of this RDD in a multi-level tree pattern. >>> add = lambda x, y: x + y >>> rdd = sc.Parallelize(new int[] { -5, -4, -3, -2, -1, 1, 2, 3, 4 }, 10).TreeReduce((x, y) => x + y)) >>> rdd.TreeReduce(add) -5 >>> rdd.TreeReduce(add, 1) -5 >>> rdd.TreeReduce(add, 2) -5 >>> rdd.TreeReduce(add, 5) -5 >>> rdd.TreeReduce(add, 10) -5
FoldAggregate the elements of each partition, and then the results for all the partitions, using a given associative and commutative function and a neutral "zero value." The function op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object allocation; however, it should not modify t2. This behaves somewhat differently from fold operations implemented for non-distributed collections in functional languages like Scala. This fold operation may be applied to partitions individually, and then fold those results into the final result, rather than apply the fold to each element sequentially in some defined ordering. For functions that are not commutative, the result may differ from that of a fold applied to a non-distributed collection. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).fold(0, add) 15
Aggregate``1Aggregate the elements of each partition, and then the results for all the partitions, using a given combine functions and a neutral "zero value." The functions op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object allocation; however, it should not modify t2. The first function (seqOp) can return a different result type, U, than the type of this RDD. Thus, we need one operation for merging a T into an U and one operation for merging two U >>> sc.parallelize(new int[] { 1, 2, 3, 4 }, 1).Aggregate(0, (x, y) => x + y, (x, y) => x + y)) 10
TreeAggregate``1Aggregates the elements of this RDD in a multi-level tree pattern. rdd = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1).TreeAggregate(0, (x, y) => x + y, (x, y) => x + y)) 10
CountReturn the number of elements in this RDD.
CountByValueReturn the count of each unique value in this RDD as a dictionary of (value, count) pairs. sc.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2).CountByValue()) [(1, 2), (2, 3)]
TakeTake the first num elements of the RDD. It works by first scanning one partition, and use the results from that partition to estimate the number of additional partitions needed to satisfy the limit. Translated from the Scala implementation in RDD#take(). sc.Parallelize(new int[] { 2, 3, 4, 5, 6 }, 2).Cache().Take(2))) [2, 3] sc.Parallelize(new int[] { 2, 3, 4, 5, 6 }, 2).Take(10) [2, 3, 4, 5, 6] sc.Parallelize(Enumerable.Range(0, 100), 100).Filter(x => x > 90).Take(3) [91, 92, 93]
FirstReturn the first element in this RDD. >>> sc.Parallelize(new int[] { 2, 3, 4 }, 2).First() 2
IsEmptyReturns true if and only if the RDD contains no elements at all. Note that an RDD may be empty even when it has at least 1 partition. sc.Parallelize(new int[0], 1).isEmpty() true sc.Parallelize(new int[] {1}).isEmpty() false
SubtractReturn each value in this RDD that is not contained in . var x = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1) var y = sc.Parallelize(new int[] { 3 }, 1) x.Subtract(y).Collect()) [1, 2, 4]
KeyBy``1Creates tuples of the elements in this RDD by applying . sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1).KeyBy(x => x * x).Collect()) (1, 1), (4, 2), (9, 3), (16, 4)
RepartitionReturn a new RDD that has exactly numPartitions partitions. Can increase or decrease the level of parallelism in this RDD. Internally, this uses a shuffle to redistribute data. If you are decreasing the number of partitions in this RDD, consider using `Coalesce`, which can avoid performing a shuffle. var rdd = sc.Parallelize(new int[] { 1, 2, 3, 4, 5, 6, 7 }, 4) rdd.Glom().Collect().Length 4 rdd.Repartition(2).Glom().Collect().Length 2
CoalesceReturn a new RDD that is reduced into `numPartitions` partitions. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 3).Glom().Collect().Length 3 >>> sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 3).Coalesce(1).Glom().Collect().Length 1
Zip``1Zips this RDD with another one, returning key-value pairs with the first element in each RDD second element in each RDD, etc. Assumes that the two RDDs have the same number of partitions and the same number of elements in each partition (e.g. one was made through a map on the other). var x = sc.parallelize(range(0,5)) var y = sc.parallelize(range(1000, 1005)) x.Zip(y).Collect() [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)]
ZipWithIndexZips this RDD with its element indices. The ordering is first based on the partition index and then the ordering of items within each partition. So the first item in the first partition gets index 0, and the last item in the last partition receives the largest index. This method needs to trigger a spark job when this RDD contains more than one partitions. sc.Parallelize(new string[] { "a", "b", "c", "d" }, 3).ZipWithIndex().Collect() [('a', 0), ('b', 1), ('c', 2), ('d', 3)]
ZipWithUniqueIdZips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method won't trigger a spark job, which is different from >>> sc.Parallelize(new string[] { "a", "b", "c", "d" }, 1).ZipWithIndex().Collect() [('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)]
SetNameAssign a name to this RDD. >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() u'RDD1'
ToDebugStringA description of this RDD and its recursive dependencies for debugging.
GetStorageLevelGet the RDD's current storage level. >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.getStorageLevel() StorageLevel(False, False, False, False, 1) >>> print(rdd1.getStorageLevel()) Serialized 1x Replicated
ToLocalIteratorReturn an iterator that contains all of the elements in this RDD. The iterator will consume as much memory as the largest partition in this RDD. sc.Parallelize(Enumerable.Range(0, 10), 1).ToLocalIterator() [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
RandomSampleWithRangeInternal method exposed for Random Splits in DataFrames. Samples an RDD given a probability range.
+
NameDescription
CachePersist this RDD with the default storage level .
PersistSet this RDD's storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. If no storage level is specified defaults to . sc.Parallelize(new string[] {"b", "a", "c").Persist().isCached True
UnpersistMark the RDD as non-persistent, and remove all blocks for it from memory and disk.
CheckpointMark this RDD for checkpointing. It will be saved to a file inside the checkpoint directory set with ) and all references to its parent RDDs will be removed. This function must be called before any job has been executed on this RDD. It is strongly recommended that this RDD is persisted in memory, otherwise saving it on a file will require recomputation.
GetNumPartitionsReturns the number of partitions of this RDD.
Map``1Return a new RDD by applying a function to each element of this RDD. sc.Parallelize(new string[]{"b", "a", "c"}, 1).Map(x => new Tuple<string, int>(x, 1)).Collect() [('a', 1), ('b', 1), ('c', 1)]
FlatMap``1Return a new RDD by first applying a function to all elements of this RDD, and then flattening the results. sc.Parallelize(new int[] {2, 3, 4}, 1).FlatMap(x => Enumerable.Range(1, x - 1)).Collect() [1, 1, 1, 2, 2, 3]
MapPartitions``1Return a new RDD by applying a function to each partition of this RDD. sc.Parallelize(new int[] {1, 2, 3, 4}, 2).MapPartitions(iter => new[]{iter.Sum(x => (x as decimal?))}).Collect() [3, 7]
MapPartitionsWithIndex``1Return a new RDD by applying a function to each partition of this RDD, while tracking the index of the original partition. sc.Parallelize(new int[]{1, 2, 3, 4}, 4).MapPartitionsWithIndex<double>((pid, iter) => (double)pid).Sum() 6
FilterReturn a new RDD containing only the elements that satisfy a predicate. sc.Parallelize(new int[]{1, 2, 3, 4, 5}, 1).Filter(x => x % 2 == 0).Collect() [2, 4]
DistinctReturn a new RDD containing the distinct elements in this RDD. >>> sc.Parallelize(new int[] {1, 1, 2, 3}, 1).Distinct().Collect() [1, 2, 3]
SampleReturn a sampled subset of this RDD. var rdd = sc.Parallelize(Enumerable.Range(0, 100), 4) 6 <= rdd.Sample(False, 0.1, 81).count() <= 14 true
RandomSplitRandomly splits this RDD with the provided weights. var rdd = sc.Parallelize(Enumerable.Range(0, 500), 1) var rdds = rdd.RandomSplit(new double[] {2, 3}, 17) 150 < rdds[0].Count() < 250 250 < rdds[1].Count() < 350
TakeSampleReturn a fixed-size sampled subset of this RDD. var rdd = sc.Parallelize(Enumerable.Range(0, 10), 2) rdd.TakeSample(true, 20, 1).Length 20 rdd.TakeSample(false, 5, 2).Length 5 rdd.TakeSample(false, 15, 3).Length 10
ComputeFractionForSampleSizeReturns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of the time. How the sampling rate is determined: Let p = num / total, where num is the sample size and total is the total number of data points in the RDD. We're trying to compute q > p such that - when sampling with replacement, we're drawing each data point with prob_i ~ Pois(q), where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total), i.e. the failure rate of not having a sufficiently large sample < 0.0001. Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for num > 12, but we need a slightly larger q (9 empirically determined). - when sampling without replacement, we're drawing each data point with prob_i ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success rate, where success rate is defined the same as in sampling with replacement.
UnionReturn the union of this RDD and another one. var rdd = sc.Parallelize(new int[] { 1, 1, 2, 3 }, 1) rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3]
IntersectionReturn the intersection of this RDD and another one. The output will not contain any duplicate elements, even if the input RDDs did. Note that this method performs a shuffle internally. var rdd1 = sc.Parallelize(new int[] { 1, 10, 2, 3, 4, 5 }, 1) var rdd2 = sc.Parallelize(new int[] { 1, 6, 2, 3, 7, 8 }, 1) var rdd1.Intersection(rdd2).Collect() [1, 2, 3]
GlomReturn an RDD created by coalescing all elements within each partition into a list. var rdd = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 2) rdd.Glom().Collect() [[1, 2], [3, 4]]
Cartesian``1Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of elements (a, b) where a is in self and b is in other. rdd = sc.Parallelize(new int[] { 1, 2 }, 1) rdd.Cartesian(rdd).Collect() [(1, 1), (1, 2), (2, 1), (2, 2)]
GroupBy``1Return an RDD of grouped items. Each group consists of a key and a sequence of elements mapping to that key. The ordering of elements within each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. Note: This operation may be very expensive. If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] or [[PairRDDFunctions.reduceByKey]] will provide much better performance. >>> rdd = sc.Parallelize(new int[] { 1, 1, 2, 3, 5, 8 }, 1) >>> result = rdd.GroupBy(lambda x: x % 2).Collect() [(0, [2, 8]), (1, [1, 1, 3, 5])]
PipeReturn an RDD created by piping elements to a forked external process. >>> sc.Parallelize(new char[] { '1', '2', '3', '4' }, 1).Pipe("cat").Collect() [u'1', u'2', u'3', u'4']
ForeachApplies a function to all elements of this RDD. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).Foreach(x => Console.Write(x))
ForeachPartitionApplies a function to each partition of this RDD. sc.parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).ForeachPartition(iter => { foreach (var x in iter) Console.Write(x + " "); })
CollectReturn a list that contains all of the elements in this RDD.
ReduceReduces the elements of this RDD using the specified commutative and associative binary operator. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 1).Reduce((x, y) => x + y) 15
TreeReduceReduces the elements of this RDD in a multi-level tree pattern. >>> add = lambda x, y: x + y >>> rdd = sc.Parallelize(new int[] { -5, -4, -3, -2, -1, 1, 2, 3, 4 }, 10).TreeReduce((x, y) => x + y)) >>> rdd.TreeReduce(add) -5 >>> rdd.TreeReduce(add, 1) -5 >>> rdd.TreeReduce(add, 2) -5 >>> rdd.TreeReduce(add, 5) -5 >>> rdd.TreeReduce(add, 10) -5
FoldAggregate the elements of each partition, and then the results for all the partitions, using a given associative and commutative function and a neutral "zero value." The function op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object allocation; however, it should not modify t2. This behaves somewhat differently from fold operations implemented for non-distributed collections in functional languages like Scala. This fold operation may be applied to partitions individually, and then fold those results into the final result, rather than apply the fold to each element sequentially in some defined ordering. For functions that are not commutative, the result may differ from that of a fold applied to a non-distributed collection. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).fold(0, add) 15
Aggregate``1Aggregate the elements of each partition, and then the results for all the partitions, using a given combine functions and a neutral "zero value." The functions op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object allocation; however, it should not modify t2. The first function (seqOp) can return a different result type, U, than the type of this RDD. Thus, we need one operation for merging a T into an U and one operation for merging two U >>> sc.parallelize(new int[] { 1, 2, 3, 4 }, 1).Aggregate(0, (x, y) => x + y, (x, y) => x + y)) 10
TreeAggregate``1Aggregates the elements of this RDD in a multi-level tree pattern. rdd = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1).TreeAggregate(0, (x, y) => x + y, (x, y) => x + y)) 10
CountReturn the number of elements in this RDD.
CountByValueReturn the count of each unique value in this RDD as a dictionary of (value, count) pairs. sc.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2).CountByValue()) [(1, 2), (2, 3)]
TakeTake the first num elements of the RDD. It works by first scanning one partition, and use the results from that partition to estimate the number of additional partitions needed to satisfy the limit. Translated from the Scala implementation in RDD#take(). sc.Parallelize(new int[] { 2, 3, 4, 5, 6 }, 2).Cache().Take(2))) [2, 3] sc.Parallelize(new int[] { 2, 3, 4, 5, 6 }, 2).Take(10) [2, 3, 4, 5, 6] sc.Parallelize(Enumerable.Range(0, 100), 100).Filter(x => x > 90).Take(3) [91, 92, 93]
FirstReturn the first element in this RDD. >>> sc.Parallelize(new int[] { 2, 3, 4 }, 2).First() 2
IsEmptyReturns true if and only if the RDD contains no elements at all. Note that an RDD may be empty even when it has at least 1 partition. sc.Parallelize(new int[0], 1).isEmpty() true sc.Parallelize(new int[] {1}).isEmpty() false
SubtractReturn each value in this RDD that is not contained in . var x = sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1) var y = sc.Parallelize(new int[] { 3 }, 1) x.Subtract(y).Collect()) [1, 2, 4]
KeyBy``1Creates tuples of the elements in this RDD by applying . sc.Parallelize(new int[] { 1, 2, 3, 4 }, 1).KeyBy(x => x * x).Collect()) (1, 1), (4, 2), (9, 3), (16, 4)
RepartitionReturn a new RDD that has exactly numPartitions partitions. Can increase or decrease the level of parallelism in this RDD. Internally, this uses a shuffle to redistribute data. If you are decreasing the number of partitions in this RDD, consider using `Coalesce`, which can avoid performing a shuffle. var rdd = sc.Parallelize(new int[] { 1, 2, 3, 4, 5, 6, 7 }, 4) rdd.Glom().Collect().Length 4 rdd.Repartition(2).Glom().Collect().Length 2
CoalesceReturn a new RDD that is reduced into `numPartitions` partitions. sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 3).Glom().Collect().Length 3 >>> sc.Parallelize(new int[] { 1, 2, 3, 4, 5 }, 3).Coalesce(1).Glom().Collect().Length 1
Zip``1Zips this RDD with another one, returning key-value pairs with the first element in each RDD second element in each RDD, etc. Assumes that the two RDDs have the same number of partitions and the same number of elements in each partition (e.g. one was made through a map on the other). var x = sc.parallelize(range(0,5)) var y = sc.parallelize(range(1000, 1005)) x.Zip(y).Collect() [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)]
ZipWithIndexZips this RDD with its element indices. The ordering is first based on the partition index and then the ordering of items within each partition. So the first item in the first partition gets index 0, and the last item in the last partition receives the largest index. This method needs to trigger a spark job when this RDD contains more than one partitions. sc.Parallelize(new string[] { "a", "b", "c", "d" }, 3).ZipWithIndex().Collect() [('a', 0), ('b', 1), ('c', 2), ('d', 3)]
ZipWithUniqueIdZips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method won't trigger a spark job, which is different from >>> sc.Parallelize(new string[] { "a", "b", "c", "d" }, 1).ZipWithIndex().Collect() [('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)]
SetNameAssign a name to this RDD. >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() u'RDD1'
ToDebugStringA description of this RDD and its recursive dependencies for debugging.
GetStorageLevelGet the RDD's current storage level. >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.getStorageLevel() StorageLevel(False, False, False, False, 1) >>> print(rdd1.getStorageLevel()) Serialized 1x Replicated
ToLocalIteratorReturn an iterator that contains all of the elements in this RDD. The iterator will consume as much memory as the largest partition in this RDD. sc.Parallelize(Enumerable.Range(0, 10), 1).ToLocalIterator() [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
RandomSampleWithRangeInternal method exposed for Random Splits in DataFrames. Samples an RDD given a probability range.
--- @@ -703,6 +703,20 @@ --- +###Microsoft.Spark.CSharp.Streaming.ConstantInputDStream`1 +####Summary + + + An input stream that always returns the same RDD on each timestep. Useful for testing. + + +####Methods + +
NameDescription
+ +--- + + ###Microsoft.Spark.CSharp.Streaming.DStream`1 ####Summary @@ -831,7 +845,7 @@ ####Summary - operations only available to KeyValuePair RDD + operations only available to Tuple RDD ####Methods diff --git a/csharp/AdapterTest/AccumulatorTest.cs b/csharp/AdapterTest/AccumulatorTest.cs index f5998f4d..a6b2d7f7 100644 --- a/csharp/AdapterTest/AccumulatorTest.cs +++ b/csharp/AdapterTest/AccumulatorTest.cs @@ -80,7 +80,7 @@ public void TestAccumuatorSuccess() // write update int key = 0; int value = 100; - KeyValuePair update = new KeyValuePair(key, value); + Tuple update = new Tuple(key, value); var ms = new MemoryStream(); var formatter = new BinaryFormatter(); formatter.Serialize(ms, update); @@ -111,7 +111,7 @@ public void TestUndefinedAccumuator() // write update int key = 1; int value = 1000; - KeyValuePair update = new KeyValuePair(key, value); + Tuple update = new Tuple(key, value); var ms = new MemoryStream(); var formatter = new BinaryFormatter(); formatter.Serialize(ms, update); @@ -123,8 +123,8 @@ public void TestUndefinedAccumuator() byte[] receiveBuffer = new byte[1]; s.Read(receiveBuffer, 0, 1); - Assert.IsTrue(Accumulator.accumulatorRegistry.ContainsKey(update.Key)); - var accumulator = Accumulator.accumulatorRegistry[update.Key] as Accumulator; + Assert.IsTrue(Accumulator.accumulatorRegistry.ContainsKey(update.Item1)); + var accumulator = Accumulator.accumulatorRegistry[update.Item1] as Accumulator; Assert.AreEqual(accumulator.Value, value); } } diff --git a/csharp/AdapterTest/DStreamTest.cs b/csharp/AdapterTest/DStreamTest.cs index a46ff061..e829a611 100644 --- a/csharp/AdapterTest/DStreamTest.cs +++ b/csharp/AdapterTest/DStreamTest.cs @@ -47,8 +47,8 @@ public void TestDStreamMapReduce() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } }); @@ -90,7 +90,7 @@ public void TestDStreamTransform() var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); var wordCounts = pairs.PartitionBy().ReduceByKey((x, y) => x + y); @@ -101,8 +101,8 @@ public void TestDStreamTransform() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } }); @@ -115,8 +115,8 @@ public void TestDStreamTransform() foreach (object record in taken) { - KeyValuePair> countByWord = (KeyValuePair>)record; - Assert.AreEqual(countByWord.Value.Count, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Tuple> countByWord = (Tuple>)record; + Assert.AreEqual(countByWord.Item2.Count, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } }); @@ -129,8 +129,8 @@ public void TestDStreamTransform() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 46 : 44); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 46 : 44); } }); } @@ -146,12 +146,12 @@ public void TestDStreamJoin() var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); var wordCounts = pairs.ReduceByKey((x, y) => x + y); - var left = wordCounts.Filter(x => x.Key != "quick" && x.Key != "lazy"); - var right = wordCounts.Filter(x => x.Key != "brown"); + var left = wordCounts.Filter(x => x.Item1 != "quick" && x.Item1 != "lazy"); + var right = wordCounts.Filter(x => x.Item1 != "brown"); var groupWith = left.GroupWith(right); groupWith.ForeachRDD((time, rdd) => @@ -161,15 +161,15 @@ public void TestDStreamJoin() foreach (object record in taken) { - KeyValuePair, List>> countByWord = (KeyValuePair, List>>)record; - if (countByWord.Key == "quick" || countByWord.Key == "lazy") - Assert.AreEqual(countByWord.Value.Item1.Count, 0); - else if (countByWord.Key == "brown") - Assert.AreEqual(countByWord.Value.Item2.Count, 0); + Tuple, List>> countByWord = (Tuple, List>>)record; + if (countByWord.Item1 == "quick" || countByWord.Item1 == "lazy") + Assert.AreEqual(countByWord.Item2.Item1.Count, 0); + else if (countByWord.Item1 == "brown") + Assert.AreEqual(countByWord.Item2.Item2.Count, 0); else { - Assert.AreEqual(countByWord.Value.Item1[0], countByWord.Key == "The" || countByWord.Key == "dog" ? 23 : 22); - Assert.AreEqual(countByWord.Value.Item2[0], countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Assert.AreEqual(countByWord.Item2.Item1[0], countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? 23 : 22); + Assert.AreEqual(countByWord.Item2.Item2[0], countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } } }); @@ -182,9 +182,9 @@ public void TestDStreamJoin() foreach (object record in taken) { - KeyValuePair> countByWord = (KeyValuePair>)record; - Assert.AreEqual(countByWord.Value.Item1, countByWord.Key == "The" || countByWord.Key == "dog" ? 23 : 22); - Assert.AreEqual(countByWord.Value.Item2, countByWord.Key == "The" || countByWord.Key == "dog" ? 23 : 22); + Tuple> countByWord = (Tuple>)record; + Assert.AreEqual(countByWord.Item2.Item1, countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? 23 : 22); + Assert.AreEqual(countByWord.Item2.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? 23 : 22); } }); @@ -196,11 +196,11 @@ public void TestDStreamJoin() foreach (object record in taken) { - KeyValuePair>> countByWord = (KeyValuePair>>)record; - Assert.AreEqual(countByWord.Value.Item1, countByWord.Key == "The" || countByWord.Key == "dog" ? 23 : 22); - Assert.IsTrue(countByWord.Key == "The" || countByWord.Key == "dog" ? - countByWord.Value.Item2.IsDefined == true && countByWord.Value.Item2.GetValue() == 23 : (countByWord.Key == "brown" ? - countByWord.Value.Item2.IsDefined == true == false : countByWord.Value.Item2.IsDefined == true && countByWord.Value.Item2.GetValue() == 22)); + Tuple>> countByWord = (Tuple>>)record; + Assert.AreEqual(countByWord.Item2.Item1, countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? 23 : 22); + Assert.IsTrue(countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? + countByWord.Item2.Item2.IsDefined == true && countByWord.Item2.Item2.GetValue() == 23 : (countByWord.Item1 == "brown" ? + countByWord.Item2.Item2.IsDefined == true == false : countByWord.Item2.Item2.IsDefined == true && countByWord.Item2.Item2.GetValue() == 22)); } }); @@ -212,12 +212,12 @@ public void TestDStreamJoin() foreach (object record in taken) { - KeyValuePair, int>> countByWord = (KeyValuePair, int>>)record; - Assert.IsTrue(countByWord.Key == "The" || countByWord.Key == "dog" ? - countByWord.Value.Item1.IsDefined == true && countByWord.Value.Item1.GetValue() == 23 : - (countByWord.Key == "quick" || countByWord.Key == "lazy" ? countByWord.Value.Item1.IsDefined == false : - countByWord.Value.Item1.IsDefined == true && countByWord.Value.Item1.GetValue() == 22)); - Assert.AreEqual(countByWord.Value.Item2, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Tuple, int>> countByWord = (Tuple, int>>)record; + Assert.IsTrue(countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? + countByWord.Item2.Item1.IsDefined == true && countByWord.Item2.Item1.GetValue() == 23 : + (countByWord.Item1 == "quick" || countByWord.Item1 == "lazy" ? countByWord.Item2.Item1.IsDefined == false : + countByWord.Item2.Item1.IsDefined == true && countByWord.Item2.Item1.GetValue() == 22)); + Assert.AreEqual(countByWord.Item2.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } }); @@ -229,15 +229,15 @@ public void TestDStreamJoin() foreach (object record in taken) { - KeyValuePair, Option>> countByWord = (KeyValuePair, Option>>)record; - Assert.IsTrue(countByWord.Key == "The" || countByWord.Key == "dog" ? - countByWord.Value.Item1.IsDefined == true && countByWord.Value.Item1.GetValue() == 23 : - (countByWord.Key == "quick" || countByWord.Key == "lazy" ? countByWord.Value.Item1.IsDefined == false : - countByWord.Value.Item1.IsDefined == true && countByWord.Value.Item1.GetValue() == 22)); - - Assert.IsTrue(countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? - countByWord.Value.Item2.IsDefined == true && countByWord.Value.Item2.GetValue() == 23 : - (countByWord.Key == "brown" ? countByWord.Value.Item2.IsDefined == false : countByWord.Value.Item2.IsDefined == true && countByWord.Value.Item2.GetValue() == 22)); + Tuple, Option>> countByWord = (Tuple, Option>>)record; + Assert.IsTrue(countByWord.Item1 == "The" || countByWord.Item1 == "dog" ? + countByWord.Item2.Item1.IsDefined == true && countByWord.Item2.Item1.GetValue() == 23 : + (countByWord.Item1 == "quick" || countByWord.Item1 == "lazy" ? countByWord.Item2.Item1.IsDefined == false : + countByWord.Item2.Item1.IsDefined == true && countByWord.Item2.Item1.GetValue() == 22)); + + Assert.IsTrue(countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? + countByWord.Item2.Item2.IsDefined == true && countByWord.Item2.Item2.GetValue() == 23 : + (countByWord.Item1 == "brown" ? countByWord.Item2.Item2.IsDefined == false : countByWord.Item2.Item2.IsDefined == true && countByWord.Item2.Item2.GetValue() == 22)); } }); } @@ -253,7 +253,7 @@ public void TestDStreamUpdateStateByKey() var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); var doubleCounts = pairs.GroupByKey().FlatMapValues(vs => vs).MapValues(v => 2 * v).ReduceByKey((x, y) => x + y); doubleCounts.ForeachRDD((time, rdd) => @@ -263,8 +263,8 @@ public void TestDStreamUpdateStateByKey() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 2 * 23 : 2 * 22); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 2 * 23 : 2 * 22); } }); @@ -279,8 +279,8 @@ public void TestDStreamUpdateStateByKey() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 24 : 23); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 24 : 23); } }); } @@ -314,7 +314,7 @@ public void TestDStreamMapWithState() var ssc = new StreamingContext(new SparkContext(sparkContextProxy.Object, sparkConf), 10000); var dstreamProxy = new Mock(); - var pairDStream = new DStream>(dstreamProxy.Object, ssc); + var pairDStream = new DStream>(dstreamProxy.Object, ssc); var stateSpec = new StateSpec((k, v, s) => v); var stateDStream = pairDStream.MapWithState(stateSpec); @@ -357,7 +357,7 @@ public void TestDStreamMapWithStateMapWithStateHelper() Assert.IsNotNull(resultRdd); // test when initialStateRdd is not null - var initialStateRdd = new RDD>(new Mock().Object, null); + var initialStateRdd = new RDD>(new Mock().Object, null); var stateSpec2 = new StateSpec((k, v, s) => v).InitialState(initialStateRdd).NumPartitions(2); var helper2 = new MapWithStateHelper((t, rdd) => rdd, stateSpec2); @@ -388,13 +388,13 @@ public void TestDStreamMapWithStateUpdateStateHelper() var input = new dynamic[4]; - var preStateRddRecord = new MapWithStateRDDRecord(ticks - TimeSpan.FromSeconds(2).Ticks, new [] { new KeyValuePair("1", 1), new KeyValuePair("2", 2)}); + var preStateRddRecord = new MapWithStateRDDRecord(ticks - TimeSpan.FromSeconds(2).Ticks, new [] { new Tuple("1", 1), new Tuple("2", 2)}); preStateRddRecord.stateMap.Add("expired", new KeyedState(0, ticks - TimeSpan.FromSeconds(60).Ticks)); input[0] = preStateRddRecord; - input[1] = new KeyValuePair("1", -1); - input[2] = new KeyValuePair("2", 2); - input[3] = new KeyValuePair("3", 3); + input[1] = new Tuple("1", -1); + input[2] = new Tuple("2", 2); + input[3] = new Tuple("3", 3); var result = helper.Execute(1, input).GetEnumerator(); Assert.IsNotNull(result); diff --git a/csharp/AdapterTest/DoubleRDDTest.cs b/csharp/AdapterTest/DoubleRDDTest.cs index d61ad2d7..5b773809 100644 --- a/csharp/AdapterTest/DoubleRDDTest.cs +++ b/csharp/AdapterTest/DoubleRDDTest.cs @@ -19,7 +19,7 @@ public static void Initialize() var sparkContext = new SparkContext(null); var lines = sparkContext.TextFile(Path.GetTempFileName()); var words = lines.FlatMap(l => l.Split(' ')); - doubles = words.Map(w => new KeyValuePair(w, 1)).ReduceByKey((x, y) => x + y).Map(kv => (double)kv.Value); + doubles = words.Map(w => new Tuple(w, 1)).ReduceByKey((x, y) => x + y).Map(kv => (double)kv.Item2); } [Test] diff --git a/csharp/AdapterTest/EventHubsUtilsTest.cs b/csharp/AdapterTest/EventHubsUtilsTest.cs index 428d4b6f..44f2fa2b 100644 --- a/csharp/AdapterTest/EventHubsUtilsTest.cs +++ b/csharp/AdapterTest/EventHubsUtilsTest.cs @@ -22,7 +22,7 @@ public void TestCreateUnionStream() var streamingContextProxy = new Mock(); var mockDstreamProxy = new Mock().Object; streamingContextProxy.Setup( - m => m.EventHubsUnionStream(It.IsAny>(), It.IsAny())) + m => m.EventHubsUnionStream(It.IsAny>>(), It.IsAny())) .Returns(mockDstreamProxy); var mockSparkClrProxy = new Mock(); @@ -32,7 +32,7 @@ public void TestCreateUnionStream() var sparkContext = new SparkContext(SparkCLREnvironment.SparkCLRProxy.SparkContextProxy, new SparkConf(new Mock().Object)); var streamingContext = new StreamingContext(sparkContext, 123); - var dstream = EventHubsUtils.CreateUnionStream(streamingContext, new Dictionary()); + var dstream = EventHubsUtils.CreateUnionStream(streamingContext, new List>()); Assert.AreEqual(mockDstreamProxy, dstream.DStreamProxy); } } diff --git a/csharp/AdapterTest/Mocks/MockRddProxy.cs b/csharp/AdapterTest/Mocks/MockRddProxy.cs index d9baa284..1a3cfe4c 100644 --- a/csharp/AdapterTest/Mocks/MockRddProxy.cs +++ b/csharp/AdapterTest/Mocks/MockRddProxy.cs @@ -136,7 +136,7 @@ public IRDDProxy Coalesce(int numPartitions, bool shuffle) return this; } - public IRDDProxy SampleByKey(bool withReplacement, Dictionary fractions, long seed) + public IRDDProxy SampleByKey(bool withReplacement, IEnumerable> fractions, long seed) { return this; } @@ -151,13 +151,13 @@ public string ToDebugString() return null; } - public void SaveAsNewAPIHadoopDataset(IEnumerable> conf) + public void SaveAsNewAPIHadoopDataset(IEnumerable> conf) { } - public void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) + public void SaveAsNewAPIHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf) { } - public void SaveAsHadoopDataset(IEnumerable> conf) + public void SaveAsHadoopDataset(IEnumerable> conf) { } public void SaveAsSequenceFile(string path, string compressionCodecClass) @@ -167,7 +167,7 @@ public void SaveAsTextFile(string path, string compressionCodecClass) { } - public void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) + public void SaveAsHadoopFile(string path, string outputFormatClass, string keyClass, string valueClass, IEnumerable> conf, string compressionCodecClass) { } diff --git a/csharp/AdapterTest/Mocks/MockSparkContextProxy.cs b/csharp/AdapterTest/Mocks/MockSparkContextProxy.cs index 3b783cda..6813daeb 100644 --- a/csharp/AdapterTest/Mocks/MockSparkContextProxy.cs +++ b/csharp/AdapterTest/Mocks/MockSparkContextProxy.cs @@ -133,22 +133,22 @@ public IRDDProxy SequenceFile(string filePath, string keyClass, string valueClas return new MockRddProxy(null); } - public IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy NewAPIHadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { return new MockRddProxy(null); } - public IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy NewAPIHadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { return new MockRddProxy(null); } - public IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy HadoopFile(string filePath, string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { return new MockRddProxy(null); } - public IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) + public IRDDProxy HadoopRDD(string inputFormatClass, string keyClass, string valueClass, string keyConverterClass, string valueConverterClass, IEnumerable> conf, int batchSize) { return new MockRddProxy(null); } diff --git a/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs b/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs index 9df6c506..fee98275 100644 --- a/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs +++ b/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs @@ -39,17 +39,17 @@ public IDStreamProxy SocketTextStream(string hostname, int port, Microsoft.Spark return new MockDStreamProxy(); } - public IDStreamProxy KafkaStream(Dictionary topics, Dictionary kafkaParams, Microsoft.Spark.CSharp.Core.StorageLevelType storageLevelType) + public IDStreamProxy KafkaStream(IEnumerable> topics, IEnumerable> kafkaParams, Microsoft.Spark.CSharp.Core.StorageLevelType storageLevelType) { return new MockDStreamProxy(); } - public IDStreamProxy DirectKafkaStream(List topics, Dictionary kafkaParams, Dictionary fromOffsets) + public IDStreamProxy DirectKafkaStream(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets) { return new MockDStreamProxy(); } - public IDStreamProxy DirectKafkaStreamWithRepartition(List topics, Dictionary kafkaParams, Dictionary fromOffsets, int numPartitions) + public IDStreamProxy DirectKafkaStreamWithRepartition(List topics, IEnumerable> kafkaParams, IEnumerable> fromOffsets, int numPartitions) { return new MockDStreamProxy(); } @@ -113,7 +113,7 @@ public IDStreamProxy CreateConstantInputDStream(IRDDProxy rddProxy) return new MockDStreamProxy(); } - public IDStreamProxy EventHubsUnionStream(Dictionary eventHubsParams, StorageLevelType storageLevelType) + public IDStreamProxy EventHubsUnionStream(IEnumerable> eventHubsParams, StorageLevelType storageLevelType) { throw new NotImplementedException(); } diff --git a/csharp/AdapterTest/PairRDDTest.cs b/csharp/AdapterTest/PairRDDTest.cs index f8ba3847..00dd3e99 100644 --- a/csharp/AdapterTest/PairRDDTest.cs +++ b/csharp/AdapterTest/PairRDDTest.cs @@ -10,7 +10,7 @@ namespace AdapterTest [TestFixture] public class PairRDDTest { - private static RDD> pairs; + private static RDD> pairs; [OneTimeSetUp] public static void Initialize() @@ -18,7 +18,7 @@ public static void Initialize() var sparkContext = new SparkContext(null); var lines = sparkContext.TextFile(Path.GetTempFileName()); var words = lines.FlatMap(l => l.Split(' ')); - pairs = words.Map(w => new KeyValuePair(w, 1)); + pairs = words.Map(w => new Tuple(w, 1)); } [Test] @@ -36,53 +36,53 @@ public void TestPairRddGroupWith() { foreach (var record in pairs.GroupWith(pairs).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); } foreach (var record in pairs.GroupWith(pairs, pairs).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item3.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item3.Count); } foreach (var record in pairs.GroupWith(pairs, pairs, pairs).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item3.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item4.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item3.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item4.Count); } } /// - /// Test RDD.GroupWith() method with different KeyValuePair types. + /// Test RDD.GroupWith() method with different Tuple types. /// [Test] public void TestPairRddGroupWith2() { - var pairs1 = pairs.Map(p => new KeyValuePair(p.Key, Convert.ToDouble(p.Value))); - var pairs2 = pairs.Map(p => new KeyValuePair(p.Key, p.Value.ToString())); - var pairs3 = pairs.Map(p => new KeyValuePair(p.Key, Convert.ToInt64(p.Value))); + var pairs1 = pairs.Map(p => new Tuple(p.Item1, Convert.ToDouble(p.Item2))); + var pairs2 = pairs.Map(p => new Tuple(p.Item1, p.Item2.ToString())); + var pairs3 = pairs.Map(p => new Tuple(p.Item1, Convert.ToInt64(p.Item2))); foreach (var record in pairs.GroupWith(pairs1).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); } foreach (var record in pairs.GroupWith(pairs1, pairs2).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item3.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item3.Count); } foreach (var record in pairs.GroupWith(pairs1, pairs2, pairs3).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item1.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item2.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item3.Count); - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Item4.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item1.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item2.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item3.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Item4.Count); } } @@ -90,10 +90,10 @@ public void TestPairRddGroupWith2() public void TestPairRddSubtractByKey() { var reduce = pairs.ReduceByKey((x, y) => x + y); - var records = reduce.SubtractByKey(reduce.Filter(kvp => kvp.Key != "The")).Collect(); + var records = reduce.SubtractByKey(reduce.Filter(kvp => kvp.Item1 != "The")).Collect(); Assert.AreEqual(1, records.Length); - Assert.AreEqual("The", records[0].Key); - Assert.AreEqual(23, records[0].Value); + Assert.AreEqual("The", records[0].Item1); + Assert.AreEqual(23, records[0].Item2); } [Test] @@ -110,7 +110,7 @@ public void TestPairRddFoldByKey() { foreach (var record in pairs.FoldByKey(() => 0, (x, y) => x + y).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2); } } @@ -119,7 +119,7 @@ public void TestPairRddAggregateByKey() { foreach (var record in pairs.AggregateByKey(() => 0, (x, y) => x + y, (x, y) => x + y).Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2); } } @@ -128,7 +128,7 @@ public void TestPairRddGroupByKey() { foreach (var record in pairs.GroupByKey().Collect()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Count); } } @@ -165,7 +165,7 @@ public void TestPairRddPartitionBy() [Test] public void TestPairRddSortByKey() { - var expectedSortedRdd = pairs.Collect().OrderBy(kv => kv.Key, StringComparer.OrdinalIgnoreCase).ToArray(); + var expectedSortedRdd = pairs.Collect().OrderBy(kv => kv.Item1, StringComparer.OrdinalIgnoreCase).ToArray(); var rddSortByKey = pairs.SortByKey(true, null, key => key.ToLowerInvariant()).Collect(); CollectionAssert.AreEqual(expectedSortedRdd, rddSortByKey); } @@ -173,7 +173,7 @@ public void TestPairRddSortByKey() [Test] public void TestPairRddSortByKey2() { - var expectedSortedRdd = pairs.Collect().OrderBy(kv => kv.Key, StringComparer.OrdinalIgnoreCase).ToArray(); + var expectedSortedRdd = pairs.Collect().OrderBy(kv => kv.Item1, StringComparer.OrdinalIgnoreCase).ToArray(); var rddSortByKey = pairs.SortByKey(true, 1, key => key.ToLowerInvariant()).Collect(); CollectionAssert.AreEqual(expectedSortedRdd, rddSortByKey); } diff --git a/csharp/AdapterTest/RDDTest.cs b/csharp/AdapterTest/RDDTest.cs index 2d3049df..23bbdc11 100644 --- a/csharp/AdapterTest/RDDTest.cs +++ b/csharp/AdapterTest/RDDTest.cs @@ -119,14 +119,14 @@ public void TestRddGroupBy() { words.GroupBy(w => w).Foreach(record => { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Count); }); words.GroupBy(w => w).ForeachPartition(iter => { foreach (var record in iter) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value.Count); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2.Count); } }); } @@ -144,7 +144,7 @@ public void TestRddZipWithIndex() int index = 0; foreach(var record in words.ZipWithIndex().Collect()) { - Assert.AreEqual(index++, record.Value); + Assert.AreEqual(index++, record.Item2); } } @@ -155,7 +155,7 @@ public void TestRddZipWithUniqueId() int num = words.GetNumPartitions(); foreach (var record in words.ZipWithUniqueId().Collect()) { - Assert.AreEqual(num * index++, record.Value); + Assert.AreEqual(num * index++, record.Item2); } } diff --git a/csharp/AdapterTest/SparkContextTest.cs b/csharp/AdapterTest/SparkContextTest.cs index 8b4f45f5..719ad3e0 100644 --- a/csharp/AdapterTest/SparkContextTest.cs +++ b/csharp/AdapterTest/SparkContextTest.cs @@ -412,7 +412,7 @@ public void TestNewAPIHadoopFile() Mock rddProxy = new Mock(); Mock sparkContextProxy = new Mock(); - sparkContextProxy.Setup(m => m.NewAPIHadoopFile(filePath, It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) + sparkContextProxy.Setup(m => m.NewAPIHadoopFile(filePath, It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) .Returns(rddProxy.Object); SparkContext sc = new SparkContext(sparkContextProxy.Object, null); @@ -440,7 +440,7 @@ public void TestHadoopFile() Mock rddProxy = new Mock(); Mock sparkContextProxy = new Mock(); - sparkContextProxy.Setup(m => m.HadoopFile(filePath, It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) + sparkContextProxy.Setup(m => m.HadoopFile(filePath, It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) .Returns(rddProxy.Object); SparkContext sc = new SparkContext(sparkContextProxy.Object, null); @@ -466,12 +466,12 @@ public void TestNewAPIHadoopRDD() Mock rddProxy = new Mock(); Mock sparkContextProxy = new Mock(); - sparkContextProxy.Setup(m => m.NewAPIHadoopRDD(It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) + sparkContextProxy.Setup(m => m.NewAPIHadoopRDD(It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) .Returns(rddProxy.Object); SparkContext sc = new SparkContext(sparkContextProxy.Object, null); const string inputFormatClass = "org.apache.hadoop.mapreduce.lib.input.TextInputFormat"; - var conf = new KeyValuePair[] { }; + var conf = new Tuple[] { }; // Act RDD rdd = sc.NewAPIHadoopRDD(inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf); @@ -493,12 +493,12 @@ public void TestHadoopRDD() Mock rddProxy = new Mock(); Mock sparkContextProxy = new Mock(); - sparkContextProxy.Setup(m => m.HadoopRDD(It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) + sparkContextProxy.Setup(m => m.HadoopRDD(It.IsAny(), keyClass, valueClass, keyConverterClass, valueConverterClass, It.IsAny>>(), It.IsAny())) .Returns(rddProxy.Object); SparkContext sc = new SparkContext(sparkContextProxy.Object, null); const string inputFormatClass = "org.apache.hadoop.mapreduce.lib.input.TextInputFormat"; - var conf = new KeyValuePair[] { }; + var conf = new Tuple[] { }; // Act RDD rdd = sc.HadoopRDD(inputFormatClass, keyClass, valueClass, keyConverterClass, valueConverterClass, conf); diff --git a/csharp/AdapterTest/StreamingContextTest.cs b/csharp/AdapterTest/StreamingContextTest.cs index af07c48e..b2628bf2 100644 --- a/csharp/AdapterTest/StreamingContextTest.cs +++ b/csharp/AdapterTest/StreamingContextTest.cs @@ -31,13 +31,13 @@ public void TestStreamingContext() var socketStream = ssc.SocketTextStream(IPAddress.Loopback.ToString(), 12345); Assert.IsNotNull(socketStream.DStreamProxy); - var kafkaStream = KafkaUtils.CreateStream(ssc, IPAddress.Loopback + ":2181", "testGroupId", new Dictionary { { "testTopic1", 1 } }, new Dictionary()); + var kafkaStream = KafkaUtils.CreateStream(ssc, IPAddress.Loopback + ":2181", "testGroupId", new [] { Tuple.Create("testTopic1", 1) }, new List>()); Assert.IsNotNull(kafkaStream.DStreamProxy); - var directKafkaStream = KafkaUtils.CreateDirectStream(ssc, new List { "testTopic2" }, new Dictionary(), new Dictionary()); + var directKafkaStream = KafkaUtils.CreateDirectStream(ssc, new List { "testTopic2" }, new List>(), new List>()); Assert.IsNotNull(directKafkaStream.DStreamProxy); - var directKafkaStreamWithRepartition = KafkaUtils.CreateDirectStreamWithRepartition(ssc, new List { "testTopic3" }, new Dictionary(), new Dictionary(), 10); + var directKafkaStreamWithRepartition = KafkaUtils.CreateDirectStreamWithRepartition(ssc, new List { "testTopic3" }, new List>(), new List>(), 10); Assert.IsNotNull(directKafkaStreamWithRepartition.DStreamProxy); var union = ssc.Union(textFile, socketStream); diff --git a/csharp/AdapterTest/TestWithMoqDemo.cs b/csharp/AdapterTest/TestWithMoqDemo.cs index 70c4f8d6..fda808f7 100644 --- a/csharp/AdapterTest/TestWithMoqDemo.cs +++ b/csharp/AdapterTest/TestWithMoqDemo.cs @@ -166,7 +166,7 @@ public void TestDStreamTransform_Moq() // Act var lines = _streamingContext.TextFileStream(Path.GetTempPath()); var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); var wordCounts = pairs.ReduceByKey((x, y) => x + y); // Assert @@ -177,8 +177,8 @@ public void TestDStreamTransform_Moq() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22); } }); // Use Verify to verify if a method to mock was invoked diff --git a/csharp/Perf/Microsoft.Spark.CSharp/FreebaseDeletionsBenchmark.cs b/csharp/Perf/Microsoft.Spark.CSharp/FreebaseDeletionsBenchmark.cs index c76fc51e..e001a990 100644 --- a/csharp/Perf/Microsoft.Spark.CSharp/FreebaseDeletionsBenchmark.cs +++ b/csharp/Perf/Microsoft.Spark.CSharp/FreebaseDeletionsBenchmark.cs @@ -63,11 +63,11 @@ internal static void RunRDDMaxDeletionsByUser(string[] args) var flaggedRows = parsedRows.Filter(s => s.Item1); //select good rows var selectedDeletions = flaggedRows.Filter(s => s.Item3.Equals(s.Item5)); //select deletions made by same creators - var userDeletions = selectedDeletions.Map(s => new KeyValuePair(s.Item3, 1)); + var userDeletions = selectedDeletions.Map(s => new Tuple(s.Item3, 1)); var userDeletionCount = userDeletions.ReduceByKey((x, y) => x + y); - var userWithMaxDeletions = userDeletionCount.Fold(new KeyValuePair("zerovalue", 0), (kvp1, kvp2) => + var userWithMaxDeletions = userDeletionCount.Fold(new Tuple("zerovalue", 0), (kvp1, kvp2) => { - if (kvp1.Value > kvp2.Value) + if (kvp1.Item2 > kvp2.Item2) return kvp1; else return kvp2; @@ -76,7 +76,7 @@ internal static void RunRDDMaxDeletionsByUser(string[] args) stopwatch.Stop(); PerfBenchmark.ExecutionTimeList.Add(stopwatch.Elapsed); - Console.WriteLine("User with max deletions is {0}, count of deletions={1}. Elapsed time={2}", userWithMaxDeletions.Key, userWithMaxDeletions.Value, stopwatch.Elapsed); + Console.WriteLine("User with max deletions is {0}, count of deletions={1}. Elapsed time={2}", userWithMaxDeletions.Item1, userWithMaxDeletions.Item2, stopwatch.Elapsed); } [PerfSuite] diff --git a/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs index 3d47966c..488a46ef 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs @@ -69,7 +69,7 @@ internal static void DStreamTextFileSamples() var lines = context.TextFileStream(Path.Combine(directory, "test")); lines = context.Union(lines, lines); var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); // since operations like ReduceByKey, Join and UpdateStateByKey are // separate dstream transformations defined in CSharpDStream.scala @@ -140,7 +140,7 @@ internal static void DStreamDirectKafkaWithRepartitionSample() {"auto.offset.reset", "smallest"} }; - var dstream = KafkaUtils.CreateDirectStreamWithRepartition(context, new List { topic }, kafkaParams, new Dictionary(), partitions); + var dstream = KafkaUtils.CreateDirectStreamWithRepartition(context, new List { topic }, kafkaParams.Select(kv => Tuple.Create(kv.Key, kv.Value)), new List>(), partitions); dstream.ForeachRDD((time, rdd) => { diff --git a/csharp/Samples/Microsoft.Spark.CSharp/DStreamStateSample.cs b/csharp/Samples/Microsoft.Spark.CSharp/DStreamStateSample.cs index 0c0e3c28..aef5fc6a 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/DStreamStateSample.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/DStreamStateSample.cs @@ -61,16 +61,16 @@ internal static void DStreamMapWithStateSample() var lines = context.TextFileStream(Path.Combine(directory, "test1")); lines = context.Union(lines, lines); var words = lines.FlatMap(l => l.Split(' ')); - var pairs = words.Map(w => new KeyValuePair(w, 1)); + var pairs = words.Map(w => new Tuple(w, 1)); var wordCounts = pairs.ReduceByKey((x, y) => x + y); - var initialState = sc.Parallelize(new[] { new KeyValuePair("NOT_A_WORD", 1024), new KeyValuePair("dog", 10000), }, 1); - var stateSpec = new StateSpec>((word, count, state) => + var initialState = sc.Parallelize(new[] { new Tuple("NOT_A_WORD", 1024), new Tuple("dog", 10000), }, 1); + var stateSpec = new StateSpec>((word, count, state) => { if (state.IsTimingOut()) { Console.WriteLine("Found timing out word: {0}", word); - return new KeyValuePair(word, state.Get()); + return new Tuple(word, state.Get()); } var sum = 0; @@ -80,7 +80,7 @@ internal static void DStreamMapWithStateSample() } state.Update(sum + count); Console.WriteLine("word: {0}, count: {1}", word, sum + count); - return new KeyValuePair(word, sum + count); + return new Tuple(word, sum + count); }).NumPartitions(1).InitialState(initialState).Timeout(TimeSpan.FromSeconds(30)); var snapshots = wordCounts.MapWithState(stateSpec).StateSnapshots(); @@ -90,9 +90,9 @@ internal static void DStreamMapWithStateSample() Console.WriteLine("Snapshots @ Time: {0}", time); Console.WriteLine("-------------------------------------------"); - foreach (KeyValuePair record in rdd.Collect()) + foreach (Tuple record in rdd.Collect()) { - Console.WriteLine("[{0}, {1}]", record.Key, record.Value); + Console.WriteLine("[{0}, {1}]", record.Item1, record.Item2); } Console.WriteLine(); }); diff --git a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs index ac63d312..97604337 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs @@ -15,7 +15,7 @@ class PairRDDSamples [Sample] internal static void PairRDDCollectAsMapSample() { - var map = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair(1, 2), new KeyValuePair(3, 4) }, 1).CollectAsMap(); + var map = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple(1, 2), new Tuple(3, 4) }, 1).CollectAsMap(); foreach (var kv in map) Console.WriteLine(kv); @@ -30,7 +30,7 @@ internal static void PairRDDCollectAsMapSample() [Sample] internal static void PairRDDKeysSample() { - var keys = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair(1, 2), new KeyValuePair(3, 4) }, 1).Keys().Collect(); + var keys = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple(1, 2), new Tuple(3, 4) }, 1).Keys().Collect(); Console.WriteLine(keys[0]); Console.WriteLine(keys[1]); @@ -45,7 +45,7 @@ internal static void PairRDDKeysSample() [Sample] internal static void PairRDDValuesSample() { - var values = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair(1, 2), new KeyValuePair(3, 4) }, 1).Values().Collect(); + var values = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple(1, 2), new Tuple(3, 4) }, 1).Values().Collect(); Console.WriteLine(values[0]); Console.WriteLine(values[1]); @@ -63,9 +63,9 @@ internal static void PairRDDReduceByKeySample() var reduced = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .ReduceByKey((x, y) => x + y).Collect(); @@ -74,8 +74,8 @@ internal static void PairRDDReduceByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(reduced.Contains(new KeyValuePair("a", 2))); - Assert.IsTrue(reduced.Contains(new KeyValuePair("b", 1))); + Assert.IsTrue(reduced.Contains(new Tuple("a", 2))); + Assert.IsTrue(reduced.Contains(new Tuple("b", 1))); } } @@ -85,9 +85,9 @@ internal static void PairRDDReduceByKeyLocallySample() var reduced = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .ReduceByKeyLocally((x, y) => x + y); @@ -105,13 +105,14 @@ internal static void PairRDDReduceByKeyLocallySample() internal static void PairRDDCountByKeySample() { var countByKey = SparkCLRSamples.SparkContext.Parallelize( - new[] - { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new[] + { + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) - .CountByKey(); + .CountByKey() + .ToLookup(x => x.Key); foreach (var kv in countByKey) Console.WriteLine(kv); @@ -129,15 +130,15 @@ internal static void PairRDDJoinSample() var l = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 4), + new Tuple("a", 1), + new Tuple("b", 4), }, 1); var r = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 2), - new KeyValuePair("a", 3), + new Tuple("a", 2), + new Tuple("a", 3), }, 1); var joined = l.Join(r, 2).Collect(); @@ -147,8 +148,8 @@ internal static void PairRDDJoinSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(joined.Contains(new KeyValuePair>("a", new Tuple(1, 2)))); - Assert.IsTrue(joined.Contains(new KeyValuePair>("a", new Tuple(1, 3)))); + Assert.IsTrue(joined.Contains(new Tuple>("a", new Tuple(1, 2)))); + Assert.IsTrue(joined.Contains(new Tuple>("a", new Tuple(1, 3)))); } } @@ -158,14 +159,14 @@ internal static void PairRDDLeftOuterJoinSample() var l = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 4), + new Tuple("a", 1), + new Tuple("b", 4), }, 2); var r = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 2), + new Tuple("a", 2), }, 1); var joined = l.LeftOuterJoin(r).Collect(); @@ -175,8 +176,8 @@ internal static void PairRDDLeftOuterJoinSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(joined.Any(kv => kv.Key == "a" && kv.Value.Item1 == 1 && kv.Value.Item2.IsDefined && kv.Value.Item2.GetValue() == 2)); - Assert.IsTrue(joined.Any(kv => kv.Key == "b" && kv.Value.Item1 == 4 && !kv.Value.Item2.IsDefined)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "a" && kv.Item2.Item1 == 1 && kv.Item2.Item2.IsDefined && kv.Item2.Item2.GetValue() == 2)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "b" && kv.Item2.Item1 == 4 && !kv.Item2.Item2.IsDefined)); } } @@ -186,14 +187,14 @@ internal static void PairRDDRightOuterJoinSample() var l = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 2), + new Tuple("a", 2), }, 1); var r = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 4), + new Tuple("a", 1), + new Tuple("b", 4), }, 2); var joined = l.RightOuterJoin(r).Collect(); @@ -203,8 +204,8 @@ internal static void PairRDDRightOuterJoinSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(joined.Any(kv => kv.Key == "a" && kv.Value.Item1.IsDefined && kv.Value.Item1.GetValue() == 2 && kv.Value.Item2 == 1)); - Assert.IsTrue(joined.Any(kv => kv.Key == "b" && !kv.Value.Item1.IsDefined && kv.Value.Item2 == 4)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "a" && kv.Item2.Item1.IsDefined && kv.Item2.Item1.GetValue() == 2 && kv.Item2.Item2 == 1)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "b" && !kv.Item2.Item1.IsDefined && kv.Item2.Item2 == 4)); } } @@ -214,15 +215,15 @@ internal static void PairRDDFullOuterJoinSample() var l = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 4), + new Tuple("a", 1), + new Tuple("b", 4), }, 2); var r = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 2), - new KeyValuePair("c", 8), + new Tuple("a", 2), + new Tuple("c", 8), }, 2); var joined = l.FullOuterJoin(r).Collect(); @@ -232,12 +233,12 @@ internal static void PairRDDFullOuterJoinSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(joined.Any(kv => kv.Key == "a" && kv.Value.Item1.IsDefined && kv.Value.Item1.GetValue() == 1 && - kv.Value.Item2.IsDefined && kv.Value.Item2.GetValue() == 2)); - Assert.IsTrue(joined.Any(kv => kv.Key == "b" && kv.Value.Item1.IsDefined && kv.Value.Item1.GetValue() == 4 && - !kv.Value.Item2.IsDefined)); - Assert.IsTrue(joined.Any(kv => kv.Key == "c" && !kv.Value.Item1.IsDefined && - kv.Value.Item2.IsDefined && kv.Value.Item2.GetValue() == 8)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "a" && kv.Item2.Item1.IsDefined && kv.Item2.Item1.GetValue() == 1 && + kv.Item2.Item2.IsDefined && kv.Item2.Item2.GetValue() == 2)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "b" && kv.Item2.Item1.IsDefined && kv.Item2.Item1.GetValue() == 4 && + !kv.Item2.Item2.IsDefined)); + Assert.IsTrue(joined.Any(kv => kv.Item1 == "c" && !kv.Item2.Item1.IsDefined && + kv.Item2.Item2.IsDefined && kv.Item2.Item2.GetValue() == 8)); } } @@ -252,7 +253,7 @@ internal static void PairRDDPartitionBySample() }; var partitioned = SparkCLRSamples.SparkContext.Parallelize(new[] { 1, 2, 3, 4, 5, 6, 1 }, 3) - .Map(x => new KeyValuePair(x, x + 100)) + .Map(x => new Tuple(x, x + 100)) .PartitionBy(3, partitionFunc) .Glom() .Collect(); @@ -270,9 +271,9 @@ internal static void PairRDDPartitionBySample() { Assert.AreEqual(3, partitioned.Length); // Assert that the partition distribution is correct with partitionFunc - Assert.IsTrue(partitioned.Count(p => p.All(key => key.Key < 3)) == 1); - Assert.IsTrue(partitioned.Count(p => p.All(key => key.Key >= 3 && key.Key < 6)) == 1); - Assert.IsTrue(partitioned.Count(p => p.All(key => key.Key >= 6)) == 1); + Assert.IsTrue(partitioned.Count(p => p.All(key => key.Item1 < 3)) == 1); + Assert.IsTrue(partitioned.Count(p => p.All(key => key.Item1 >= 3 && key.Item1 < 6)) == 1); + Assert.IsTrue(partitioned.Count(p => p.All(key => key.Item1 >= 6)) == 1); } } @@ -282,9 +283,9 @@ internal static void PairRDDCombineByKeySample() var combineByKey = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .CombineByKey(() => string.Empty, (x, y) => x + y.ToString(CultureInfo.InvariantCulture), (x, y) => x + y).Collect(); @@ -293,8 +294,8 @@ internal static void PairRDDCombineByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(combineByKey.Contains(new KeyValuePair("a", "11"))); - Assert.IsTrue(combineByKey.Contains(new KeyValuePair("b", "1"))); + Assert.IsTrue(combineByKey.Contains(new Tuple("a", "11"))); + Assert.IsTrue(combineByKey.Contains(new Tuple("b", "1"))); } } @@ -304,9 +305,9 @@ internal static void PairRDDAggregateByKeySample() var aggregateByKey = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .AggregateByKey(() => 0, (x, y) => x + y, (x, y) => x + y).Collect(); @@ -315,8 +316,8 @@ internal static void PairRDDAggregateByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(aggregateByKey.Contains(new KeyValuePair("a", 2))); - Assert.IsTrue(aggregateByKey.Contains(new KeyValuePair("b", 1))); + Assert.IsTrue(aggregateByKey.Contains(new Tuple("a", 2))); + Assert.IsTrue(aggregateByKey.Contains(new Tuple("b", 1))); } } @@ -326,9 +327,9 @@ internal static void PairRDDFoldByKeySample() var FoldByKey = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .FoldByKey(() => 0, (x, y) => x + y).Collect(); @@ -337,8 +338,8 @@ internal static void PairRDDFoldByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(FoldByKey.Contains(new KeyValuePair("a", 2))); - Assert.IsTrue(FoldByKey.Contains(new KeyValuePair("b", 1))); + Assert.IsTrue(FoldByKey.Contains(new Tuple("a", 2))); + Assert.IsTrue(FoldByKey.Contains(new Tuple("b", 1))); } } @@ -348,19 +349,19 @@ internal static void PairRDDGroupByKeySample() var groupByKey = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", 1), - new KeyValuePair("b", 1), - new KeyValuePair("a", 1) + new Tuple("a", 1), + new Tuple("b", 1), + new Tuple("a", 1) }, 2) .GroupByKey().Collect(); foreach (var kv in groupByKey) - Console.WriteLine(kv.Key + ", " + "(" + string.Join(",", kv.Value) + ")"); + Console.WriteLine(kv.Item1 + ", " + "(" + string.Join(",", kv.Item2) + ")"); if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(groupByKey.Any(kv => kv.Key == "a" && kv.Value.Count == 2 && kv.Value[0] == 1 && kv.Value[1] == 1)); - Assert.IsTrue(groupByKey.Any(kv => kv.Key == "b" && kv.Value.Count == 1 && kv.Value[0] == 1)); + Assert.IsTrue(groupByKey.Any(kv => kv.Item1 == "a" && kv.Item2.Count == 2 && kv.Item2[0] == 1 && kv.Item2[1] == 1)); + Assert.IsTrue(groupByKey.Any(kv => kv.Item1 == "b" && kv.Item2.Count == 1 && kv.Item2[0] == 1)); } } @@ -370,8 +371,8 @@ internal static void PairRDDMapValuesSample() var mapValues = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", new[]{"apple", "banana", "lemon"}), - new KeyValuePair("b", new[]{"grapes"}) + new Tuple("a", new[]{"apple", "banana", "lemon"}), + new Tuple("b", new[]{"grapes"}) }, 2) .MapValues(x => x.Length).Collect(); @@ -380,8 +381,8 @@ internal static void PairRDDMapValuesSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(mapValues.Any(kv => kv.Key == "a" && kv.Value == 3)); - Assert.IsTrue(mapValues.Any(kv => kv.Key == "b" && kv.Value == 1)); + Assert.IsTrue(mapValues.Any(kv => kv.Item1 == "a" && kv.Item2 == 3)); + Assert.IsTrue(mapValues.Any(kv => kv.Item1 == "b" && kv.Item2 == 1)); } } @@ -391,8 +392,8 @@ internal static void PairRDDFlatMapValuesSample() var flatMapValues = SparkCLRSamples.SparkContext.Parallelize( new[] { - new KeyValuePair("a", new[]{"x", "y", "z"}), - new KeyValuePair("b", new[]{"p", "r"}) + new Tuple("a", new[]{"x", "y", "z"}), + new Tuple("b", new[]{"p", "r"}) }, 2) .FlatMapValues(x => x).Collect(); @@ -401,48 +402,48 @@ internal static void PairRDDFlatMapValuesSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(flatMapValues.Any(kv => kv.Key == "a" && kv.Value == "x")); - Assert.IsTrue(flatMapValues.Any(kv => kv.Key == "a" && kv.Value == "y")); - Assert.IsTrue(flatMapValues.Any(kv => kv.Key == "a" && kv.Value == "z")); - Assert.IsTrue(flatMapValues.Any(kv => kv.Key == "b" && kv.Value == "p")); - Assert.IsTrue(flatMapValues.Any(kv => kv.Key == "b" && kv.Value == "r")); + Assert.IsTrue(flatMapValues.Any(kv => kv.Item1 == "a" && kv.Item2 == "x")); + Assert.IsTrue(flatMapValues.Any(kv => kv.Item1 == "a" && kv.Item2 == "y")); + Assert.IsTrue(flatMapValues.Any(kv => kv.Item1 == "a" && kv.Item2 == "z")); + Assert.IsTrue(flatMapValues.Any(kv => kv.Item1 == "b" && kv.Item2 == "p")); + Assert.IsTrue(flatMapValues.Any(kv => kv.Item1 == "b" && kv.Item2 == "r")); } } [Sample] internal static void PairRDDGroupWithSample() { - var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 1), new KeyValuePair("b", 4)}, 2); - var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 2)}, 1); + var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 1), new Tuple("b", 4)}, 2); + var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 2)}, 1); var groupWith = x.GroupWith(y).Collect(); foreach (var kv in groupWith) - Console.WriteLine(kv.Key + ", " + "(" + string.Join(",", kv.Value) + ")"); + Console.WriteLine(kv.Item1 + ", " + "(" + string.Join(",", kv.Item2) + ")"); if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(groupWith.Any(kv => kv.Key == "a" && kv.Value.Item1[0] == 1 && kv.Value.Item2[0] == 2)); - Assert.IsTrue(groupWith.Any(kv => kv.Key == "b" && kv.Value.Item1[0] == 4 && !kv.Value.Item2.Any())); + Assert.IsTrue(groupWith.Any(kv => kv.Item1 == "a" && kv.Item2.Item1[0] == 1 && kv.Item2.Item2[0] == 2)); + Assert.IsTrue(groupWith.Any(kv => kv.Item1 == "b" && kv.Item2.Item1[0] == 4 && !kv.Item2.Item2.Any())); } } [Sample] internal static void PairRDDGroupWithSample2() { - var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 5), new KeyValuePair("b", 6) }, 2); - var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 1), new KeyValuePair("b", 4) }, 2); - var z = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 2) }, 1); + var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 5), new Tuple("b", 6) }, 2); + var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 1), new Tuple("b", 4) }, 2); + var z = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 2) }, 1); var groupWith = x.GroupWith(y, z).Collect(); foreach (var kv in groupWith) - Console.WriteLine(kv.Key + ", " + "(" + string.Join(",", kv.Value) + ")"); + Console.WriteLine(kv.Item1 + ", " + "(" + string.Join(",", kv.Item2) + ")"); if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(groupWith.Any(kv => kv.Key == "a" && kv.Value.Item1[0] == 5 && kv.Value.Item2[0] == 1 && kv.Value.Item3[0] == 2)); - Assert.IsTrue(groupWith.Any(kv => kv.Key == "b" && kv.Value.Item1[0] == 6 && kv.Value.Item2[0] == 4 && !kv.Value.Item3.Any())); + Assert.IsTrue(groupWith.Any(kv => kv.Item1 == "a" && kv.Item2.Item1[0] == 5 && kv.Item2.Item2[0] == 1 && kv.Item2.Item3[0] == 2)); + Assert.IsTrue(groupWith.Any(kv => kv.Item1 == "b" && kv.Item2.Item1[0] == 6 && kv.Item2.Item2[0] == 4 && !kv.Item2.Item3.Any())); } } @@ -452,7 +453,7 @@ internal static void PairRDDGroupWithSample2() //{ // var fractions = new Dictionary { { "a", 0.2 }, { "b", 0.1 } }; // var rdd = SparkCLRSamples.SparkContext.Parallelize(fractions.Keys.ToArray(), 2).Cartesian(SparkCLRSamples.SparkContext.Parallelize(Enumerable.Range(0, 1000), 2)); - // var sample = rdd.Map(t => new KeyValuePair(t.Item1, t.Item2)).SampleByKey(false, fractions, 2).GroupByKey().Collect(); + // var sample = rdd.Map(t => new Tuple(t.Item1, t.Item2)).SampleByKey(false, fractions, 2).GroupByKey().Collect(); // Console.WriteLine(sample); //} @@ -460,8 +461,8 @@ internal static void PairRDDGroupWithSample2() [Sample] internal static void PairRDDSubtractByKeySample() { - var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 1), new KeyValuePair("b", 4), new KeyValuePair("b", 5), new KeyValuePair("a", 2) }, 2); - var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("a", 3), new KeyValuePair("c", null) }, 2); + var x = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 1), new Tuple("b", 4), new Tuple("b", 5), new Tuple("a", 2) }, 2); + var y = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("a", 3), new Tuple("c", null) }, 2); var subtractByKey = x.SubtractByKey(y).Collect(); @@ -471,15 +472,15 @@ internal static void PairRDDSubtractByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { Assert.AreEqual(2, subtractByKey.Length); - subtractByKey.Contains(new KeyValuePair("b", 4)); - subtractByKey.Contains(new KeyValuePair("b", 5)); + subtractByKey.Contains(new Tuple("b", 4)); + subtractByKey.Contains(new Tuple("b", 5)); } } [Sample] internal static void PairRDDLookupSample() { - var rdd = SparkCLRSamples.SparkContext.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new KeyValuePair(x, y)), 10); + var rdd = SparkCLRSamples.SparkContext.Parallelize(Enumerable.Range(0, 1000).Zip(Enumerable.Range(0, 1000), (x, y) => new Tuple(x, y)), 10); var lookup42 = rdd.Lookup(42); var lookup1024 = rdd.Lookup(1024); Console.WriteLine(string.Join(",", lookup42)); @@ -495,9 +496,9 @@ internal static void PairRDDLookupSample() [Sample] internal static void PairRDDSortByKeySample() { - var rdd = SparkCLRSamples.SparkContext.Parallelize(new[] { new KeyValuePair("B", 2), - new KeyValuePair("a", 1), new KeyValuePair("c", 3), - new KeyValuePair("E", 5), new KeyValuePair("D", 4)}, 3); + var rdd = SparkCLRSamples.SparkContext.Parallelize(new[] { new Tuple("B", 2), + new Tuple("a", 1), new Tuple("c", 3), + new Tuple("E", 5), new Tuple("D", 4)}, 3); var sortedRdd = rdd.SortByKey(true, 2); var sortedInTotal = sortedRdd.Collect(); @@ -507,7 +508,7 @@ internal static void PairRDDSortByKeySample() { Assert.AreEqual(2, sortedPartitions.Length); // by default SortByKey is case sensitive - CollectionAssert.AreEqual(new[] { "B", "D", "E", "a", "c" }, sortedInTotal.Select(kv => kv.Key).ToArray()); + CollectionAssert.AreEqual(new[] { "B", "D", "E", "a", "c" }, sortedInTotal.Select(kv => kv.Item1).ToArray()); } // convert the keys to lower case in order to sort with case insensitive @@ -518,7 +519,7 @@ internal static void PairRDDSortByKeySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { Assert.AreEqual(2, sortedPartitions.Length); - CollectionAssert.AreEqual(new[] { "a", "B", "c", "D", "E" }, sortedInTotal.Select(kv => kv.Key).ToArray()); + CollectionAssert.AreEqual(new[] { "a", "B", "c", "D", "E" }, sortedInTotal.Select(kv => kv.Item1).ToArray()); } } } diff --git a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs index bc201f82..37943c26 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs @@ -129,7 +129,7 @@ internal static void RDDGroupBySample() var rdd = SparkCLRSamples.SparkContext.Parallelize(new int[] { 1, 1, 2, 3, 5, 8 }, 1); var groups = rdd.GroupBy(x => x % 2).Collect(); foreach (var kv in groups) - Console.WriteLine(kv.Key + ", " + string.Join(",", kv.Value)); + Console.WriteLine(kv.Item1 + ", " + string.Join(",", kv.Item2)); if (SparkCLRSamples.Configuration.IsValidationEnabled) { @@ -137,9 +137,9 @@ internal static void RDDGroupBySample() foreach (var kv in groups) { // the group with key=1 is odd numbers - if (kv.Key == 1) CollectionAssert.AreEquivalent(new[] { 1, 1, 3, 5 }, kv.Value); + if (kv.Item1 == 1) CollectionAssert.AreEquivalent(new[] { 1, 1, 3, 5 }, kv.Item2); // the group with key=0 is even numbers - else if (kv.Key == 0) CollectionAssert.AreEquivalent(new[] { 2, 8 }, kv.Value); + else if (kv.Item1 == 0) CollectionAssert.AreEquivalent(new[] { 2, 8 }, kv.Item2); } } } @@ -292,10 +292,10 @@ internal static void RDDKeyBySample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(keyBy.Contains(new KeyValuePair(1, 1))); - Assert.IsTrue(keyBy.Contains(new KeyValuePair(4, 2))); - Assert.IsTrue(keyBy.Contains(new KeyValuePair(9, 3))); - Assert.IsTrue(keyBy.Contains(new KeyValuePair(16, 4))); + Assert.IsTrue(keyBy.Contains(new Tuple(1, 1))); + Assert.IsTrue(keyBy.Contains(new Tuple(4, 2))); + Assert.IsTrue(keyBy.Contains(new Tuple(9, 3))); + Assert.IsTrue(keyBy.Contains(new Tuple(16, 4))); } } @@ -344,7 +344,7 @@ internal static void RDDZipSample() { for (int i = 0; i < 5; i++) { - Assert.IsTrue(zip.Contains(new KeyValuePair(i, 1000 + i))); + Assert.IsTrue(zip.Contains(new Tuple(i, 1000 + i))); } } } @@ -358,10 +358,10 @@ internal static void RDDZipWithIndexSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(zipWithIndex.Contains(new KeyValuePair("a", 0))); - Assert.IsTrue(zipWithIndex.Contains(new KeyValuePair("b", 1))); - Assert.IsTrue(zipWithIndex.Contains(new KeyValuePair("c", 2))); - Assert.IsTrue(zipWithIndex.Contains(new KeyValuePair("d", 3))); + Assert.IsTrue(zipWithIndex.Contains(new Tuple("a", 0))); + Assert.IsTrue(zipWithIndex.Contains(new Tuple("b", 1))); + Assert.IsTrue(zipWithIndex.Contains(new Tuple("c", 2))); + Assert.IsTrue(zipWithIndex.Contains(new Tuple("d", 3))); } } @@ -374,11 +374,11 @@ internal static void RDDZipWithUniqueIdSample() if (SparkCLRSamples.Configuration.IsValidationEnabled) { - Assert.IsTrue(zipWithUniqueId.Contains(new KeyValuePair("a", 0))); - Assert.IsTrue(zipWithUniqueId.Contains(new KeyValuePair("b", 1))); - Assert.IsTrue(zipWithUniqueId.Contains(new KeyValuePair("c", 4))); - Assert.IsTrue(zipWithUniqueId.Contains(new KeyValuePair("d", 2))); - Assert.IsTrue(zipWithUniqueId.Contains(new KeyValuePair("e", 5))); + Assert.IsTrue(zipWithUniqueId.Contains(new Tuple("a", 0))); + Assert.IsTrue(zipWithUniqueId.Contains(new Tuple("b", 1))); + Assert.IsTrue(zipWithUniqueId.Contains(new Tuple("c", 4))); + Assert.IsTrue(zipWithUniqueId.Contains(new Tuple("d", 2))); + Assert.IsTrue(zipWithUniqueId.Contains(new Tuple("e", 5))); } } @@ -530,22 +530,22 @@ internal static void RDDWordCountSample() var words = lines.FlatMap(s => s.Split(' ')); - var wordCounts = words.Map(w => new KeyValuePair(w.Trim(), 1)) + var wordCounts = words.Map(w => new Tuple(w.Trim(), 1)) .ReduceByKey((x, y) => x + y).Collect(); Console.WriteLine("*** Printing words and their counts ***"); foreach (var kvp in wordCounts) { - Console.WriteLine("'{0}':{1}", kvp.Key, kvp.Value); + Console.WriteLine("'{0}':{1}", kvp.Item1, kvp.Item2); } - var wordCountsCaseInsensitve = words.Map(w => new KeyValuePair(w.ToLower().Trim(), 1)) + var wordCountsCaseInsensitve = words.Map(w => new Tuple(w.ToLower().Trim(), 1)) .ReduceByKey((x, y) => x + y).Collect(); Console.WriteLine("*** Printing words and their counts ignoring case ***"); foreach (var kvp in wordCountsCaseInsensitve) { - Console.WriteLine("'{0}':{1}", kvp.Key, kvp.Value); + Console.WriteLine("'{0}':{1}", kvp.Item1, kvp.Item2); } if (SparkCLRSamples.Configuration.IsValidationEnabled) @@ -553,7 +553,7 @@ internal static void RDDWordCountSample() var dictionary = new Dictionary(); foreach (var kvp in wordCounts) { - dictionary[kvp.Key] = kvp.Value; + dictionary[kvp.Item1] = kvp.Item2; } Assert.AreEqual(22, dictionary["the"]); @@ -563,7 +563,7 @@ internal static void RDDWordCountSample() var caseInsenstiveWordCountDictionary = new Dictionary(); foreach (var kvp in wordCountsCaseInsensitve) { - caseInsenstiveWordCountDictionary[kvp.Key] = kvp.Value; + caseInsenstiveWordCountDictionary[kvp.Item1] = kvp.Item2; } Assert.AreEqual(45, caseInsenstiveWordCountDictionary["the"]); @@ -584,12 +584,12 @@ internal static void RDDJoinSample() var requestsColumns = requests.Map(s => { var columns = s.Split(','); - return new KeyValuePair(columns[0], new[] { columns[1], columns[2], columns[3] }); + return new Tuple(columns[0], new[] { columns[1], columns[2], columns[3] }); }); var metricsColumns = metrics.Map(s => { var columns = s.Split(','); - return new KeyValuePair(columns[3], new[] { columns[4], columns[5], columns[6] }); + return new Tuple(columns[3], new[] { columns[4], columns[5], columns[6] }); }); var requestsJoinedWithMetrics = requestsColumns.Join(metricsColumns) @@ -597,29 +597,29 @@ internal static void RDDJoinSample() s => new [] { - s.Key, //guid - s.Value.Item1[0], s.Value.Item1[1], s.Value.Item1[2], //dc, abtestid, traffictype - s.Value.Item2[0],s.Value.Item2[1], s.Value.Item2[2] //lang, country, metric + s.Item1, //guid + s.Item2.Item1[0], s.Item2.Item1[1], s.Item2.Item1[2], //dc, abtestid, traffictype + s.Item2.Item2[0],s.Item2.Item2[1], s.Item2.Item2[2] //lang, country, metric }); - var latencyByDatacenter = requestsJoinedWithMetrics.Map(i => new KeyValuePair (i[1], int.Parse(i[6]))); //key is "datacenter" + var latencyByDatacenter = requestsJoinedWithMetrics.Map(i => new Tuple (i[1], int.Parse(i[6]))); //key is "datacenter" var maxLatencyByDataCenterList = latencyByDatacenter.ReduceByKey(Math.Max).Collect(); Console.WriteLine("***** Max latency metrics by DC *****"); - foreach (var keyValuePair in maxLatencyByDataCenterList) + foreach (var Tuple in maxLatencyByDataCenterList) { - Console.WriteLine("Datacenter={0}, Max latency={1}", keyValuePair.Key, keyValuePair.Value); + Console.WriteLine("Datacenter={0}, Max latency={1}", Tuple.Item1, Tuple.Item2); } - var latencyAndCountByDatacenter = requestsJoinedWithMetrics.Map(i => new KeyValuePair> (i[1], new Tuple(int.Parse(i[6]), 1))); + var latencyAndCountByDatacenter = requestsJoinedWithMetrics.Map(i => new Tuple> (i[1], new Tuple(int.Parse(i[6]), 1))); var sumLatencyAndCountByDatacenter = latencyAndCountByDatacenter.ReduceByKey((tuple, tuple1) => new Tuple((tuple == null ? 0 : tuple.Item1) + tuple1.Item1, (tuple == null ? 0 : tuple.Item2) + tuple1.Item2)); var sumLatencyAndCountByDatacenterList = sumLatencyAndCountByDatacenter.Collect(); Console.WriteLine("***** Mean latency metrics by DC *****"); - foreach (var keyValuePair in sumLatencyAndCountByDatacenterList) + foreach (var Tuple in sumLatencyAndCountByDatacenterList) { - Console.WriteLine("Datacenter={0}, Mean latency={1}", keyValuePair.Key, keyValuePair.Value.Item1/keyValuePair.Value.Item2); + Console.WriteLine("Datacenter={0}, Mean latency={1}", Tuple.Item1, Tuple.Item2.Item1/Tuple.Item2.Item2); } if (SparkCLRSamples.Configuration.IsValidationEnabled) @@ -627,7 +627,7 @@ internal static void RDDJoinSample() var dictionary = new Dictionary(); foreach (var kvp in maxLatencyByDataCenterList) { - dictionary[kvp.Key] = kvp.Value; + dictionary[kvp.Item1] = kvp.Item2; } Assert.AreEqual(835, dictionary["iowa"]); @@ -636,7 +636,7 @@ internal static void RDDJoinSample() var meanDictionary = new Dictionary>(); foreach (var kvp in sumLatencyAndCountByDatacenterList) { - meanDictionary[kvp.Key] = new Tuple(kvp.Value.Item1, kvp.Value.Item2); + meanDictionary[kvp.Item1] = new Tuple(kvp.Item2.Item1, kvp.Item2.Item2); } Assert.AreEqual(1621, meanDictionary["iowa"].Item1); @@ -737,7 +737,7 @@ internal static void RDDCombineBySample() var markets = SparkCLRSamples.SparkContext.TextFile(SparkCLRSamples.Configuration.GetInputDataPath("market.tab"), 1); long totalMarketsCount = markets.Count(); - var marketsByKey = markets.Map(x => new KeyValuePair(x.Substring(0, x.IndexOf('-')), x)); + var marketsByKey = markets.Map(x => new Tuple(x.Substring(0, x.IndexOf('-')), x)); var categories = marketsByKey.PartitionBy(2) .CombineByKey(() => "", (c, v) => v.Substring(0, v.IndexOf('-')), (c1, c2) => c1, 2); var categoriesCollectedCount = categories.Collect().Count(); From 873286cb15add1a83140724931c8e86b103db5ee Mon Sep 17 00:00:00 2001 From: Isaac Abraham Date: Thu, 21 Apr 2016 00:54:33 +0100 Subject: [PATCH 02/20] Updated a few more methods. --- .../Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs | 4 ++-- csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs | 4 ++-- csharp/AdapterTest/PairRDDTest.cs | 2 +- csharp/AdapterTest/RDDTest.cs | 2 +- csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs | 2 +- csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs index a40c6197..8600e498 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs @@ -146,9 +146,9 @@ public static IDictionary ReduceByKeyLocally(this RDD> s /// /// /// - public static IDictionary CountByKey(this RDD> self) + public static IEnumerable> CountByKey(this RDD> self) { - return self.MapValues(v => 1L).ReduceByKey((a, b) => a + b).CollectAsMap(); + return self.MapValues(v => 1L).ReduceByKey((a, b) => a + b).Collect(); } /// diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs index 345672f7..f1af3edd 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs @@ -762,9 +762,9 @@ public long Count() /// /// /// - public IDictionary CountByValue() + public IEnumerable> CountByValue() { - return Map>(v => new Tuple(v, default(T))).CountByKey(); + return Map(v => new Tuple(v, default(T))).CountByKey(); } /// diff --git a/csharp/AdapterTest/PairRDDTest.cs b/csharp/AdapterTest/PairRDDTest.cs index 00dd3e99..7fe06083 100644 --- a/csharp/AdapterTest/PairRDDTest.cs +++ b/csharp/AdapterTest/PairRDDTest.cs @@ -27,7 +27,7 @@ public void TestPairRddCountByKey() foreach (var record in pairs.CountByKey()) { // the 1st paramter of AreEqual() method is the expected value, the 2nd one is the acutal value. - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2); } } diff --git a/csharp/AdapterTest/RDDTest.cs b/csharp/AdapterTest/RDDTest.cs index 23bbdc11..ab406b32 100644 --- a/csharp/AdapterTest/RDDTest.cs +++ b/csharp/AdapterTest/RDDTest.cs @@ -42,7 +42,7 @@ public void TestRddCountByValue() { foreach (var record in words.CountByValue()) { - Assert.AreEqual(record.Key == "The" || record.Key == "dog" || record.Key == "lazy" ? 23 : 22, record.Value); + Assert.AreEqual(record.Item1 == "The" || record.Item1 == "dog" || record.Item1 == "lazy" ? 23 : 22, record.Item2); } } diff --git a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs index 97604337..22de846f 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs @@ -112,7 +112,7 @@ internal static void PairRDDCountByKeySample() new Tuple("a", 1) }, 2) .CountByKey() - .ToLookup(x => x.Key); + .ToLookup(x => x.Item1); foreach (var kv in countByKey) Console.WriteLine(kv); diff --git a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs index 37943c26..62477c3f 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs @@ -221,7 +221,7 @@ internal static void RDDTreeAggregateSample() [Sample] internal static void RDDCountByValueSample() { - var countByValue = SparkCLRSamples.SparkContext.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2).CountByValue(); + var countByValue = SparkCLRSamples.SparkContext.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2).CountByValue().ToArray(); foreach (var item in countByValue) Console.WriteLine(item); From 34567724738ce095c268c8e1408a48504cb440bc Mon Sep 17 00:00:00 2001 From: Kaarthik Sivashanmugam Date: Fri, 7 Oct 2016 17:40:49 -0700 Subject: [PATCH 03/20] updating HiveExample to use SparkSession --- examples/Sql/HiveDataFrame/Program.cs | 38 ++++++++++++++++++++------- 1 file changed, 29 insertions(+), 9 deletions(-) diff --git a/examples/Sql/HiveDataFrame/Program.cs b/examples/Sql/HiveDataFrame/Program.cs index 101dd206..d488cfa3 100644 --- a/examples/Sql/HiveDataFrame/Program.cs +++ b/examples/Sql/HiveDataFrame/Program.cs @@ -1,4 +1,4 @@ -// Copyright (c) Microsoft. All rights reserved. +// Copyright (c) Microsoft. All rights reserved. // Licensed under the MIT license. See LICENSE file in the project root for full license information. using System; @@ -20,21 +20,40 @@ static void Main(string[] args) { LoggerServiceFactory.SetLoggerService(Log4NetLoggerService.Instance); //this is optional - DefaultLoggerService will be used if not set var logger = LoggerServiceFactory.GetLogger(typeof(HiveDataFrameExample)); + var jsonFilePath = args[0]; + const string dbName = "SampleHiveDataBaseForMobius"; + const string tableName = "people"; + + var builder = SparkSession.Builder().EnableHiveSupport(); + // The following setting is required to use Spark 2.0 in Windows + // It may be provided in command line when running Mobius app + //builder = builder.Config("spark.sql.warehouse.dir", ""); + var session = builder.GetOrCreate(); + var peopleDataFrame = session.Read().Json(jsonFilePath); + session.Sql(string.Format("CREATE DATABASE IF NOT EXISTS {0}", dbName)); // create database if not exists + session.Sql(string.Format("USE {0}", dbName)); + //hiveContext.Sql(string.Format("DROP TABLE {0}", tableName)); // drop table if exists + + peopleDataFrame.Write().Mode(SaveMode.Overwrite).SaveAsTable(tableName); // create table + var tablesDataFrame = session.Table(tableName); // get all tables in database + logger.LogInfo(string.Format("table count in database {0}: {1}", dbName, tablesDataFrame.Count())); + tablesDataFrame.Show(); + + session.Sql(string.Format("SELECT * FROM {0}", tableName)).Show(); // select from table + // Following example is for the deprecated API + /* var sparkConf = new SparkConf(); + // The following setting is required to use Spark 2.0 in Windows + // It may be provided in command line when running Mobius app + //sparkConf.Set("spark.sql.warehouse.dir", @""); var sparkContext = new SparkContext(sparkConf); - var hiveContext = new HiveContext(sparkContext); - - // please give the path to input json file - var jsonFilePath = args[0]; + var hiveContext = new HiveContext(sparkContext); var peopleDataFrame = hiveContext.Read().Json(jsonFilePath); - - const string dbName = "SampleHiveDataBaseForMobius"; - const string tableName = "people"; hiveContext.Sql(string.Format("CREATE DATABASE IF NOT EXISTS {0}", dbName)); // create database if not exists hiveContext.Sql(string.Format("USE {0}", dbName)); - hiveContext.Sql(string.Format("DROP TABLE {0}", tableName)); // drop table if exists + //hiveContext.Sql(string.Format("DROP TABLE {0}", tableName)); // drop table if exists peopleDataFrame.Write().Mode(SaveMode.Overwrite).SaveAsTable(tableName); // create table var tablesDataFrame = hiveContext.Tables(dbName); // get all tables in database @@ -42,6 +61,7 @@ static void Main(string[] args) tablesDataFrame.Show(); hiveContext.Sql(string.Format("SELECT * FROM {0}", tableName)).Show(); // select from table + */ } } } From 8597df2d0f4a9ac823cff138e256361fda184054 Mon Sep 17 00:00:00 2001 From: Kaarthik Sivashanmugam Date: Tue, 11 Oct 2016 16:20:41 -0700 Subject: [PATCH 04/20] updating examples to use latest preview release --- dev/scripts/SetSparkClrJarVersion.ps1 | 4 ++-- examples/Batch/WordCount/WordCount.csproj | 10 +++++----- examples/Batch/WordCount/packages.config | 2 +- examples/Batch/pi/Pi.csproj | 12 ++++++------ examples/Batch/pi/packages.config | 2 +- .../Sql/CassandraDataFrame/CassandraDataFrame.csproj | 4 ++-- examples/Sql/CassandraDataFrame/packages.config | 2 +- examples/Sql/HiveDataFrame/HiveDataFrame.csproj | 8 ++++---- examples/Sql/HiveDataFrame/packages.config | 2 +- examples/Sql/JdbcDataFrame/JdbcDataFrame.csproj | 10 +++++----- examples/Sql/JdbcDataFrame/packages.config | 2 +- examples/Sql/SparkXml/SparkXml.csproj | 10 +++++----- examples/Sql/SparkXml/packages.config | 2 +- examples/Streaming/EventHub/EventHub.csproj | 6 +++--- examples/Streaming/EventHub/packages.config | 2 +- .../Streaming/HdfsWordCount/HdfsWordCount.csproj | 6 +++--- examples/Streaming/HdfsWordCount/packages.config | 2 +- examples/Streaming/Kafka/Kafka.csproj | 4 ++-- examples/Streaming/Kafka/packages.config | 2 +- examples/fsharp/JsonDataFrame/JsonDataFrame.fsproj | 4 ++-- examples/fsharp/JsonDataFrame/packages.config | 2 +- examples/fsharp/WordCount/WordCountFSharp.fsproj | 4 ++-- examples/fsharp/WordCount/packages.config | 2 +- 23 files changed, 52 insertions(+), 52 deletions(-) diff --git a/dev/scripts/SetSparkClrJarVersion.ps1 b/dev/scripts/SetSparkClrJarVersion.ps1 index f8d06ead..604ff548 100644 --- a/dev/scripts/SetSparkClrJarVersion.ps1 +++ b/dev/scripts/SetSparkClrJarVersion.ps1 @@ -45,7 +45,7 @@ function Update-SparkClrSubmit($targetDir, $version) # Get-ChildItem $targetDir -filter "sparkclr-submit.cmd" -recurs | % { Write-Output "[SetSparkClrJarVersion.Update-SparkClrSubmit] updating $($_.FullName)" - ((Get-Content $_.FullName) -replace "\(set SPARKCLR_JAR=.*\)", "(set SPARKCLR_JAR=spark-clr_2.10-$version.jar)") | Set-Content $_.FullName -force + ((Get-Content $_.FullName) -replace "\(set SPARKCLR_JAR=.*\)", "(set SPARKCLR_JAR=spark-clr_2.11-$version.jar)") | Set-Content $_.FullName -force } Write-Output "[SetSparkClrJarVersion.Update-SparkClrSubmit] Done setting sparkclr-submit.cmd under $targetDir to version=$version" @@ -59,7 +59,7 @@ function Update-SparkClrSubmit($targetDir, $version) # Get-ChildItem $targetDir -filter "sparkclr-submit.sh" -recurs | % { Write-Output "[SetSparkClrJarVersion.Update-SparkClrSubmit] updating $($_.FullName)" - ((Get-Content $_.FullName) -replace "export SPARKCLR_JAR=.*", "export SPARKCLR_JAR=spark-clr_2.10-$version.jar") | Set-Content $_.FullName -force + ((Get-Content $_.FullName) -replace "export SPARKCLR_JAR=.*", "export SPARKCLR_JAR=spark-clr_2.11-$version.jar") | Set-Content $_.FullName -force } Write-Output "[SetSparkClrJarVersion.Update-SparkClrSubmit] Done setting sparkclr-submit.sh under $targetDir to version=$version" diff --git a/examples/Batch/WordCount/WordCount.csproj b/examples/Batch/WordCount/WordCount.csproj index 40b8f52a..054cbbb3 100644 --- a/examples/Batch/WordCount/WordCount.csproj +++ b/examples/Batch/WordCount/WordCount.csproj @@ -32,17 +32,17 @@ 4 - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -67,7 +67,7 @@ - + CSharpWorker.exe.config diff --git a/examples/Batch/WordCount/packages.config b/examples/Batch/WordCount/packages.config index 293105d3..e8bf6251 100644 --- a/examples/Batch/WordCount/packages.config +++ b/examples/Batch/WordCount/packages.config @@ -4,5 +4,5 @@ - + diff --git a/examples/Batch/pi/Pi.csproj b/examples/Batch/pi/Pi.csproj index 751852f8..0417fe0a 100644 --- a/examples/Batch/pi/Pi.csproj +++ b/examples/Batch/pi/Pi.csproj @@ -35,17 +35,17 @@ 4 - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -66,7 +66,7 @@ - + CSharpWorker.exe.config @@ -84,4 +84,4 @@ --> - + \ No newline at end of file diff --git a/examples/Batch/pi/packages.config b/examples/Batch/pi/packages.config index eaa63869..dec53cfc 100644 --- a/examples/Batch/pi/packages.config +++ b/examples/Batch/pi/packages.config @@ -1,7 +1,7 @@  - + diff --git a/examples/Sql/CassandraDataFrame/CassandraDataFrame.csproj b/examples/Sql/CassandraDataFrame/CassandraDataFrame.csproj index 228764bd..87638435 100644 --- a/examples/Sql/CassandraDataFrame/CassandraDataFrame.csproj +++ b/examples/Sql/CassandraDataFrame/CassandraDataFrame.csproj @@ -35,13 +35,13 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False diff --git a/examples/Sql/CassandraDataFrame/packages.config b/examples/Sql/CassandraDataFrame/packages.config index 293105d3..e8bf6251 100644 --- a/examples/Sql/CassandraDataFrame/packages.config +++ b/examples/Sql/CassandraDataFrame/packages.config @@ -4,5 +4,5 @@ - + diff --git a/examples/Sql/HiveDataFrame/HiveDataFrame.csproj b/examples/Sql/HiveDataFrame/HiveDataFrame.csproj index 6c81a1d4..f682b02c 100644 --- a/examples/Sql/HiveDataFrame/HiveDataFrame.csproj +++ b/examples/Sql/HiveDataFrame/HiveDataFrame.csproj @@ -37,12 +37,12 @@ False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe True - - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll True diff --git a/examples/Sql/HiveDataFrame/packages.config b/examples/Sql/HiveDataFrame/packages.config index 218d018f..fc7565bb 100644 --- a/examples/Sql/HiveDataFrame/packages.config +++ b/examples/Sql/HiveDataFrame/packages.config @@ -1,7 +1,7 @@  - + diff --git a/examples/Sql/JdbcDataFrame/JdbcDataFrame.csproj b/examples/Sql/JdbcDataFrame/JdbcDataFrame.csproj index f19954c4..28f78836 100644 --- a/examples/Sql/JdbcDataFrame/JdbcDataFrame.csproj +++ b/examples/Sql/JdbcDataFrame/JdbcDataFrame.csproj @@ -34,17 +34,17 @@ 4 - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -65,7 +65,7 @@ - + CSharpWorker.exe.config diff --git a/examples/Sql/JdbcDataFrame/packages.config b/examples/Sql/JdbcDataFrame/packages.config index 293105d3..e8bf6251 100644 --- a/examples/Sql/JdbcDataFrame/packages.config +++ b/examples/Sql/JdbcDataFrame/packages.config @@ -4,5 +4,5 @@ - + diff --git a/examples/Sql/SparkXml/SparkXml.csproj b/examples/Sql/SparkXml/SparkXml.csproj index 381ec5db..63407574 100644 --- a/examples/Sql/SparkXml/SparkXml.csproj +++ b/examples/Sql/SparkXml/SparkXml.csproj @@ -34,17 +34,17 @@ 4 - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - + False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -65,7 +65,7 @@ - + CSharpWorker.exe.config diff --git a/examples/Sql/SparkXml/packages.config b/examples/Sql/SparkXml/packages.config index 293105d3..e8bf6251 100644 --- a/examples/Sql/SparkXml/packages.config +++ b/examples/Sql/SparkXml/packages.config @@ -4,5 +4,5 @@ - + diff --git a/examples/Streaming/EventHub/EventHub.csproj b/examples/Streaming/EventHub/EventHub.csproj index 631b2c3d..dec83bcb 100644 --- a/examples/Streaming/EventHub/EventHub.csproj +++ b/examples/Streaming/EventHub/EventHub.csproj @@ -35,7 +35,7 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False @@ -43,7 +43,7 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -68,7 +68,7 @@ - + CSharpWorker.exe.config diff --git a/examples/Streaming/EventHub/packages.config b/examples/Streaming/EventHub/packages.config index abe733c5..4deb4954 100644 --- a/examples/Streaming/EventHub/packages.config +++ b/examples/Streaming/EventHub/packages.config @@ -4,7 +4,7 @@ - + diff --git a/examples/Streaming/HdfsWordCount/HdfsWordCount.csproj b/examples/Streaming/HdfsWordCount/HdfsWordCount.csproj index 277fa405..22763af3 100644 --- a/examples/Streaming/HdfsWordCount/HdfsWordCount.csproj +++ b/examples/Streaming/HdfsWordCount/HdfsWordCount.csproj @@ -38,7 +38,7 @@ False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False @@ -46,7 +46,7 @@ False - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False @@ -64,7 +64,7 @@ - + PreserveNewest diff --git a/examples/Streaming/HdfsWordCount/packages.config b/examples/Streaming/HdfsWordCount/packages.config index eaa63869..dec53cfc 100644 --- a/examples/Streaming/HdfsWordCount/packages.config +++ b/examples/Streaming/HdfsWordCount/packages.config @@ -1,7 +1,7 @@  - + diff --git a/examples/Streaming/Kafka/Kafka.csproj b/examples/Streaming/Kafka/Kafka.csproj index c221fa81..f05b5b94 100644 --- a/examples/Streaming/Kafka/Kafka.csproj +++ b/examples/Streaming/Kafka/Kafka.csproj @@ -33,14 +33,14 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe False ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll False diff --git a/examples/Streaming/Kafka/packages.config b/examples/Streaming/Kafka/packages.config index eaa63869..dec53cfc 100644 --- a/examples/Streaming/Kafka/packages.config +++ b/examples/Streaming/Kafka/packages.config @@ -1,7 +1,7 @@  - + diff --git a/examples/fsharp/JsonDataFrame/JsonDataFrame.fsproj b/examples/fsharp/JsonDataFrame/JsonDataFrame.fsproj index b49e9de0..1e38738c 100644 --- a/examples/fsharp/JsonDataFrame/JsonDataFrame.fsproj +++ b/examples/fsharp/JsonDataFrame/JsonDataFrame.fsproj @@ -66,13 +66,13 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe ..\..\packages\log4net.2.0.5\lib\net45-full\log4net.dll - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll diff --git a/examples/fsharp/JsonDataFrame/packages.config b/examples/fsharp/JsonDataFrame/packages.config index 941d504a..e0e1e271 100644 --- a/examples/fsharp/JsonDataFrame/packages.config +++ b/examples/fsharp/JsonDataFrame/packages.config @@ -2,7 +2,7 @@ - + diff --git a/examples/fsharp/WordCount/WordCountFSharp.fsproj b/examples/fsharp/WordCount/WordCountFSharp.fsproj index 3b40aad9..facc6813 100644 --- a/examples/fsharp/WordCount/WordCountFSharp.fsproj +++ b/examples/fsharp/WordCount/WordCountFSharp.fsproj @@ -71,7 +71,7 @@ - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\CSharpWorker.exe + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\CSharpWorker.exe True @@ -83,7 +83,7 @@ True - ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-1\lib\net45\Microsoft.Spark.CSharp.Adapter.dll + ..\..\packages\Microsoft.SparkCLR.2.0.0-PREVIEW-2\lib\net45\Microsoft.Spark.CSharp.Adapter.dll True diff --git a/examples/fsharp/WordCount/packages.config b/examples/fsharp/WordCount/packages.config index 941d504a..e0e1e271 100644 --- a/examples/fsharp/WordCount/packages.config +++ b/examples/fsharp/WordCount/packages.config @@ -2,7 +2,7 @@ - + From 142701109b6056973f486ffd7b8a043d987bd6dd Mon Sep 17 00:00:00 2001 From: Kaarthik Sivashanmugam Date: Mon, 17 Oct 2016 11:58:54 -0700 Subject: [PATCH 05/20] documentation for implementing Spark Apps in F# using Mobius --- README.md | 1 + notes/mobius-init.fsx | 31 +++++++++++++++++++++++++++++++ notes/spark-fsharp-mobius.md | 17 +++++++++++++++++ 3 files changed, 49 insertions(+) create mode 100644 notes/mobius-init.fsx create mode 100644 notes/spark-fsharp-mobius.md diff --git a/README.md b/README.md index 55b3d28c..ff598731 100644 --- a/README.md +++ b/README.md @@ -119,6 +119,7 @@ Refer to the [docs folder](docs) for design overview and other info on Mobius * [Configuration parameters in Mobius](./notes/configuration-mobius.md) * [Troubleshoot errors in Mobius](./notes/troubleshooting-mobius.md) * [Debug Mobius apps](./notes/running-mobius-app.md#debug-mode) +* [Implementing Spark Apps in F# using Mobius](./notes/spark-fsharp-mobius.md) ## Supported Spark Versions diff --git a/notes/mobius-init.fsx b/notes/mobius-init.fsx new file mode 100644 index 00000000..11e28e10 --- /dev/null +++ b/notes/mobius-init.fsx @@ -0,0 +1,31 @@ +// *** Replace the paths below to point to correct location of Mobius binaries *** +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\Microsoft.Spark.CSharp.Adapter.dll" +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\log4net.dll" +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\Newtonsoft.Json.dll" +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\Razorvine.Pyrolite.dll" +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\Razorvine.Serpent.dll" +#r @"C:\spark-clr_2.11-2.0.000\runtime\bin\CSharpWorker.exe" +open Microsoft.Spark.CSharp.Core +open Microsoft.Spark.CSharp.Services +open Microsoft.Spark.CSharp.Sql +open System.Reflection +open System.Collections.Generic +LoggerServiceFactory.SetLoggerService Log4NetLoggerService.Instance + +// *** Uncomment & use the following code block to use SqlContext API *** +//let conf = SparkConf().SetAppName "FSharpInteractiveShell" +// *** uncomment & update master URL if running in non-local mode *** +//conf.Master "spark://sparkmaster:7077" +// *** Spark 2.0 in Windows requires the following config *** +//conf.Set("spark.sql.warehouse.dir", @"file:///C:/sparktemp") +//let sc = SparkContext conf +//let sqlContext = SqlContext sc + +// *** Uncomment & use the following code block to use SparkSession API *** +let builder = SparkSession.Builder() +builder = builder.AppName "FSharpInteractiveShell" +// *** uncomment & update master URL if running in non-local mode *** +//builder = builder.Master "spark://sparkmaster:7077" +// *** Spark 2.0 in Windows requires the following config *** +builder = builder.Config("spark.sql.warehouse.dir", "file:///C:/sparktemp") +let session = builder.GetOrCreate() diff --git a/notes/spark-fsharp-mobius.md b/notes/spark-fsharp-mobius.md new file mode 100644 index 00000000..7ff6372c --- /dev/null +++ b/notes/spark-fsharp-mobius.md @@ -0,0 +1,17 @@ +# Implementing Spark Apps in F# using Mobius + +## Non-Interactive Apps +1. Develop your application in a F# IDE using Mobius API. Refer to [F# examples](../examples/fsharp) for sample code +2. Use [`sparkclr-submit.cmd`](running-mobius-app.md) to run your Mobius-based Spark application implemented in F# + +## Interactive Apps +### Using F# Interactive (fsi.exe) +1. Run `sparkclr-submit.cmd debug` in a command prompt after setting necessary [environment variables](running-mobius-app.md#pre-requisites). Note that this `debug` parameter is a misnomer in this context and this command initializes .NET-JVM bridge similiar to [running Mobius apps in debug mode](./running-mobius-app.md#debug-mode). +2. In Developer Command Prompt for VS, run `fsi.exe --use:c:\temp\mobius-init.fsx`. [mobius-init.fsx](mobius-init.fsx) has the initialization code that can be used to create `SparkContext`, `SqlContext` or `SparkSession`. You need to update the location of Mobius binaries referenced in the beginning of the script file. You may also need to update other configuration settings in the script. +3. When the F# command prompt is available, Spark functionality can be invoked using Mobius API. For example, the following code can be used process JSON file. +``` +let dataframe = sparkSession.Read().Json @"C:\temp\data.json";; +dataframe.Show();; +dataframe.ShowSchema();; +dataframe.Count();; +``` From 5311e3027a119aa3b8b08f047902925efedc1675 Mon Sep 17 00:00:00 2001 From: Kai Zeng Date: Tue, 25 Oct 2016 14:06:42 -0700 Subject: [PATCH 06/20] add RunJob method to SparkContext; add SparkContext property to RDD (#590) * add RunJob method to SparkContext; add SparkContext property to RDD * test RDD.SparkContext --- .../Adapter/Microsoft.Spark.CSharp/Core/RDD.cs | 11 +++++++++++ .../Microsoft.Spark.CSharp/Core/SparkContext.cs | 11 +++++++++++ csharp/AdapterTest/SparkContextTest.cs | 17 +++++++++++++++++ 3 files changed, 39 insertions(+) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs index 44111778..31a0edc7 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs @@ -51,6 +51,17 @@ internal virtual IRDDProxy RddProxy } } + /// + /// Return the SparkContext that created this RDD + /// + public SparkContext SparkContext + { + get + { + return sparkContext; + } + } + /// /// Return whether this RDD has been cached or not /// diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs index bc8faac4..f6d8a1a5 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs @@ -571,6 +571,17 @@ public void SetLogLevel(string logLevel) SparkContextProxy.SetLogLevel(logLevel); } + /// + /// Run a job on a given set of partitions of an RDD. + /// + /// + /// + /// + public void RunJob(RDD rdd, IEnumerable partitions) + { + SparkContextProxy.RunJob(rdd.RddProxy, partitions); + } + /// /// Cancel active jobs for the specified group. See for more information. /// diff --git a/csharp/AdapterTest/SparkContextTest.cs b/csharp/AdapterTest/SparkContextTest.cs index d0005683..c7e2803a 100644 --- a/csharp/AdapterTest/SparkContextTest.cs +++ b/csharp/AdapterTest/SparkContextTest.cs @@ -152,6 +152,23 @@ public void TestSparkContextHadoopConfigurationProperty() Assert.IsNotNull(hadoopConf); } + [Test] + public void TestRunJob() + { + // Arrange + Mock sparkContextProxy = new Mock(); + SparkContext sc = new SparkContext(sparkContextProxy.Object, null); + RDD rdd = sc.Parallelize(new int[] {0, 1, 2, 3, 4, 5}, 2); + sparkContextProxy.Setup(m => m.RunJob(It.IsAny(), It.IsAny>())); + + // Act + int[] partitions = new int[] { 0, 1 }; + rdd.SparkContext.RunJob(rdd, partitions); + + // Assert + sparkContextProxy.Verify(m => m.RunJob(rdd.RddProxy, partitions), Times.Once); + } + [Test] public void TestCancelAllJobs() { From 6e2b820524c8184b19d2650094480c7c3ae0229c Mon Sep 17 00:00:00 2001 From: Jorgen Thelin Date: Tue, 1 Nov 2016 22:41:17 -0700 Subject: [PATCH 07/20] Fix Spark version build problems * Check that Download-File function got a real file. - Check in the `Download-File` function that the current downloaded copy of the file is not empty [zero length]. * archive.apache.org - Use `archive.apache.org` as the default server location for downloading Spark [and Maven] binary distributions. - Previous location was `www.us.apache.org` but that only supports the most recent distributions for each major release [ie currently `2.0.1` but not `2.0.0` which Mobius is using at the moment.] * The Apache distribution server is used during both build and run phases. --- build/localmode/RunSamples.cmd | 3 ++- build/localmode/downloadtools.ps1 | 28 ++++++++++++++++++++++------ build/localmode/run-samples.sh | 5 +++-- 3 files changed, 27 insertions(+), 9 deletions(-) diff --git a/build/localmode/RunSamples.cmd b/build/localmode/RunSamples.cmd index 3b75e857..d99b39bd 100644 --- a/build/localmode/RunSamples.cmd +++ b/build/localmode/RunSamples.cmd @@ -49,7 +49,8 @@ if "%precheck%" == "bad" (goto :EOF) @rem set SPARK_VERSION=2.0.0 set HADOOP_VERSION=2.6 -@echo [RunSamples.cmd] SPARK_VERSION=%SPARK_VERSION%, HADOOP_VERSION=%HADOOP_VERSION% +set APACHE_DIST_SERVER=archive.apache.org +@echo [RunSamples.cmd] SPARK_VERSION=%SPARK_VERSION%, HADOOP_VERSION=%HADOOP_VERSION%, APACHE_DIST_SERVER=%APACHE_DIST_SERVER% @rem download runtime dependencies pushd "%CMDHOME%" diff --git a/build/localmode/downloadtools.ps1 b/build/localmode/downloadtools.ps1 index 78d20a91..e3609a56 100644 --- a/build/localmode/downloadtools.ps1 +++ b/build/localmode/downloadtools.ps1 @@ -10,6 +10,9 @@ # Param([string] $stage, [string] $verbose) +$envValue = [Environment]::GetEnvironmentVariable("APACHE_DIST_SERVER") +$apacheDistServer = if ($envValue -eq $null) { "archive.apache.org" } else { $envValue } + if ($stage.ToLower() -eq "run") { # retrieve hadoop and spark versions from environment variables @@ -18,8 +21,8 @@ if ($stage.ToLower() -eq "run") $envValue = [Environment]::GetEnvironmentVariable("SPARK_VERSION") $sparkVersion = if ($envValue -eq $null) { "1.6.1" } else { $envValue } - - Write-Output "[downloadtools] hadoopVersion=$hadoopVersion, sparkVersion=$sparkVersion" + + Write-Output "[downloadtools] hadoopVersion=$hadoopVersion, sparkVersion=$sparkVersion, apacheDistServer=$apacheDistServer" } function Get-ScriptDirectory @@ -73,8 +76,16 @@ function Download-File($url, $output) $output = [System.IO.Path]::GetFullPath($output) if (test-path $output) { - Write-Output "[downloadtools.Download-File] $output exists. No need to download." - return + if ((Get-Item $output).Length -gt 0) + { + Write-Output "[downloadtools.Download-File] $output exists. No need to download." + return + } + else + { + Write-Output "[downloadtools.Download-File] [WARNING] $output exists but is empty. We need to download a new copy of the file." + Remove-Item $output + } } $start_time = Get-Date @@ -122,6 +133,11 @@ function Download-File($url, $output) } Write-Output "[downloadtools.Download-File] Download completed. Time taken: $howlong" + + if ( !(test-path $output) -or (Get-Item $output).Length -eq 0) + { + throw [System.IO.FileNotFoundException] "Failed to download file $output from $url" + } } function Unzip-File($zipFile, $targetDir) @@ -252,7 +268,7 @@ function Download-BuildTools $mvnCmd = "$toolsDir\$mvnVer\bin\mvn.cmd" if (!(test-path $mvnCmd)) { - $url = "http://www.us.apache.org/dist/maven/maven-3/3.3.9/binaries/$mvnVer-bin.tar.gz" + $url = "http://$apacheDistServer/dist/maven/maven-3/3.3.9/binaries/$mvnVer-bin.tar.gz" $output="$toolsDir\$mvnVer-bin.tar.gz" Download-File $url $output Untar-File $output $toolsDir @@ -402,7 +418,7 @@ function Download-RuntimeDependencies $sparkSubmit="$S_HOME\bin\spark-submit.cmd" if (!(test-path $sparkSubmit)) { - $url = "http://www.us.apache.org/dist/spark/spark-$sparkVersion/spark-$sparkVersion-bin-hadoop$hadoopVersion.tgz" + $url = "http://$apacheDistServer/dist/spark/spark-$sparkVersion/spark-$sparkVersion-bin-hadoop$hadoopVersion.tgz" $output = "$toolsDir\spark-$sparkVersion-bin-hadoop$hadoopVersion.tgz" Download-File $url $output Untar-File $output $toolsDir diff --git a/build/localmode/run-samples.sh b/build/localmode/run-samples.sh index 4e94bca2..73d15cf3 100755 --- a/build/localmode/run-samples.sh +++ b/build/localmode/run-samples.sh @@ -18,7 +18,8 @@ done # setup Hadoop and Spark versions export SPARK_VERSION=2.0.0 export HADOOP_VERSION=2.6 -echo "[run-samples.sh] SPARK_VERSION=$SPARK_VERSION, HADOOP_VERSION=$HADOOP_VERSION" +export APACHE_DIST_SERVER=archive.apache.org +echo "[run-samples.sh] SPARK_VERSION=$SPARK_VERSION, HADOOP_VERSION=$HADOOP_VERSION, APACHE_DIST_SERVER=$APACHE_DIST_SERVER" export FWDIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" @@ -30,7 +31,7 @@ export SPARK=spark-$SPARK_VERSION-bin-hadoop$HADOOP_VERSION export SPARK_HOME="$TOOLS_DIR/$SPARK" if [ ! -d "$SPARK_HOME" ]; then - wget "http://www.us.apache.org/dist/spark/spark-$SPARK_VERSION/$SPARK.tgz" -O "$TOOLS_DIR/$SPARK.tgz" + wget "http://$APACHE_DIST_SERVER/dist/spark/spark-$SPARK_VERSION/$SPARK.tgz" -O "$TOOLS_DIR/$SPARK.tgz" tar xfz "$TOOLS_DIR/$SPARK.tgz" -C "$TOOLS_DIR" fi export PATH="$SPARK_HOME/bin:$PATH" From 12c46c8ed129783e66f8e2c21ee50c76d1596052 Mon Sep 17 00:00:00 2001 From: Kaarthik Sivashanmugam Date: Sat, 3 Dec 2016 22:28:59 -0800 Subject: [PATCH 08/20] fixing Travis build issue * using openjdk7 --- .travis.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.travis.yml b/.travis.yml index 986efb1a..2e56c979 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,6 +2,8 @@ language: csharp solution: csharp/SparkCLR.sln sudo: required dist: trusty +env: + - JDK=openjdk7 before_install: - sudo apt-get install xsltproc - nuget install NUnit.Runners -Version 3.0.0 -OutputDirectory testrunner @@ -12,6 +14,8 @@ before_install: - export M2="$M2_HOME/bin" - export PATH="$M2:$PATH" - hash -r +before_script: + - jdk_switcher use $JDK script: - export MAVEN_OPTS="-XX:MaxPermSize=2g -Xmx4g" - export JAVA_OPTS="-XX:MaxPermSize=2g -Xmx4g" From 88f76af038b303ebd871c006b3234549f981c7f2 Mon Sep 17 00:00:00 2001 From: Attila Szucs Date: Sun, 4 Dec 2016 07:49:00 +0100 Subject: [PATCH 09/20] Fix ReduceByKey failure when the value type is a reference type --- .../Core/PairRDDFunctions.cs | 6 +++- csharp/AdapterTest/PairRDDTest.cs | 33 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs index 5fe3c679..133b2f9d 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/PairRDDFunctions.cs @@ -97,7 +97,11 @@ public static RDD Values(this RDD> self) /// public static RDD> ReduceByKey(this RDD> self, Func reduceFunc, int numPartitions = 0) { - return CombineByKey(self, () => default(V), reduceFunc, reduceFunc, numPartitions); + var locallyCombined = self.MapPartitionsWithIndex(new GroupByMergeHelper(reduceFunc).Execute, true); + + var shuffled = locallyCombined.PartitionBy(numPartitions); + + return shuffled.MapPartitionsWithIndex(new GroupByMergeHelper(reduceFunc).Execute, true); } /// diff --git a/csharp/AdapterTest/PairRDDTest.cs b/csharp/AdapterTest/PairRDDTest.cs index f8ba3847..15054af5 100644 --- a/csharp/AdapterTest/PairRDDTest.cs +++ b/csharp/AdapterTest/PairRDDTest.cs @@ -105,6 +105,39 @@ public void TestPairRddReduceByKeyLocally() } } + [Serializable] + private class IntWrapper + { + public IntWrapper(int value) + { + Value = value; + } + + public int Value { get; } + } + + [Test] + public void TestPairRddReduceByKeyWithObjects() + { + // The ReduceByKey method below fails with NPE if ReduceByKey + // calls CombineByKey with () => default(V) as seed generator + var sums = pairs + .MapValues(value => new IntWrapper(value)) + .ReduceByKey((x, y) => new IntWrapper(x.Value + y.Value)); + + var result = sums + .CollectAsMap() + .Select(pair => new KeyValuePair(pair.Key, pair.Value.Value)) + .ToList(); + + var expectedResult = pairs + .ReduceByKey((x, y) => x + y) + .CollectAsMap() + .ToList(); + + Assert.That(result, Is.EquivalentTo(expectedResult)); + } + [Test] public void TestPairRddFoldByKey() { From 66fc5123d01d25a54f02cbf867f5e8313c9c350f Mon Sep 17 00:00:00 2001 From: dwnichols Date: Tue, 13 Dec 2016 16:08:02 -0500 Subject: [PATCH 10/20] Fix test TestDStreamUpdateStateByKey to use Tuples --- csharp/AdapterTest/DStreamTest.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/csharp/AdapterTest/DStreamTest.cs b/csharp/AdapterTest/DStreamTest.cs index 90281eb3..c6a363bf 100644 --- a/csharp/AdapterTest/DStreamTest.cs +++ b/csharp/AdapterTest/DStreamTest.cs @@ -281,8 +281,8 @@ public void TestDStreamUpdateStateByKey() foreach (object record in taken) { - KeyValuePair countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Key == "The" || countByWord.Key == "dog" || countByWord.Key == "lazy" ? 23 : 22, countByWord.Value); + Tuple countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item1 == "The" || countByWord.Item1 == "dog" || countByWord.Item1 == "lazy" ? 23 : 22, countByWord.Item2); } }); From a36b16009ed0c1f66d5bf90f95fe094dac17cfb6 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Tue, 13 Dec 2016 16:24:59 -0500 Subject: [PATCH 11/20] Further work on Tuple change --- .../Core/OrderedRDDFunctions.cs | 12 ++++++------ .../Microsoft.Spark.CSharp/Core/SparkContext.cs | 12 ++++++------ csharp/AdapterTest/SparkContextTest.cs | 4 ++-- .../Samples/Microsoft.Spark.CSharp/DStreamSamples.cs | 10 +++++----- csharp/Worker/Microsoft.Spark.CSharp/Worker.cs | 4 ++-- csharp/WorkerTest/WorkerTest.cs | 10 +++++----- 6 files changed, 26 insertions(+), 26 deletions(-) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs index 23509e56..e5deb09a 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/OrderedRDDFunctions.cs @@ -18,7 +18,7 @@ public static class OrderedRDDFunctions { /// - /// Sorts this RDD, which is assumed to consist of KeyValuePair pairs. + /// Sorts this RDD, which is assumed to consist of Tuple pairs. /// /// /// @@ -32,7 +32,7 @@ public static RDD> SortByKey(this RDD> self, return SortByKey(self, ascending, numPartitions, new DefaultSortKeyFuncHelper().Execute); } /// - /// Sorts this RDD, which is assumed to consist of KeyValuePairs. If key is type of string, case is sensitive. + /// Sorts this RDD, which is assumed to consist of Tuples. If Item1 is type of string, case is sensitive. /// /// /// @@ -40,7 +40,7 @@ public static RDD> SortByKey(this RDD> self, /// /// /// Number of partitions. Each partition of the sorted RDD contains a sorted range of the elements. - /// RDD will sort by keyFunc(key) for every key in KeyValuePair. Must not be null. + /// RDD will sort by keyFunc(key) for every Item1 in Tuple. Must not be null. /// public static RDD> SortByKey(this RDD> self, bool ascending, int? numPartitions, Func keyFunc) @@ -103,13 +103,13 @@ public static RDD> SortByKey(this RDD> self, /// /// /// - public static RDD> repartitionAndSortWithinPartitions( - this RDD> self, + public static RDD> repartitionAndSortWithinPartitions( + this RDD> self, int? numPartitions = null, Func partitionFunc = null, bool ascending = true) { - return self.MapPartitionsWithIndex>((pid, iter) => ascending ? iter.OrderBy(kv => kv.Key) : iter.OrderByDescending(kv => kv.Key)); + return self.MapPartitionsWithIndex>((pid, iter) => ascending ? iter.OrderBy(kv => kv.Item1) : iter.OrderByDescending(kv => kv.Item1)); } [Serializable] diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs index 20861c2b..9add6da9 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/SparkContext.cs @@ -241,7 +241,7 @@ public RDD EmptyRDD() /// /// Do /// {{{ - /// RDD<KeyValuePair<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") + /// RDD<Tuple<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") /// }}} /// /// then `rdd` contains @@ -259,9 +259,9 @@ public RDD EmptyRDD() /// /// /// - public RDD> WholeTextFiles(string filePath, int? minPartitions = null) + public RDD> WholeTextFiles(string filePath, int? minPartitions = null) { - return new RDD>(SparkContextProxy.WholeTextFiles(filePath, minPartitions ?? DefaultMinPartitions), this, SerializedMode.Pair); + return new RDD>(SparkContextProxy.WholeTextFiles(filePath, minPartitions ?? DefaultMinPartitions), this, SerializedMode.Pair); } /// @@ -279,7 +279,7 @@ public RDD> WholeTextFiles(string filePath, int? mi /// }}} /// /// Do - /// RDD<KeyValuePair<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + /// RDD<Tuple<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, /// /// then `rdd` contains /// {{{ @@ -296,9 +296,9 @@ public RDD> WholeTextFiles(string filePath, int? mi /// /// /// - public RDD> BinaryFiles(string filePath, int? minPartitions) + public RDD> BinaryFiles(string filePath, int? minPartitions) { - return new RDD>(SparkContextProxy.BinaryFiles(filePath, minPartitions ?? DefaultMinPartitions), this, SerializedMode.Pair); + return new RDD>(SparkContextProxy.BinaryFiles(filePath, minPartitions ?? DefaultMinPartitions), this, SerializedMode.Pair); } /// diff --git a/csharp/AdapterTest/SparkContextTest.cs b/csharp/AdapterTest/SparkContextTest.cs index 223f99c3..cd5b03f0 100644 --- a/csharp/AdapterTest/SparkContextTest.cs +++ b/csharp/AdapterTest/SparkContextTest.cs @@ -372,7 +372,7 @@ public void TestWholeTextFiles() SparkContext sc = new SparkContext(sparkContextProxy.Object, null); // Act - RDD> rdd = sc.WholeTextFiles(filePath, null); + RDD> rdd = sc.WholeTextFiles(filePath, null); // Assert Assert.IsNotNull(rdd); @@ -394,7 +394,7 @@ public void TestBinaryFiles() SparkContext sc = new SparkContext(sparkContextProxy.Object, null); // Act - RDD> rdd = sc.BinaryFiles(filePath, null); + RDD> rdd = sc.BinaryFiles(filePath, null); // Assert Assert.IsNotNull(rdd); diff --git a/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs index c20c1855..f4978e0c 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/DStreamSamples.cs @@ -99,8 +99,8 @@ internal static void DStreamTextFileSample() { Console.WriteLine(record); - var countByWord = (KeyValuePair)record; - Assert.AreEqual(countByWord.Value, countByWord.Key == "The" || countByWord.Key == "lazy" || countByWord.Key == "dog" ? 92 : 88); + var countByWord = (Tuple)record; + Assert.AreEqual(countByWord.Item2, countByWord.Item1 == "The" || countByWord.Item1 == "lazy" || countByWord.Item1 == "dog" ? 92 : 88); } Console.WriteLine(); @@ -283,10 +283,10 @@ private static void DStreamReduceByKeyAndWindowSample() foreach (object record in taken) { - KeyValuePair sum = (KeyValuePair)record; - Console.WriteLine("Key: {0}, Value: {1}", sum.Key, sum.Value); + Tuple sum = (Tuple)record; + Console.WriteLine("Key: {0}, Value: {1}", sum.Item1, sum.Item2); // when batch count reaches window size, sum of even/odd number stay at windowDuration / slideDuration * (2450, 2500) respectively - Assert.AreEqual(sum.Value, (count > windowDuration / slideDuration ? windowDuration : count * slideDuration) / (bacthIntervalMs / 1000) * (sum.Key == 0 ? 2450 : 2500)); + Assert.AreEqual(sum.Item2, (count > windowDuration / slideDuration ? windowDuration : count * slideDuration) / (bacthIntervalMs / 1000) * (sum.Item1 == 0 ? 2450 : 2500)); } }); diff --git a/csharp/Worker/Microsoft.Spark.CSharp/Worker.cs b/csharp/Worker/Microsoft.Spark.CSharp/Worker.cs index 9c95d112..486a1bc7 100644 --- a/csharp/Worker/Microsoft.Spark.CSharp/Worker.cs +++ b/csharp/Worker/Microsoft.Spark.CSharp/Worker.cs @@ -533,7 +533,7 @@ private static void WriteAccumulatorValues(Stream networkStream, IFormatter form .GetField("value", BindingFlags.NonPublic | BindingFlags.Instance) .GetValue(item.Value); logger.LogDebug("({0}, {1})", item.Key, value); - formatter.Serialize(ms, new KeyValuePair(item.Key, value)); + formatter.Serialize(ms, new Tuple(item.Key, value)); byte[] buffer = ms.ToArray(); SerDe.Write(networkStream, buffer.Length); SerDe.Write(networkStream, buffer); @@ -649,7 +649,7 @@ private static IEnumerable GetIterator(Stream inputStream, string seria } watch.Stop(); - yield return new KeyValuePair(pairKey, pairValue); + yield return new Tuple(pairKey, pairValue); break; } diff --git a/csharp/WorkerTest/WorkerTest.cs b/csharp/WorkerTest/WorkerTest.cs index 19df98d5..18264375 100644 --- a/csharp/WorkerTest/WorkerTest.cs +++ b/csharp/WorkerTest/WorkerTest.cs @@ -573,7 +573,7 @@ public void TestWorkerWithPairDeserializedModeAndNoneSerializedMode() { WritePayloadHeaderToWorker(s); byte[] command = SparkContext.BuildCommand( - new CSharpWorkerFunc((pid, iter) => iter.Cast>().Select(pair => pair.Key)), + new CSharpWorkerFunc((pid, iter) => iter.Cast>().Select(pair => pair.Item1)), SerializedMode.Pair, SerializedMode.None); SerDe.Write(s, command.Length); @@ -713,7 +713,7 @@ private IEnumerable ReadDataSection(Stream s, int expectedCount = 0) /// /// read accumulator /// - private IEnumerable> ReadAccumulator(Stream s, int expectedCount = 0) + private IEnumerable> ReadAccumulator(Stream s, int expectedCount = 0) { int count = 0; var formatter = new BinaryFormatter(); @@ -723,7 +723,7 @@ private IEnumerable> ReadAccumulator(Stream s, int ex if (length > 0) { var ms = new MemoryStream(SerDe.ReadBytes(s, length)); - yield return (KeyValuePair)formatter.Deserialize(ms); + yield return (Tuple)formatter.Deserialize(ms); if (expectedCount > 0 && ++count >= expectedCount) { @@ -780,8 +780,8 @@ public void TestAccumulatorInWorker() int accumulatorsCount = SerDe.ReadInt(s); Assert.IsTrue(accumulatorsCount == 1); var accumulatorFromWorker = ReadAccumulator(s, accumulatorsCount).First(); - Assert.AreEqual(accumulatorId, accumulatorFromWorker.Key); - Assert.AreEqual(expectedCount, accumulatorFromWorker.Value); + Assert.AreEqual(accumulatorId, accumulatorFromWorker.Item1); + Assert.AreEqual(expectedCount, accumulatorFromWorker.Item2); SerDe.ReadInt(s); } From 2fe5ec409ec00ba812f1947c5648a82527c3b516 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Tue, 13 Dec 2016 16:25:23 -0500 Subject: [PATCH 12/20] Update documatation for tuples change --- README.md | 4 ++-- .../Microsoft.Spark.CSharp.Adapter.Doc.XML | 12 ++++++------ .../documentation/Mobius_API_Documentation.md | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index ff598731..f318e52c 100644 --- a/README.md +++ b/README.md @@ -8,7 +8,7 @@ For example, the word count sample in Apache Spark can be implemented in C# as f ```c# var lines = sparkContext.TextFile(@"hdfs://path/to/input.txt"); var words = lines.FlatMap(s => s.Split(' ')); -var wordCounts = words.Map(w => new KeyValuePair(w.Trim(), 1)) +var wordCounts = words.Map(w => new Tuple(w.Trim(), 1)) .ReduceByKey((x, y) => x + y); var wordCountCollection = wordCounts.Collect(); wordCounts.SaveAsTextFile(@"hdfs://path/to/wordcount.txt"); @@ -63,7 +63,7 @@ StreamingContext sparkStreamingContext = StreamingContext.GetOrCreate(checkpoint .Map(kvp => Encoding.UTF8.GetString(kvp.Value)) .Filter(line => line.Contains(",")) .Map(line => line.Split(',')) - .Map(columns => new KeyValuePair( + .Map(columns => new Tuple( string.Format("{0},{1}", columns[0], columns[1]), 1)) .ReduceByKeyAndWindow((x, y) => x + y, (x, y) => x - y, windowDurationInSecs, slideDurationInSecs, 3) diff --git a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML index 2923020b..0f4c49ce 100644 --- a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML +++ b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML @@ -453,7 +453,7 @@ - Sorts this RDD, which is assumed to consist of KeyValuePair pairs. + Sorts this RDD, which is assumed to consist of Tuple pairs. @@ -464,7 +464,7 @@ - Sorts this RDD, which is assumed to consist of KeyValuePairs. If key is type of string, case is sensitive. + Sorts this RDD, which is assumed to consist of Tuples. If Item1 is type of string, case is sensitive. @@ -472,10 +472,10 @@ Number of partitions. Each partition of the sorted RDD contains a sorted range of the elements. - RDD will sort by keyFunc(key) for every key in KeyValuePair. Must not be null. + RDD will sort by keyFunc(key) for every Item1 in Tuple. Must not be null. - + Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys. @@ -2137,7 +2137,7 @@ Do {{{ - RDD<KeyValuePair<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") + RDD<Tuple<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") }}} then `rdd` contains @@ -2172,7 +2172,7 @@ }}} Do - RDD<KeyValuePair<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + RDD<Tuple<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, then `rdd` contains {{{ diff --git a/csharp/Adapter/documentation/Mobius_API_Documentation.md b/csharp/Adapter/documentation/Mobius_API_Documentation.md index 92d5d764..c6d235b1 100644 --- a/csharp/Adapter/documentation/Mobius_API_Documentation.md +++ b/csharp/Adapter/documentation/Mobius_API_Documentation.md @@ -187,7 +187,7 @@ ####Methods -
NameDescription
SortByKey``2Sorts this RDD, which is assumed to consist of KeyValuePair pairs.
SortByKey``3Sorts this RDD, which is assumed to consist of KeyValuePairs. If key is type of string, case is sensitive.
repartitionAndSortWithinPartitions``2Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys. This is more efficient than calling `repartition` and then sorting within each partition because it can push the sorting down into the shuffle machinery.
+
NameDescription
SortByKey``2Sorts this RDD, which is assumed to consist of Tuple pairs.
SortByKey``3Sorts this RDD, which is assumed to consist of Tuples. If Item1 is type of string, case is sensitive.
repartitionAndSortWithinPartitions``2Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys. This is more efficient than calling `repartition` and then sorting within each partition because it can push the sorting down into the shuffle machinery.
--- @@ -327,7 +327,7 @@ ####Methods -
NameDescription
GetActiveSparkContextGet existing SparkContext
GetConfReturn a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be changed at runtime.
GetOrCreateThis function may be used to get or instantiate a SparkContext and register it as a singleton object. Because we can only have one active SparkContext per JVM, this is useful when applications may wish to share a SparkContext. Note: This function cannot be used to create multiple SparkContext instances even if multiple contexts are allowed.
TextFileRead a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings.
Parallelize``1Distribute a local collection to form an RDD. sc.Parallelize(new int[] {0, 2, 3, 4, 6}, 5).Glom().Collect() [[0], [2], [3], [4], [6]]
EmptyRDDCreate an RDD that has no partitions or elements.
WholeTextFilesRead a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. For example, if you have the following files: {{{ hdfs://a-hdfs-path/part-00000 hdfs://a-hdfs-path/part-00001 ... hdfs://a-hdfs-path/part-nnnnn }}} Do {{{ RDD<KeyValuePair<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") }}} then `rdd` contains {{{ (a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content) }}} Small files are preferred, large file is also allowable, but may cause bad performance. minPartitions A suggestion value of the minimal splitting number for input data.
BinaryFilesRead a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI as a byte array. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. For example, if you have the following files: {{{ hdfs://a-hdfs-path/part-00000 hdfs://a-hdfs-path/part-00001 ... hdfs://a-hdfs-path/part-nnnnn }}} Do RDD<KeyValuePair<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, then `rdd` contains {{{ (a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content) }}} @note Small files are preferred; very large files but may cause bad performance. @param minPartitions A suggestion value of the minimal splitting number for input data.
SequenceFileRead a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is as follows: 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes 2. Serialization is attempted via Pyrolite pickling 3. If this fails, the fallback is to call 'toString' on each key and value 4. PickleSerializer is used to deserialize pickled objects on the Python side
NewAPIHadoopFileRead a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for sc.sequenceFile. A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java
NewAPIHadoopRDDRead a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile.
HadoopFileRead an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for sc.sequenceFile. A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java.
HadoopRDDRead an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile.
Union``1Build the union of a list of RDDs. This supports unions() of RDDs with different serialized formats, although this forces them to be reserialized using the default serializer: >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] >>> parallelized = sc.parallelize(["World!"]) >>> sorted(sc.union([textFile, parallelized]).collect()) [u'Hello', 'World!']
Broadcast``1Broadcast a read-only variable to the cluster, returning a Broadcast object for reading it in distributed functions. The variable will be sent to each cluster only once.
Accumulator``1Create an with the given initial value, using a given helper object to define how to add values of the data type if provided. Default AccumulatorParams are used for integers and floating-point numbers if you do not provide one. For other types, a custom AccumulatorParam can be used.
StopShut down the SparkContext.
AddFileAdd a file to be downloaded with this Spark job on every node. The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, use `SparkFiles.get(fileName)` to find its download location.
SetCheckpointDirSet the directory under which RDDs are going to be checkpointed. The directory must be a HDFS path if running on a cluster.
SetJobGroupAssigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared. Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group. The application can also use [[org.apache.spark.api.java.JavaSparkContext.cancelJobGroup]] to cancel all running jobs in this group. For example, {{{ // In the main thread: sc.setJobGroup("some_job_to_cancel", "some job description"); rdd.map(...).count(); // In a separate thread: sc.cancelJobGroup("some_job_to_cancel"); }}} If interruptOnCancel is set to true for the job group, then job cancellation will result in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead.
SetLocalPropertySet a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool.
GetLocalPropertyGet a local property set in this thread, or null if it is missing. See [[org.apache.spark.api.java.JavaSparkContext.setLocalProperty]].
SetLogLevelControl our logLevel. This overrides any user-defined log settings. @param logLevel The desired log level as a string. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN
RunJob``1Run a job on a given set of partitions of an RDD.
CancelJobGroupCancel active jobs for the specified group. See for more information.
CancelAllJobsCancel all jobs that have been scheduled or are running.
+
NameDescription
GetActiveSparkContextGet existing SparkContext
GetConfReturn a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be changed at runtime.
GetOrCreateThis function may be used to get or instantiate a SparkContext and register it as a singleton object. Because we can only have one active SparkContext per JVM, this is useful when applications may wish to share a SparkContext. Note: This function cannot be used to create multiple SparkContext instances even if multiple contexts are allowed.
TextFileRead a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings.
Parallelize``1Distribute a local collection to form an RDD. sc.Parallelize(new int[] {0, 2, 3, 4, 6}, 5).Glom().Collect() [[0], [2], [3], [4], [6]]
EmptyRDDCreate an RDD that has no partitions or elements.
WholeTextFilesRead a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. For example, if you have the following files: {{{ hdfs://a-hdfs-path/part-00000 hdfs://a-hdfs-path/part-00001 ... hdfs://a-hdfs-path/part-nnnnn }}} Do {{{ RDD<Tuple<string, string>> rdd = sparkContext.WholeTextFiles("hdfs://a-hdfs-path") }}} then `rdd` contains {{{ (a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content) }}} Small files are preferred, large file is also allowable, but may cause bad performance. minPartitions A suggestion value of the minimal splitting number for input data.
BinaryFilesRead a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI as a byte array. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. For example, if you have the following files: {{{ hdfs://a-hdfs-path/part-00000 hdfs://a-hdfs-path/part-00001 ... hdfs://a-hdfs-path/part-nnnnn }}} Do RDD<Tuple<string, byte[]>>"/> rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, then `rdd` contains {{{ (a-hdfs-path/part-00000, its content) (a-hdfs-path/part-00001, its content) ... (a-hdfs-path/part-nnnnn, its content) }}} @note Small files are preferred; very large files but may cause bad performance. @param minPartitions A suggestion value of the minimal splitting number for input data.
SequenceFileRead a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is as follows: 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes 2. Serialization is attempted via Pyrolite pickling 3. If this fails, the fallback is to call 'toString' on each key and value 4. PickleSerializer is used to deserialize pickled objects on the Python side
NewAPIHadoopFileRead a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for sc.sequenceFile. A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java
NewAPIHadoopRDDRead a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile.
HadoopFileRead an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is the same as for sc.sequenceFile. A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java.
HadoopRDDRead an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile.
Union``1Build the union of a list of RDDs. This supports unions() of RDDs with different serialized formats, although this forces them to be reserialized using the default serializer: >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] >>> parallelized = sc.parallelize(["World!"]) >>> sorted(sc.union([textFile, parallelized]).collect()) [u'Hello', 'World!']
Broadcast``1Broadcast a read-only variable to the cluster, returning a Broadcast object for reading it in distributed functions. The variable will be sent to each cluster only once.
Accumulator``1Create an with the given initial value, using a given helper object to define how to add values of the data type if provided. Default AccumulatorParams are used for integers and floating-point numbers if you do not provide one. For other types, a custom AccumulatorParam can be used.
StopShut down the SparkContext.
AddFileAdd a file to be downloaded with this Spark job on every node. The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, use `SparkFiles.get(fileName)` to find its download location.
SetCheckpointDirSet the directory under which RDDs are going to be checkpointed. The directory must be a HDFS path if running on a cluster.
SetJobGroupAssigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared. Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group. The application can also use [[org.apache.spark.api.java.JavaSparkContext.cancelJobGroup]] to cancel all running jobs in this group. For example, {{{ // In the main thread: sc.setJobGroup("some_job_to_cancel", "some job description"); rdd.map(...).count(); // In a separate thread: sc.cancelJobGroup("some_job_to_cancel"); }}} If interruptOnCancel is set to true for the job group, then job cancellation will result in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead.
SetLocalPropertySet a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool.
GetLocalPropertyGet a local property set in this thread, or null if it is missing. See [[org.apache.spark.api.java.JavaSparkContext.setLocalProperty]].
SetLogLevelControl our logLevel. This overrides any user-defined log settings. @param logLevel The desired log level as a string. Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN
RunJob``1Run a job on a given set of partitions of an RDD.
CancelJobGroupCancel active jobs for the specified group. See for more information.
CancelAllJobsCancel all jobs that have been scheduled or are running.
--- From 3e3a209bf45555636c6906c6fb1f6eb0bbaf5ec5 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Tue, 13 Dec 2016 22:44:07 -0500 Subject: [PATCH 13/20] Test fixes for Samples in PairRDDSamples and RDDSamples --- csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs | 2 +- csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs index 22de846f..9a56de93 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/PairRDDSamples.cs @@ -112,7 +112,7 @@ internal static void PairRDDCountByKeySample() new Tuple("a", 1) }, 2) .CountByKey() - .ToLookup(x => x.Item1); + .ToDictionary(k => k.Item1, v => v.Item2); foreach (var kv in countByKey) Console.WriteLine(kv); diff --git a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs index 62477c3f..1112a823 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/RDDSamples.cs @@ -3,6 +3,7 @@ using System; using System.Collections.Generic; +using System.Diagnostics; using System.IO; using System.Linq; using Microsoft.Spark.CSharp.Core; @@ -221,7 +222,10 @@ internal static void RDDTreeAggregateSample() [Sample] internal static void RDDCountByValueSample() { - var countByValue = SparkCLRSamples.SparkContext.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2).CountByValue().ToArray(); + var countByValue = SparkCLRSamples.SparkContext.Parallelize(new int[] { 1, 2, 1, 2, 2 }, 2) + .CountByValue() + .ToDictionary(k => k.Item1, v => v.Item2); + foreach (var item in countByValue) Console.WriteLine(item); From 50d65dead523c16798a69b975bc931c93a30b213 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Wed, 14 Dec 2016 00:22:00 -0500 Subject: [PATCH 14/20] Added a few tests against Adapter Builder and RDDs --- csharp/AdapterTest/BuilderTest.cs | 18 ++++++++++++++++++ csharp/AdapterTest/RDDTest.cs | 5 +++++ 2 files changed, 23 insertions(+) diff --git a/csharp/AdapterTest/BuilderTest.cs b/csharp/AdapterTest/BuilderTest.cs index aae3c626..d77d73ef 100644 --- a/csharp/AdapterTest/BuilderTest.cs +++ b/csharp/AdapterTest/BuilderTest.cs @@ -1,4 +1,6 @@ using System; +using System.Linq; +using Microsoft.Spark.CSharp.Core; using Microsoft.Spark.CSharp.Sql; using NUnit.Framework; @@ -46,5 +48,21 @@ public void TestDoubleConfig() builder.Config("doublevalue", 3.5D); Assert.True(builder.options["doublevalue"].Equals("3.5", StringComparison.InvariantCultureIgnoreCase)); } + + [Test] + public void TestEnableHiveSupport() + { + var builder = new Builder(); + builder.EnableHiveSupport(); + Assert.True(builder.options["spark.sql.catalogImplementation"].Equals("hive", StringComparison.InvariantCultureIgnoreCase)); + } + + [Test] + public void TestGetOrCreate() + { + var builder = new Builder(); + builder.GetOrCreate(); + Assert.IsEmpty(builder.options); + } } } diff --git a/csharp/AdapterTest/RDDTest.cs b/csharp/AdapterTest/RDDTest.cs index ab406b32..add22429 100644 --- a/csharp/AdapterTest/RDDTest.cs +++ b/csharp/AdapterTest/RDDTest.cs @@ -20,6 +20,7 @@ namespace AdapterTest public class RDDTest { private static RDD words; + private static RDD empty; [OneTimeSetUp] public static void Initialize() @@ -27,6 +28,7 @@ public static void Initialize() var sparkContext = new SparkContext(null); var lines = sparkContext.TextFile(Path.GetTempFileName()); words = lines.FlatMap(l => l.Split(' ')); + empty = sparkContext.EmptyRDD(); } [Test] @@ -82,6 +84,7 @@ public void TestRddTreeReduce() public void TestRddTreeAggregate() { Assert.AreEqual(201, words.Map(w => 1).TreeAggregate(0, (x, y) => x + y, (x, y) => x + y)); + Assert.Throws(() => empty.TreeAggregate(0, (x, y) => 1, (x, y) => x + y, 0)); } [Test] @@ -135,6 +138,7 @@ public void TestRddGroupBy() public void TestRddIsEmpty() { Assert.IsFalse(words.IsEmpty()); + Assert.IsTrue(empty.IsEmpty()); Assert.IsTrue(words.Filter(w => w == null).IsEmpty()); } @@ -166,6 +170,7 @@ public void TestRddTakeSample() Assert.AreEqual(20, words.TakeSample(true, 20, 1).Length); Assert.Throws(() => words.TakeSample(true, -1, 1)); Assert.AreEqual(0, words.TakeSample(true, 0, 1).Length); + Assert.AreEqual(20, words.TakeSample(false, 20, 1).Length); } [Test] From 8eb8cd0b5456248ab8796487dc04fb046247b2f0 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Wed, 14 Dec 2016 00:28:12 -0500 Subject: [PATCH 15/20] Remove test --- csharp/AdapterTest/BuilderTest.cs | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/csharp/AdapterTest/BuilderTest.cs b/csharp/AdapterTest/BuilderTest.cs index d77d73ef..f4afc6a9 100644 --- a/csharp/AdapterTest/BuilderTest.cs +++ b/csharp/AdapterTest/BuilderTest.cs @@ -55,14 +55,6 @@ public void TestEnableHiveSupport() var builder = new Builder(); builder.EnableHiveSupport(); Assert.True(builder.options["spark.sql.catalogImplementation"].Equals("hive", StringComparison.InvariantCultureIgnoreCase)); - } - - [Test] - public void TestGetOrCreate() - { - var builder = new Builder(); - builder.GetOrCreate(); - Assert.IsEmpty(builder.options); - } + } } } From ed73082895dfea6448d604ae5c15794dbc1e7ec3 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Wed, 14 Dec 2016 14:04:56 -0500 Subject: [PATCH 16/20] Test changes to improve test coverage --- .../Mocks/MockStreamingContextProxy.cs | 18 ++++++++++++++---- csharp/AdapterTest/StreamingContextTest.cs | 2 +- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs b/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs index f4a7d546..c2161916 100644 --- a/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs +++ b/csharp/AdapterTest/Mocks/MockStreamingContextProxy.cs @@ -89,13 +89,23 @@ public IDStreamProxy CreateCSharpTransformed2DStream(IDStreamProxy jdstream, IDS public IDStreamProxy CreateCSharpReducedWindowedDStream(IDStreamProxy jdstream, byte[] func, byte[] invFunc, int windowSeconds, int slideSeconds, string serializationMode) { - Func, RDD, RDD> f = (Func, RDD, RDD>)formatter.Deserialize(new MemoryStream(func)); - RDD rdd = f(DateTime.UtcNow.Ticks, + Func, RDD, RDD> f = (Func, RDD, RDD>) formatter.Deserialize(new MemoryStream(func)); + + var ticks = DateTime.UtcNow.Ticks; + RDD rdd = f(ticks, new RDD((jdstream as MockDStreamProxy).rddProxy ?? new MockRddProxy(null), new SparkContext("", "")), new RDD((jdstream as MockDStreamProxy).rddProxy ?? new MockRddProxy(null), new SparkContext("", ""))); - return new MockDStreamProxy(rdd.RddProxy); - } + if (invFunc == null) return new MockDStreamProxy(rdd.RddProxy); + + Func, RDD, RDD> invf = (Func, RDD, RDD>) formatter.Deserialize(new MemoryStream(invFunc)); + RDD invRdd = invf(ticks, + new RDD((jdstream as MockDStreamProxy).rddProxy ?? new MockRddProxy(null), new SparkContext("", "")), + new RDD((jdstream as MockDStreamProxy).rddProxy ?? new MockRddProxy(null), new SparkContext("", ""))); + var difference = rdd.Subtract(invRdd); + + return new MockDStreamProxy(difference.RddProxy); + } public IDStreamProxy CreateCSharpStateDStream(IDStreamProxy jdstream, byte[] func, string className, string serializationMode, string serializationMode2) { diff --git a/csharp/AdapterTest/StreamingContextTest.cs b/csharp/AdapterTest/StreamingContextTest.cs index 78ea6deb..20d9325f 100644 --- a/csharp/AdapterTest/StreamingContextTest.cs +++ b/csharp/AdapterTest/StreamingContextTest.cs @@ -32,7 +32,7 @@ public void TestStreamingContext() var socketStream = ssc.SocketTextStream(IPAddress.Loopback.ToString(), 12345); Assert.IsNotNull(socketStream.DStreamProxy); - var kafkaStream = KafkaUtils.CreateStream(ssc, IPAddress.Loopback + ":2181", "testGroupId", new [] { Tuple.Create("testTopic1", 1) }, new List>()); + var kafkaStream = KafkaUtils.CreateStream(ssc, IPAddress.Loopback + ":2181", "testGroupId", new [] { Tuple.Create("testTopic1", 1) }, null); Assert.IsNotNull(kafkaStream.DStreamProxy); var directKafkaStream = KafkaUtils.CreateDirectStream(ssc, new List { "testTopic2" }, new List>(), new List>()); From 46e09f6ab6ce80dd42fb02be20a8caf3976e0d91 Mon Sep 17 00:00:00 2001 From: dwnichols Date: Wed, 14 Dec 2016 14:49:41 -0500 Subject: [PATCH 17/20] Add PairRDD SortBy descending test --- csharp/AdapterTest/PairRDDTest.cs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/csharp/AdapterTest/PairRDDTest.cs b/csharp/AdapterTest/PairRDDTest.cs index e772d4bf..22df0279 100644 --- a/csharp/AdapterTest/PairRDDTest.cs +++ b/csharp/AdapterTest/PairRDDTest.cs @@ -211,6 +211,14 @@ public void TestPairRddSortByKey2() CollectionAssert.AreEqual(expectedSortedRdd, rddSortByKey); } + [Test] + public void TestPairRddSortByKey3() + { + var expectedSortedRdd = pairs.Collect().OrderByDescending(kv => kv.Item1, StringComparer.OrdinalIgnoreCase).ToArray(); + var rddSortByKey = pairs.SortByKey(false, 1, key => key.ToLowerInvariant()).Collect(); + CollectionAssert.AreEqual(expectedSortedRdd, rddSortByKey); + } + [Test] public void TestPairRddProxy() { From 313dbf308bcdd91d22843d91279448af8e7111d4 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 5 Jan 2017 21:23:40 +0100 Subject: [PATCH 18/20] Update linux-instructions.md Add a note about `wget` (Im on macOS and had to install it using homebrew) + formatting --- notes/linux-instructions.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/notes/linux-instructions.md b/notes/linux-instructions.md index 4a6e1972..df55cace 100644 --- a/notes/linux-instructions.md +++ b/notes/linux-instructions.md @@ -6,7 +6,8 @@ * Maven 3.0.5 or above. * Mono 4.2 stable or above. The download and installation instructions for Mono are available in [http://www.mono-project.com/download/#download-lin](http://www.mono-project.com/download/#download-lin) (see [Debian, Ubuntu and derivatives](http://www.mono-project.com/docs/getting-started/install/linux/#debian-ubuntu-and-derivatives) or [CentOS, Fedora, similar Linux distributions or OS X](http://www.mono-project.com/docs/getting-started/install/linux/#centos-7-fedora-19-and-later-and-derivatives)) * F# for Mono. The download and installation instructions for the F# Mono extension are available in [http://fsharp.org/use/linux/](http://fsharp.org/use/linux/) -* NuGet. +* NuGet +* wget * XSLTPROC The following environment variables should be set properly: @@ -15,9 +16,10 @@ The following environment variables should be set properly: ## Instructions -Instructions to build Mobius in Linux are same as [instructions for Windows](./windows-instructions.md#instructions). The only change required is to use the following script files instead of .cmd files: -* build.sh -* clean.sh +Instructions to build Mobius on Linux are same as [instructions for Windows](./windows-instructions.md#instructions). The only change required is to use the following script files instead of `.cmd` files: + +* `build.sh` +* `clean.sh` # Running Unit Tests in Linux From a1b74ad8a7085388d4e4e117ba377431771f3efd Mon Sep 17 00:00:00 2001 From: skaarthik Date: Sat, 28 Jan 2017 01:03:17 -0800 Subject: [PATCH 19/20] added UDF support to SparkSession --- .../Microsoft.Spark.CSharp/Adapter.csproj | 1 + .../Proxy/ISparkSessionProxy.cs | 7 +- .../Proxy/Ipc/SparkSessionIpcProxy.cs | 27 +- .../Sql/SparkSession.cs | 5 + .../Sql/UdfRegistration.cs | 254 ++++++++++++++++++ .../Microsoft.Spark.CSharp.Adapter.Doc.XML | 165 ++++++++++++ csharp/AdapterTest/AdapterTest.csproj | 1 + .../Mocks/MockSparkSessionProxy.cs | 2 +- csharp/AdapterTest/UdfRegistrationTest.cs | 57 ++++ .../SparkSessionSamples.cs | 26 ++ 10 files changed, 534 insertions(+), 11 deletions(-) create mode 100644 csharp/Adapter/Microsoft.Spark.CSharp/Sql/UdfRegistration.cs create mode 100644 csharp/AdapterTest/UdfRegistrationTest.cs diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj b/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj index 4daf4aa5..d887daf8 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj @@ -157,6 +157,7 @@ + diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkSessionProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkSessionProxy.cs index 56f869cd..11a58cd0 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkSessionProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkSessionProxy.cs @@ -10,12 +10,15 @@ namespace Microsoft.Spark.CSharp.Proxy { - internal interface IUdfRegistration { } + internal interface IUdfRegistrationProxy + { + void RegisterFunction(string name, byte[] command, string returnType); + } interface ISparkSessionProxy { ISqlContextProxy SqlContextProxy { get; } - IUdfRegistration Udf { get; } + IUdfRegistrationProxy Udf { get; } ICatalogProxy GetCatalog(); IDataFrameReaderProxy Read(); ISparkSessionProxy NewSession(); diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkSessionIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkSessionIpcProxy.cs index d134c086..febfd3b5 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkSessionIpcProxy.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkSessionIpcProxy.cs @@ -17,18 +17,13 @@ internal class SparkSessionIpcProxy : ISparkSessionProxy private readonly JvmObjectReference jvmSparkSessionReference; private readonly ISqlContextProxy sqlContextProxy; - private readonly IUdfRegistration udfRegistration; + private readonly IUdfRegistrationProxy udfRegistrationProxy; - public IUdfRegistration Udf + public IUdfRegistrationProxy Udf { get { - if (udfRegistration == null) - { - //TODO implementation needed - } - - return udfRegistration; + return udfRegistrationProxy; } } @@ -46,6 +41,7 @@ internal SparkSessionIpcProxy(JvmObjectReference jvmSparkSessionReference) { this.jvmSparkSessionReference = jvmSparkSessionReference; sqlContextProxy = new SqlContextIpcProxy(GetSqlContextReference()); + udfRegistrationProxy = new UdfRegistrationIpcProxy(sqlContextProxy); } private JvmObjectReference GetSqlContextReference() @@ -98,4 +94,19 @@ public void Stop() SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jvmSparkSessionReference, "stop"); } } + + [ExcludeFromCodeCoverage] //IPC calls to JVM validated using validation-enabled samples - unit test coverage not reqiured + internal class UdfRegistrationIpcProxy : IUdfRegistrationProxy + { + private readonly ISqlContextProxy sqlContextProxy; + internal UdfRegistrationIpcProxy(ISqlContextProxy sqlContextProxy) + { + this.sqlContextProxy = sqlContextProxy; + } + + public void RegisterFunction(string name, byte[] command, string returnType) + { + sqlContextProxy.RegisterFunction(name, command, returnType); + } + } } diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/SparkSession.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/SparkSession.cs index 3ff8a8ab..55b58462 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/SparkSession.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/SparkSession.cs @@ -48,6 +48,11 @@ internal SparkContext SparkContext get { return sparkContext; } } + public UdfRegistration Udf + { + get { return new UdfRegistration(sparkSessionProxy.Udf); } + } + /// /// Builder for SparkSession /// diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/UdfRegistration.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/UdfRegistration.cs new file mode 100644 index 00000000..b9c50083 --- /dev/null +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/UdfRegistration.cs @@ -0,0 +1,254 @@ +// Copyright (c) Microsoft. All rights reserved. +// Licensed under the MIT license. See LICENSE file in the project root for full license information. + +using System; +using System.Collections.Generic; +using System.Diagnostics.CodeAnalysis; +using System.Linq; +using System.Text; +using System.Threading.Tasks; +using Microsoft.Spark.CSharp.Core; +using Microsoft.Spark.CSharp.Proxy; +using Microsoft.Spark.CSharp.Services; + +namespace Microsoft.Spark.CSharp.Sql +{ + /// + /// Used for registering User Defined Functions. SparkSession.Udf is used to access instance of this type. + /// + public class UdfRegistration + { + private readonly ILoggerService logger = LoggerServiceFactory.GetLogger(typeof(UdfRegistration)); + + private IUdfRegistrationProxy udfRegistrationProxy; + + internal UdfRegistration(IUdfRegistrationProxy udfRegistrationProxy) + { + this.udfRegistrationProxy = udfRegistrationProxy; + } + + //TODO - the following section is a copy of the same functionality in SQLContext..refactoring needed + #region UDF Registration + /// + /// Register UDF with no input argument, e.g: + /// SqlContext.RegisterFunction<bool>("MyFilter", () => true); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter()"); + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 1 input argument, e.g: + /// SqlContext.RegisterFunction<bool, string>("MyFilter", (arg1) => arg1 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1)"); + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 2 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string>("MyFilter", (arg1, arg2) => arg1 != null && arg2 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2)"); + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 3 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, string>("MyFilter", (arg1, arg2, arg3) => arg1 != null && arg2 != null && arg3 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, columnName3)"); + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 4 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg4) => arg1 != null && arg2 != null && ... && arg3 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName4)"); + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 5 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg5) => arg1 != null && arg2 != null && ... && arg5 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName5)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 6 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg6) => arg1 != null && arg2 != null && ... && arg6 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName6)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 7 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg7) => arg1 != null && arg2 != null && ... && arg7 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName7)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 8 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg8) => arg1 != null && arg2 != null && ... && arg8 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName8)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 9 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg9) => arg1 != null && arg2 != null && ... && arg9 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName9)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + + /// + /// Register UDF with 10 input arguments, e.g: + /// SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg10) => arg1 != null && arg2 != null && ... && arg10 != null); + /// sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName10)"); + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// + public void RegisterFunction(string name, Func f) + { + logger.LogInfo("Name of the function to register {0}, method info", name, f.Method); + Func, IEnumerable> udfHelper = new UdfHelper(f).Execute; + udfRegistrationProxy.RegisterFunction(name, SparkContext.BuildCommand(new CSharpWorkerFunc(udfHelper), SerializedMode.Row, SerializedMode.Row), Functions.GetReturnType(typeof(RT))); + } + #endregion + } +} diff --git a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML index 0f4c49ce..0fde9067 100644 --- a/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML +++ b/csharp/Adapter/documentation/Microsoft.Spark.CSharp.Adapter.Doc.XML @@ -7675,6 +7675,171 @@ The Json object used to construct a StructType A new StructType instance + + + Register UDF with no input argument, e.g: + SqlContext.RegisterFunction<bool>("MyFilter", () => true); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter()"); + + + + + + + + Register UDF with 1 input argument, e.g: + SqlContext.RegisterFunction<bool, string>("MyFilter", (arg1) => arg1 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1)"); + + + + + + + + + Register UDF with 2 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string>("MyFilter", (arg1, arg2) => arg1 != null && arg2 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2)"); + + + + + + + + + + Register UDF with 3 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, string>("MyFilter", (arg1, arg2, arg3) => arg1 != null && arg2 != null && arg3 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, columnName3)"); + + + + + + + + + + + Register UDF with 4 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg4) => arg1 != null && arg2 != null && ... && arg3 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName4)"); + + + + + + + + + + + + Register UDF with 5 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg5) => arg1 != null && arg2 != null && ... && arg5 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName5)"); + + + + + + + + + + + + + Register UDF with 6 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg6) => arg1 != null && arg2 != null && ... && arg6 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName6)"); + + + + + + + + + + + + + + Register UDF with 7 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg7) => arg1 != null && arg2 != null && ... && arg7 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName7)"); + + + + + + + + + + + + + + + Register UDF with 8 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg8) => arg1 != null && arg2 != null && ... && arg8 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName8)"); + + + + + + + + + + + + + + + + Register UDF with 9 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg9) => arg1 != null && arg2 != null && ... && arg9 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName9)"); + + + + + + + + + + + + + + + + + Register UDF with 10 input arguments, e.g: + SqlContext.RegisterFunction<bool, string, string, ..., string>("MyFilter", (arg1, arg2, ..., arg10) => arg1 != null && arg2 != null && ... && arg10 != null); + sqlContext.Sql("SELECT * FROM MyTable where MyFilter(columnName1, columnName2, ..., columnName10)"); + + + + + + + + + + + + + + + An input stream that always returns the same RDD on each timestep. Useful for testing. diff --git a/csharp/AdapterTest/AdapterTest.csproj b/csharp/AdapterTest/AdapterTest.csproj index ca95b87c..c32ed7aa 100644 --- a/csharp/AdapterTest/AdapterTest.csproj +++ b/csharp/AdapterTest/AdapterTest.csproj @@ -119,6 +119,7 @@ + diff --git a/csharp/AdapterTest/Mocks/MockSparkSessionProxy.cs b/csharp/AdapterTest/Mocks/MockSparkSessionProxy.cs index da695c3f..b7cf2ca8 100644 --- a/csharp/AdapterTest/Mocks/MockSparkSessionProxy.cs +++ b/csharp/AdapterTest/Mocks/MockSparkSessionProxy.cs @@ -13,7 +13,7 @@ namespace AdapterTest.Mocks class MockSparkSessionProxy : ISparkSessionProxy { public ISqlContextProxy SqlContextProxy { get { return new MockSqlContextProxy(new MockSparkContextProxy(new MockSparkConfProxy()));} } - public IUdfRegistration Udf { get; } + public IUdfRegistrationProxy Udf { get; } public ICatalogProxy GetCatalog() { throw new NotImplementedException(); diff --git a/csharp/AdapterTest/UdfRegistrationTest.cs b/csharp/AdapterTest/UdfRegistrationTest.cs new file mode 100644 index 00000000..918fe9df --- /dev/null +++ b/csharp/AdapterTest/UdfRegistrationTest.cs @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft. All rights reserved. +// Licensed under the MIT license. See LICENSE file in the project root for full license information. + +using System; +using Microsoft.Spark.CSharp.Proxy; +using Microsoft.Spark.CSharp.Sql; +using Moq; +using NUnit.Framework; + +namespace AdapterTest +{ + [TestFixture] + public class UdfRegistrationTest + { + [Test] + public void TestRegisterFunction() + { + Mock mockUdfRegistrationProxy = new Mock(); + mockUdfRegistrationProxy.Setup(m => m.RegisterFunction(It.IsAny(), It.IsAny(), It.IsAny())); + + var udfRegistration = new UdfRegistration(mockUdfRegistrationProxy.Object); + + udfRegistration.RegisterFunction("Func0", () => "Func0"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func0", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func1", s => "Func1"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func1", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func2", (s1, s2) => "Func2"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func2", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func3", (s1, s2, s3) => "Func3"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func3", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func4", (s1, s2, s3, s4) => "Func4"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func4", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func5", (s1, s2, s3, s4, s5) => "Func5"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func5", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func6", (s1, s2, s3, s4, s5, s6) => "Func6"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func6", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func7", (s1, s2, s3, s4, s5, s6, s7) => "Func7"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func7", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func8", (s1, s2, s3, s4, s5, s6, s7, s8) => "Func8"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func8", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func9", (s1, s2, s3, s4, s5, s6, s7, s8, s9) => "Func9"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func9", It.IsAny(), "string")); + + udfRegistration.RegisterFunction("Func10", (s1, s2, s3, s4, s5, s6, s7, s8, s9, s10) => "Func10"); + mockUdfRegistrationProxy.Verify(m => m.RegisterFunction("Func10", It.IsAny(), "string")); + } + } +} diff --git a/csharp/Samples/Microsoft.Spark.CSharp/SparkSessionSamples.cs b/csharp/Samples/Microsoft.Spark.CSharp/SparkSessionSamples.cs index f628e1c8..7fd1dd3f 100644 --- a/csharp/Samples/Microsoft.Spark.CSharp/SparkSessionSamples.cs +++ b/csharp/Samples/Microsoft.Spark.CSharp/SparkSessionSamples.cs @@ -185,5 +185,31 @@ internal static void SSCreateDataFrameSample() Assert.AreEqual(schemaPeople.Json, dataFramePeople.Schema.Json); } } + + [Sample] + internal static void SparkSessionUdfSample() + { + GetSparkSession().Udf.RegisterFunction("FullAddress", (city, state) => city + " " + state); + GetSparkSession().Udf.RegisterFunction("PeopleFilter", (name, age) => name == "Bill" && age > 80); + + var peopleDataFrame = GetSparkSession().Read().Json(SparkCLRSamples.Configuration.GetInputDataPath(DataFrameSamples.PeopleJson)); + var functionAppliedDF = peopleDataFrame.SelectExpr("name", "age * 2 as age", + "FullAddress(address.city, address.state) as address") + .Where("PeopleFilter(name, age)"); + + functionAppliedDF.ShowSchema(); + functionAppliedDF.Show(); + + if (SparkCLRSamples.Configuration.IsValidationEnabled) + { + var collected = functionAppliedDF.Collect().ToArray(); + CollectionAssert.AreEquivalent(new[] { "name", "age", "address" }, + functionAppliedDF.Schema.Fields.Select(f => f.Name).ToArray()); + Assert.AreEqual(1, collected.Length); + Assert.AreEqual("Bill", collected[0].Get("name")); + Assert.AreEqual(86, collected[0].Get("age")); + Assert.AreEqual("Seattle Washington", collected[0].Get("address")); + } + } } } From b2be81c37fac8a9f0c7e0f8b73522340aa2f3156 Mon Sep 17 00:00:00 2001 From: skaarthik Date: Sat, 28 Jan 2017 01:21:35 -0800 Subject: [PATCH 20/20] disabling log messages that are not useful to Mobius app developers --- .../Interop/Ipc/WeakObjectManager.cs | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/WeakObjectManager.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/WeakObjectManager.cs index 62576db5..77be1776 100644 --- a/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/WeakObjectManager.cs +++ b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/WeakObjectManager.cs @@ -69,8 +69,8 @@ public int AdjustCheckCount(int currentReferenceCount) int previousReferencesCountBenchmark = referencesCountBenchmark; checkCount *= 2; referencesCountBenchmark = referencesCountBenchmark + referencesCountBenchmark / 2; - logger.LogDebug("Adjust checkCount from {0} to {1}, referencesCountBenchmark from {2} to {3}", - previousCheckCount, checkCount, previousReferencesCountBenchmark, referencesCountBenchmark); + //logger.LogDebug("Adjust checkCount from {0} to {1}, referencesCountBenchmark from {2} to {3}", + // previousCheckCount, checkCount, previousReferencesCountBenchmark, referencesCountBenchmark); } return checkCount; } @@ -134,14 +134,14 @@ public int GetReferencesCount() private void RunReleaseObjectLoop() { - logger.LogDebug("Checking objects thread start ..."); + //logger.LogDebug("Checking objects thread start ..."); while (shouldKeepRunning) { ReleseGarbageCollectedObjects(); Thread.Sleep(CheckInterval); } - logger.LogDebug("Checking objects thread stopped."); + //logger.LogDebug("Checking objects thread stopped."); } ~WeakObjectManagerImpl() @@ -165,13 +165,13 @@ private void ReleseGarbageCollectedObjects() int referencesCount = weakReferences.Count; if (referencesCount == 0) { - logger.LogDebug("check begin : quit as weakReferences.Count = 0"); + //logger.LogDebug("check begin : quit as weakReferences.Count = 0"); return; } var beginTime = DateTime.Now; int checkCount = checkCountController.AdjustCheckCount(referencesCount); - logger.LogDebug("check begin : weakReferences.Count = {0}, checkCount: {1}", referencesCount, checkCount); + //logger.LogDebug("check begin : weakReferences.Count = {0}, checkCount: {1}", referencesCount, checkCount); int garbageCount; var aliveList = ReleseGarbageCollectedObjects(checkCount, out garbageCount); @@ -179,11 +179,11 @@ private void ReleseGarbageCollectedObjects() aliveList.ForEach(item => weakReferences.Enqueue(item)); var timeStoreAlive = DateTime.Now; - logger.LogDebug("check end : released {0} garbage, remain {1} alive, used {2} ms : release garbage used {3} ms, store alive used {4} ms", - garbageCount, weakReferences.Count, (DateTime.Now - beginTime).TotalMilliseconds, - (timeReleaseGarbage - beginTime).TotalMilliseconds, - (timeStoreAlive - timeReleaseGarbage).TotalMilliseconds - ); + //logger.LogDebug("check end : released {0} garbage, remain {1} alive, used {2} ms : release garbage used {3} ms, store alive used {4} ms", + // garbageCount, weakReferences.Count, (DateTime.Now - beginTime).TotalMilliseconds, + // (timeReleaseGarbage - beginTime).TotalMilliseconds, + // (timeStoreAlive - timeReleaseGarbage).TotalMilliseconds + // ); } private List ReleseGarbageCollectedObjects(int checkCount, out int garbageCount) @@ -208,7 +208,7 @@ private List ReleseGarbageCollectedObjects(int checkC i++; if (i >= checkCount) { - logger.LogDebug("Stop releasing as exceeded allowed checkCount: {0}", checkCount); + //logger.LogDebug("Stop releasing as exceeded allowed checkCount: {0}", checkCount); break; } } @@ -238,7 +238,7 @@ public int GetAliveCount() public virtual void Dispose() { - logger.LogInfo("Dispose {0}", this.GetType()); + //logger.LogInfo("Dispose {0}", this.GetType()); shouldKeepRunning = false; } }