diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj b/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj
index 6dccf1b..316e131 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Adapter.csproj
@@ -76,6 +76,7 @@
+
@@ -118,7 +119,7 @@
-
+
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs
index b36b620..0e8e8ba 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Core/RDD.cs
@@ -283,7 +283,7 @@ namespace Microsoft.Spark.CSharp.Core
///
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 KeyValuePair(x, 0)).ReduceByKey((x, y) => x, numPartitions).Map(x => x.Key);
}
///
@@ -417,7 +417,7 @@ namespace Microsoft.Spark.CSharp.Core
else
{
const double delta = 0.00005;
- var gamma = - Math.Log(delta) / total;
+ var gamma = -Math.Log(delta) / total;
return Math.Min(1, fraction + gamma + Math.Sqrt(gamma * gamma + 2 * gamma * fraction));
}
}
@@ -811,6 +811,7 @@ namespace Microsoft.Spark.CSharp.Core
int left = num - items.Count;
IEnumerable partitions = Enumerable.Range(partsScanned, Math.Min(numPartsToTry, totalParts - partsScanned));
+
var mappedRDD = MapPartitionsWithIndex(new TakeHelper(left).Execute);
int port = sparkContext.SparkContextProxy.RunJob(mappedRDD.RddProxy, partitions);
@@ -867,7 +868,7 @@ namespace Microsoft.Spark.CSharp.Core
{
return Map>(v => new KeyValuePair(v, default(T))).SubtractByKey
(
- other.Map>(v => new KeyValuePair(v, default(T))),
+ other.Map>(v => new KeyValuePair(v, default(T))),
numPartitions
)
.Keys();
@@ -1044,7 +1045,7 @@ namespace Microsoft.Spark.CSharp.Core
///
public IEnumerable ToLocalIterator()
{
- foreach(int partition in Enumerable.Range(0, GetNumPartitions()))
+ foreach (int partition in Enumerable.Range(0, GetNumPartitions()))
{
var mappedRDD = MapPartitionsWithIndex((pid, iter) => iter);
int port = sparkContext.SparkContextProxy.RunJob(mappedRDD.RddProxy, Enumerable.Range(partition, 1));
@@ -1382,7 +1383,7 @@ namespace Microsoft.Spark.CSharp.Core
internal KeyValuePair Execute(T input)
{
- return new KeyValuePair(func(input), input);
+ return new KeyValuePair(func(input), input);
}
}
[Serializable]
@@ -1429,7 +1430,7 @@ namespace Microsoft.Spark.CSharp.Core
else if (y.Value)
return x;
else
- return new KeyValuePair(func(x.Key, y.Key), false);
+ return new KeyValuePair(func(x.Key, y.Key), false);
}
}
[Serializable]
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JsonSerDe.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JsonSerDe.cs
new file mode 100644
index 0000000..0ba0aea
--- /dev/null
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Interop/Ipc/JsonSerDe.cs
@@ -0,0 +1,68 @@
+// Copyright (c) Microsoft. All rights reserved.
+// Licensed under the MIT license. See LICENSE file in the project root for full license information.
+
+using System.Linq;
+using Newtonsoft.Json.Linq;
+
+namespace Microsoft.Spark.CSharp.Interop.Ipc
+{
+ ///
+ /// Json.NET Serialization/Deserialization helper class.
+ ///
+ public static class JsonSerDe
+ {
+ // Note: Scala side uses JSortedObject when parse Json, so the properties in JObject need to be sorted
+ ///
+ /// Extend method to sort items in a JSON object by keys.
+ ///
+ ///
+ ///
+ public static JObject SortProperties(this JObject jObject)
+ {
+ JObject sortedJObject = new JObject();
+ foreach (var property in jObject.Properties().OrderBy(p => p.Name))
+ {
+ if (property.Value is JObject)
+ {
+ var propJObject = property.Value as JObject;
+ sortedJObject.Add(property.Name, propJObject.SortProperties());
+ }
+ else if (property.Value is JArray)
+ {
+ var propJArray = property.Value as JArray;
+ sortedJObject.Add(property.Name, propJArray.SortProperties());
+ }
+ else
+ {
+ sortedJObject.Add(property.Name, property.Value);
+ }
+ }
+ return sortedJObject;
+ }
+
+ ///
+ /// Extend method to sort items in a JSON array by keys.
+ ///
+ public static JArray SortProperties(this JArray jArray)
+ {
+ JArray sortedJArray = new JArray();
+ if (jArray.Count == 0) return jArray;
+
+ foreach (var item in jArray)
+ {
+ if (item is JObject)
+ {
+ var sortedItem = ((JObject)item).SortProperties();
+ sortedJArray.Add(sortedItem);
+ }
+ else if (item is JArray)
+ {
+ var sortedItem = ((JArray)item).SortProperties();
+ sortedJArray.Add(sortedItem);
+ }
+ }
+ return sortedJArray;
+ }
+ }
+
+}
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkCLRProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkCLRProxy.cs
index 1d01d95..c3abf77 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkCLRProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISparkCLRProxy.cs
@@ -17,8 +17,6 @@ namespace Microsoft.Spark.CSharp.Proxy
ISparkContextProxy SparkContextProxy { get; }
ISparkConfProxy CreateSparkConf(bool loadDefaults = true);
ISparkContextProxy CreateSparkContext(ISparkConfProxy conf);
- IStructFieldProxy CreateStructField(string name, string dataType, bool isNullable);
- IStructTypeProxy CreateStructType(List fields);
IDStreamProxy CreateCSharpDStream(IDStreamProxy jdstream, byte[] func, string deserializer);
IDStreamProxy CreateCSharpTransformed2DStream(IDStreamProxy jdstream, IDStreamProxy jother, byte[] func, string deserializer, string deserializerOther);
IDStreamProxy CreateCSharpReducedWindowedDStream(IDStreamProxy jdstream, byte[] func, byte[] invFunc, int windowSeconds, int slideSeconds, string deserializer);
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISqlContextProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISqlContextProxy.cs
index e49204c..b84c86e 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISqlContextProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/ISqlContextProxy.cs
@@ -13,6 +13,7 @@ namespace Microsoft.Spark.CSharp.Proxy
{
internal interface ISqlContextProxy
{
+ IDataFrameProxy CreateDataFrame(IRDDProxy rddProxy, IStructTypeProxy structTypeProxy);
IDataFrameProxy ReadDataFrame(string path, StructType schema, Dictionary options);
IDataFrameProxy JsonFile(string path);
IDataFrameProxy TextFile(string path, StructType schema, string delimiter);
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/DataFrameIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/DataFrameIpcProxy.cs
index d735e73..d354efa 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/DataFrameIpcProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/DataFrameIpcProxy.cs
@@ -6,7 +6,6 @@ using System.Collections.Generic;
using System.Linq;
using Microsoft.Spark.CSharp.Core;
using Microsoft.Spark.CSharp.Interop.Ipc;
-using Microsoft.Spark.CSharp.Sql;
namespace Microsoft.Spark.CSharp.Proxy.Ipc
{
@@ -77,7 +76,7 @@ namespace Microsoft.Spark.CSharp.Proxy.Ipc
return
SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(
jvmDataFrameReference, "showString",
- new object[] { numberOfRows , truncate }).ToString();
+ new object[] { numberOfRows, truncate }).ToString();
}
public bool IsLocal()
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkCLRIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkCLRIpcProxy.cs
index 39d7fb9..9b521a4 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkCLRIpcProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkCLRIpcProxy.cs
@@ -62,33 +62,6 @@ namespace Microsoft.Spark.CSharp.Proxy.Ipc
return sparkContextProxy;
}
- public IStructFieldProxy CreateStructField(string name, string dataType, bool isNullable)
- {
- return new StructFieldIpcProxy(
- new JvmObjectReference(
- JvmBridge.CallStaticJavaMethod(
- "org.apache.spark.sql.api.csharp.SQLUtils", "createStructField",
- new object[] { name, dataType, isNullable }).ToString()
- )
- );
- }
-
- public IStructTypeProxy CreateStructType(List fields)
- {
- var fieldsReference = fields.Select(s => (s.StructFieldProxy as StructFieldIpcProxy).JvmStructFieldReference).ToList().Cast();
-
- var seq =
- new JvmObjectReference(
- JvmBridge.CallStaticJavaMethod("org.apache.spark.sql.api.csharp.SQLUtils",
- "toSeq", new object[] { fieldsReference }).ToString());
-
- return new StructTypeIpcProxy(
- new JvmObjectReference(
- JvmBridge.CallStaticJavaMethod("org.apache.spark.sql.api.csharp.SQLUtils", "createStructType", new object[] { seq }).ToString()
- )
- );
- }
-
public IDStreamProxy CreateCSharpDStream(IDStreamProxy jdstream, byte[] func, string deserializer)
{
var jvmDStreamReference = SparkCLRIpcProxy.JvmBridge.CallConstructor("org.apache.spark.streaming.api.csharp.CSharpDStream",
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs
index 8046fd6..9bcd7ef 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SparkContextIpcProxy.cs
@@ -116,7 +116,6 @@ namespace Microsoft.Spark.CSharp.Proxy.Ipc
return new RDDIpcProxy(jvmRddReference);
}
- //TODO - this implementation is slow. Replace with call to createRDDFromArray() in CSharpRDD
public IRDDProxy Parallelize(IEnumerable values, int numSlices)
{
var jvmRddReference = new JvmObjectReference((string)SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.api.csharp.CSharpRDD", "createRDDFromArray", new object[] { jvmSparkContextReference, values, numSlices }));
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SqlContextIpcProxy.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SqlContextIpcProxy.cs
index 6638143..5e4f62a 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SqlContextIpcProxy.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Proxy/Ipc/SqlContextIpcProxy.cs
@@ -21,6 +21,17 @@ namespace Microsoft.Spark.CSharp.Proxy.Ipc
this.jvmSqlContextReference = jvmSqlContextReference;
}
+ public IDataFrameProxy CreateDataFrame(IRDDProxy rddProxy, IStructTypeProxy structTypeProxy)
+ {
+ var rdd = new JvmObjectReference(SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod("org.apache.spark.sql.api.csharp.SQLUtils", "byteArrayRDDToAnyArrayRDD",
+ new object[] { (rddProxy as RDDIpcProxy).JvmRddReference }).ToString());
+
+ return new DataFrameIpcProxy(
+ new JvmObjectReference(
+ SparkCLRIpcProxy.JvmBridge.CallNonStaticJavaMethod(jvmSqlContextReference, "applySchemaToPythonRDD",
+ new object[] { rdd, (structTypeProxy as StructTypeIpcProxy).JvmStructTypeReference }).ToString()), this);
+ }
+
public IDataFrameProxy ReadDataFrame(string path, StructType schema, Dictionary options)
{
//TODO parameter Dictionary options is not used right now - it is meant to be passed on to data sources
@@ -44,7 +55,7 @@ namespace Microsoft.Spark.CSharp.Proxy.Ipc
new JvmObjectReference(
SparkCLRIpcProxy.JvmBridge.CallStaticJavaMethod(
"org.apache.spark.sql.api.csharp.SQLUtils", "loadTextFile",
- new object[] {jvmSqlContextReference, path, delimiter, (schema.StructTypeProxy as StructTypeIpcProxy).JvmStructTypeReference}).ToString()
+ new object[] { jvmSqlContextReference, path, delimiter, schema.Json}).ToString()
), this
);
}
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/DataFrame.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/DataFrame.cs
index bda8be0..400ab0c 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/DataFrame.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/DataFrame.cs
@@ -22,7 +22,7 @@ namespace Microsoft.Spark.CSharp.Sql
private readonly IDataFrameProxy dataFrameProxy;
[NonSerialized]
private readonly SparkContext sparkContext;
- [NonSerialized]
+
private StructType schema;
[NonSerialized]
private RDD rdd;
@@ -40,7 +40,7 @@ namespace Microsoft.Spark.CSharp.Sql
if (rdd == null)
{
rddProxy = dataFrameProxy.JavaToCSharp();
- rdd = new RDD(rddProxy, sparkContext, SerializedMode.Row);
+ rdd = new RDD(rddProxy, sparkContext, SerializedMode.Row);
}
return rdd;
}
@@ -137,7 +137,7 @@ namespace Microsoft.Spark.CSharp.Sql
///
public void ShowSchema()
{
- List nameTypeList = Schema.Fields.Select(structField => string.Format("{0}:{1}", structField.Name, structField.DataType.SimpleString())).ToList();
+ var nameTypeList = Schema.Fields.Select(structField => structField.SimpleString);
Console.WriteLine(string.Join(", ", nameTypeList));
}
@@ -145,18 +145,18 @@ namespace Microsoft.Spark.CSharp.Sql
/// Returns all of Rows in this DataFrame
///
public IEnumerable Collect()
- {
+ {
int port = RddProxy.CollectAndServe();
return Rdd.Collect(port).Cast();
}
///
- /// Converts the DataFrame to RDD of byte[]
+ /// Converts the DataFrame to RDD of Row
///
/// resulting RDD
- public RDD ToRDD() //RDD created using byte representation of GenericRow objects
+ public RDD ToRDD() //RDD created using byte representation of Row objects
{
- return new RDD(dataFrameProxy.ToRDD(), sparkContext);
+ return Rdd;
}
///
diff --git a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/Row.cs b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/Row.cs
index e5b0e2f..04af6e4 100644
--- a/csharp/Adapter/Microsoft.Spark.CSharp/Sql/Row.cs
+++ b/csharp/Adapter/Microsoft.Spark.CSharp/Sql/Row.cs
@@ -2,13 +2,9 @@
// Licensed under the MIT license. See LICENSE file in the project root for full license information.
using System;
+using System.Collections;
using System.Collections.Generic;
using System.Linq;
-using System.Text;
-using System.Threading.Tasks;
-using Newtonsoft.Json;
-using Newtonsoft.Json.Linq;
-
using Microsoft.Spark.CSharp.Services;
namespace Microsoft.Spark.CSharp.Sql
@@ -31,24 +27,24 @@ namespace Microsoft.Spark.CSharp.Sql
///
/// Schema for the row.
///
- public abstract RowSchema GetSchema();
+ public abstract StructType GetSchema();
///
/// Returns the value at position i.
///
- public abstract object Get(int i);
+ public abstract dynamic Get(int i);
///
/// Returns the value of a given columnName.
///
- public abstract object Get(string columnName);
+ public abstract dynamic Get(string columnName);
///
/// Returns the value at position i, the return value will be cast to type T.
///
public T GetAs(int i)
{
- object o = Get(i);
+ dynamic o = Get(i);
try
{
T result = (T)o;
@@ -66,7 +62,7 @@ namespace Microsoft.Spark.CSharp.Sql
///
public T GetAs(string columnName)
{
- object o = Get(columnName);
+ dynamic o = Get(columnName);
try
{
T result = (T)o;
@@ -80,145 +76,12 @@ namespace Microsoft.Spark.CSharp.Sql
}
}
- ///
- /// Schema of Row
- ///
- [Serializable]
- public class RowSchema
- {
- public string type;
- public List columns;
-
- private readonly Dictionary columnName2Index = new Dictionary();
-
- public RowSchema(string type)
- {
- this.type = type;
- this.columns = new List();
- }
-
- public RowSchema(string type, List cols)
- {
- int index = 0;
- foreach (var col in cols)
- {
- string columnName = col.name;
- //TODO - investigate the issue and uncomment the following checks
- /*
- * UDFs produce empty column name. Commenting out the following code at the time of upgrading to 1.5.2
- */
- //if (string.IsNullOrEmpty(columnName))
- //{
- // throw new Exception(string.Format("Null column name at pos: {0}", index));
- //}
-
- //if (columnName2Index.ContainsKey(columnName))
- //{
- // throw new Exception(string.Format("duplicate column name ({0}) in pos ({1}) and ({2})",
- // columnName, columnName2Index[columnName], index));
- //}
- columnName2Index[columnName] = index;
- index++;
- }
-
- this.type = type;
- this.columns = cols;
- }
-
- internal int GetIndexByColumnName(string ColumnName)
- {
- if (!columnName2Index.ContainsKey(ColumnName))
- {
- throw new Exception(string.Format("unknown ColumnName: {0}", ColumnName));
- }
-
- return columnName2Index[ColumnName];
- }
-
- public override string ToString()
- {
- string result;
-
- if (columns.Any())
- {
- List cols = new List();
- foreach (var col in columns)
- {
- cols.Add(col.ToString());
- }
-
- result = "{" +
- string.Format("type: {0}, columns: [{1}]", type, string.Join(", ", cols.ToArray())) +
- "}";
- }
- else
- {
- result = type;
- }
-
- return result;
- }
-
- internal static RowSchema ParseRowSchemaFromJson(string json)
- {
- JObject joType = JObject.Parse(json);
- string type = joType["type"].ToString();
-
- List columns = new List();
- List jtFields = joType["fields"].Children().ToList();
- foreach (JToken jtField in jtFields)
- {
- ColumnSchema col = ColumnSchema.ParseColumnSchemaFromJson(jtField.ToString());
- columns.Add(col);
- }
-
- return new RowSchema(type, columns);
- }
-
- }
-
- ///
- /// Schema for column
- ///
- [Serializable]
- public class ColumnSchema
- {
- public string name;
- public RowSchema type;
- public bool nullable;
-
- public override string ToString()
- {
- string str = string.Format("name: {0}, type: {1}, nullable: {2}", name, type, nullable);
- return "{" + str + "}";
- }
-
- internal static ColumnSchema ParseColumnSchemaFromJson(string json)
- {
- ColumnSchema col = new ColumnSchema();
- JObject joField = JObject.Parse(json);
- col.name = joField["name"].ToString();
- col.nullable = (bool)(joField["nullable"]);
-
- JToken jtType = joField["type"];
- if (jtType.Type == JTokenType.String)
- {
- col.type = new RowSchema(joField["type"].ToString());
- }
- else
- {
- col.type = RowSchema.ParseRowSchemaFromJson(joField["type"].ToString());
- }
-
- return col;
- }
- }
-
[Serializable]
internal class RowImpl : Row
{
- private readonly RowSchema schema;
- private readonly object[] values;
+ private readonly StructType schema;
+ public dynamic[] Values { get { return values; } }
+ private readonly dynamic[] values;
private readonly int columnCount;
@@ -229,17 +92,15 @@ namespace Microsoft.Spark.CSharp.Sql
return Get(index);
}
}
-
-
- internal RowImpl(object data, RowSchema schema)
+ internal RowImpl(dynamic data, StructType schema)
{
- if (data is object[])
+ if (data is dynamic[])
{
- values = data as object[];
+ values = data as dynamic[];
}
- else if (data is List