Skip to content
Snippets Groups Projects
  1. Nov 07, 2016
    • hyukjinkwon's avatar
      [SPARK-18295][SQL] Make to_json function null safe (matching it to from_json) · 3eda0570
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.
      
      Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.
      
      with the data below:
      
      ```scala
      import spark.implicits._
      
      val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
      df.show()
      ```
      
      ```
      +----+
      |   a|
      +----+
      | [1]|
      |null|
      +----+
      ```
      
      the codes below
      
      ```scala
      import org.apache.spark.sql.functions._
      
      df.select(to_json($"a")).show()
      ```
      
      produces..
      
      **Before**
      
      throws `NullPointException` as below:
      
      ```
      java.lang.NullPointerException
        at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
        at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
        at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
        at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
        at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
        at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
        at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
        at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      ```
      
      **After**
      
      ```
      +---------------+
      |structtojson(a)|
      +---------------+
      |       {"_1":1}|
      |           null|
      +---------------+
      ```
      
      ## How was this patch tested?
      
      Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #15792 from HyukjinKwon/SPARK-18295.
      3eda0570
    • Josh Rosen's avatar
      [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer · 3a710b94
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      When profiling heap dumps from the HistoryServer and live Spark web UIs, I found a large amount of memory being wasted on duplicated objects and strings. This patch's changes remove most of this duplication, resulting in over 40% memory savings for some benchmarks.
      
      - **Task metrics** (6441f0624dfcda9c7193a64bfb416a145b5aabdf): previously, every `TaskUIData` object would have its own instances of `InputMetricsUIData`, `OutputMetricsUIData`, `ShuffleReadMetrics`, and `ShuffleWriteMetrics`, but for many tasks these metrics are irrelevant because they're all zero. This patch changes how we construct these metrics in order to re-use a single immutable "empty" value for the cases where these metrics are empty.
      - **TaskInfo.accumulables** (ade86db901127bf13c0e0bdc3f09c933a093bb76): Previously, every `TaskInfo` object had its own empty `ListBuffer` for holding updates from named accumulators. Tasks which didn't use named accumulators still paid for the cost of allocating and storing this empty buffer. To avoid this overhead, I changed the `val` with a mutable buffer into a `var` which holds an immutable Scala list, allowing tasks which do not have named accumulator updates to share the same singleton `Nil` object.
      - **String.intern() in JSONProtocol** (7e05630e9a78c455db8c8c499f0590c864624e05): in the HistoryServer, executor hostnames and ids are deserialized from JSON, leading to massive duplication of these string objects. By calling `String.intern()` on the deserialized values we can remove all of this duplication. Since Spark now requires Java 7+ we don't have to worry about string interning exhausting the permgen (see http://java-performance.info/string-intern-in-java-6-7-8/).
      
      ## How was this patch tested?
      
      I ran
      
      ```
      sc.parallelize(1 to 100000, 100000).count()
      ```
      
      in `spark-shell` with event logging enabled, then loaded that event log in the HistoryServer, performed a full GC, and took a heap dump. According to YourKit, the changes in this patch reduced memory consumption by roughly 28 megabytes (or 770k Java objects):
      
      ![image](https://cloud.githubusercontent.com/assets/50748/19953276/4f3a28aa-a129-11e6-93df-d7fa91396f66.png)
      
      Here's a table illustrating the drop in objects due to deduplication (the drop is <100k for some objects because some events were dropped from the listener bus; this is a separate, existing bug that I'll address separately after CPU-profiling):
      
      ![image](https://cloud.githubusercontent.com/assets/50748/19953290/6a271290-a129-11e6-93ad-b825f1448886.png)
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #15743 from JoshRosen/spark-ui-memory-usage.
      3a710b94
    • Kazuaki Ishizaki's avatar
      [SPARK-17490][SQL] Optimize SerializeFromObject() for a primitive array · 19cf2080
      Kazuaki Ishizaki authored
      ## What changes were proposed in this pull request?
      
      Waiting for merging #13680
      
      This PR optimizes `SerializeFromObject()` for an primitive array. This is derived from #13758 to address one of problems by using a simple way in #13758.
      
      The current implementation always generates `GenericArrayData` from `SerializeFromObject()` for any type of an array in a logical plan. This involves a boxing at a constructor of `GenericArrayData` when `SerializedFromObject()` has an primitive array.
      
      This PR enables to generate `UnsafeArrayData` from `SerializeFromObject()` for a primitive array. It can avoid boxing to create an instance of `ArrayData` in the generated code by Catalyst.
      
      This PR also generate `UnsafeArrayData` in a case for `RowEncoder.serializeFor` or `CatalystTypeConverters.createToCatalystConverter`.
      
      Performance improvement of `SerializeFromObject()` is up to 2.0x
      
      ```
      OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
      Intel Xeon E3-12xx v2 (Ivy Bridge)
      
      Without this PR
      Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
      ------------------------------------------------------------------------------------------------
      Int                                            556 /  608         15.1          66.3       1.0X
      Double                                        1668 / 1746          5.0         198.8       0.3X
      
      with this PR
      Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
      ------------------------------------------------------------------------------------------------
      Int                                            352 /  401         23.8          42.0       1.0X
      Double                                         821 /  885         10.2          97.9       0.4X
      ```
      
      Here is an example program that will happen in mllib as described in [SPARK-16070](https://issues.apache.org/jira/browse/SPARK-16070).
      
      ```
      sparkContext.parallelize(Seq(Array(1, 2)), 1).toDS.map(e => e).show
      ```
      
      Generated code before applying this PR
      
      ``` java
      /* 039 */   protected void processNext() throws java.io.IOException {
      /* 040 */     while (inputadapter_input.hasNext()) {
      /* 041 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
      /* 042 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
      /* 043 */
      /* 044 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
      /* 045 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
      /* 046 */
      /* 047 */       boolean mapelements_isNull = false || false;
      /* 048 */       int[] mapelements_value = null;
      /* 049 */       if (!mapelements_isNull) {
      /* 050 */         Object mapelements_funcResult = null;
      /* 051 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
      /* 052 */         if (mapelements_funcResult == null) {
      /* 053 */           mapelements_isNull = true;
      /* 054 */         } else {
      /* 055 */           mapelements_value = (int[]) mapelements_funcResult;
      /* 056 */         }
      /* 057 */
      /* 058 */       }
      /* 059 */       mapelements_isNull = mapelements_value == null;
      /* 060 */
      /* 061 */       serializefromobject_argIsNulls[0] = mapelements_isNull;
      /* 062 */       serializefromobject_argValue = mapelements_value;
      /* 063 */
      /* 064 */       boolean serializefromobject_isNull = false;
      /* 065 */       for (int idx = 0; idx < 1; idx++) {
      /* 066 */         if (serializefromobject_argIsNulls[idx]) { serializefromobject_isNull = true; break; }
      /* 067 */       }
      /* 068 */
      /* 069 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
      /* 070 */       serializefromobject_holder.reset();
      /* 071 */
      /* 072 */       serializefromobject_rowWriter.zeroOutNullBytes();
      /* 073 */
      /* 074 */       if (serializefromobject_isNull) {
      /* 075 */         serializefromobject_rowWriter.setNullAt(0);
      /* 076 */       } else {
      /* 077 */         // Remember the current cursor so that we can calculate how many bytes are
      /* 078 */         // written later.
      /* 079 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
      /* 080 */
      /* 081 */         if (serializefromobject_value instanceof UnsafeArrayData) {
      /* 082 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
      /* 083 */           // grow the global buffer before writing data.
      /* 084 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
      /* 085 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 086 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
      /* 087 */
      /* 088 */         } else {
      /* 089 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
      /* 090 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
      /* 091 */
      /* 092 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
      /* 093 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
      /* 094 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
      /* 095 */             } else {
      /* 096 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
      /* 097 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
      /* 098 */             }
      /* 099 */           }
      /* 100 */         }
      /* 101 */
      /* 102 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
      /* 103 */       }
      /* 104 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
      /* 105 */       append(serializefromobject_result);
      /* 106 */       if (shouldStop()) return;
      /* 107 */     }
      /* 108 */   }
      /* 109 */ }
      ```
      
      Generated code after applying this PR
      
      ``` java
      /* 035 */   protected void processNext() throws java.io.IOException {
      /* 036 */     while (inputadapter_input.hasNext()) {
      /* 037 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
      /* 038 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
      /* 039 */
      /* 040 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
      /* 041 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
      /* 042 */
      /* 043 */       boolean mapelements_isNull = false || false;
      /* 044 */       int[] mapelements_value = null;
      /* 045 */       if (!mapelements_isNull) {
      /* 046 */         Object mapelements_funcResult = null;
      /* 047 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
      /* 048 */         if (mapelements_funcResult == null) {
      /* 049 */           mapelements_isNull = true;
      /* 050 */         } else {
      /* 051 */           mapelements_value = (int[]) mapelements_funcResult;
      /* 052 */         }
      /* 053 */
      /* 054 */       }
      /* 055 */       mapelements_isNull = mapelements_value == null;
      /* 056 */
      /* 057 */       boolean serializefromobject_isNull = mapelements_isNull;
      /* 058 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(mapelements_value);
      /* 059 */       serializefromobject_isNull = serializefromobject_value == null;
      /* 060 */       serializefromobject_holder.reset();
      /* 061 */
      /* 062 */       serializefromobject_rowWriter.zeroOutNullBytes();
      /* 063 */
      /* 064 */       if (serializefromobject_isNull) {
      /* 065 */         serializefromobject_rowWriter.setNullAt(0);
      /* 066 */       } else {
      /* 067 */         // Remember the current cursor so that we can calculate how many bytes are
      /* 068 */         // written later.
      /* 069 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
      /* 070 */
      /* 071 */         if (serializefromobject_value instanceof UnsafeArrayData) {
      /* 072 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
      /* 073 */           // grow the global buffer before writing data.
      /* 074 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
      /* 075 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
      /* 076 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
      /* 077 */
      /* 078 */         } else {
      /* 079 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
      /* 080 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
      /* 081 */
      /* 082 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
      /* 083 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
      /* 084 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
      /* 085 */             } else {
      /* 086 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
      /* 087 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
      /* 088 */             }
      /* 089 */           }
      /* 090 */         }
      /* 091 */
      /* 092 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
      /* 093 */       }
      /* 094 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
      /* 095 */       append(serializefromobject_result);
      /* 096 */       if (shouldStop()) return;
      /* 097 */     }
      /* 098 */   }
      /* 099 */ }
      ```
      ## How was this patch tested?
      
      Added a test in `DatasetSuite`, `RowEncoderSuite`, and `CatalystTypeConvertersSuite`
      
      Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
      
      Closes #15044 from kiszk/SPARK-17490.
      19cf2080
    • Hyukjin Kwon's avatar
      [SPARK-14914][CORE] Fix Resource not closed after using, mostly for unit tests · 8f0ea011
      Hyukjin Kwon authored
      ## What changes were proposed in this pull request?
      
      Close `FileStreams`, `ZipFiles` etc to release the resources after using. Not closing the resources will cause IO Exception to be raised while deleting temp files.
      ## How was this patch tested?
      
      Existing tests
      
      Author: U-FAREAST\tl <tl@microsoft.com>
      Author: hyukjinkwon <gurwls223@gmail.com>
      Author: Tao LI <tl@microsoft.com>
      
      Closes #15618 from HyukjinKwon/SPARK-14914-1.
      8f0ea011
    • Weiqing Yang's avatar
      [SPARK-17108][SQL] Fix BIGINT and INT comparison failure in spark sql · 0d95662e
      Weiqing Yang authored
      ## What changes were proposed in this pull request?
      
      Add a function to check if two integers are compatible when invoking `acceptsType()` in `DataType`.
      ## How was this patch tested?
      
      Manually.
      E.g.
      
      ```
          spark.sql("create table t3(a map<bigint, array<string>>)")
          spark.sql("select * from t3 where a[1] is not null")
      ```
      
      Before:
      
      ```
      cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
      org.apache.spark.sql.AnalysisException: cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
          at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
          at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:82)
          at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
      at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:307)
      ```
      
      After:
       Run the sql queries above. No errors.
      
      Author: Weiqing Yang <yangweiqing001@gmail.com>
      
      Closes #15448 from weiqingy/SPARK_17108.
      0d95662e
    • Tathagata Das's avatar
      [SPARK-18283][STRUCTURED STREAMING][KAFKA] Added test to check whether default... · b06c23db
      Tathagata Das authored
      [SPARK-18283][STRUCTURED STREAMING][KAFKA] Added test to check whether default starting offset in latest
      
      ## What changes were proposed in this pull request?
      
      Added test to check whether default starting offset in latest
      
      ## How was this patch tested?
      new unit test
      
      Author: Tathagata Das <tathagata.das1565@gmail.com>
      
      Closes #15778 from tdas/SPARK-18283.
      b06c23db
    • Yanbo Liang's avatar
      [SPARK-18291][SPARKR][ML] SparkR glm predict should output original label when family = binomial. · daa975f4
      Yanbo Liang authored
      ## What changes were proposed in this pull request?
      SparkR ```spark.glm``` predict should output original label when family = "binomial".
      
      ## How was this patch tested?
      Add unit test.
      You can also run the following code to test:
      ```R
      training <- suppressWarnings(createDataFrame(iris))
      training <- training[training$Species %in% c("versicolor", "virginica"), ]
      model <- spark.glm(training, Species ~ Sepal_Length + Sepal_Width,family = binomial(link = "logit"))
      showDF(predict(model, training))
      ```
      Before this change:
      ```
      +------------+-----------+------------+-----------+----------+-----+-------------------+
      |Sepal_Length|Sepal_Width|Petal_Length|Petal_Width|   Species|label|         prediction|
      +------------+-----------+------------+-----------+----------+-----+-------------------+
      |         7.0|        3.2|         4.7|        1.4|versicolor|  0.0| 0.8271421517601544|
      |         6.4|        3.2|         4.5|        1.5|versicolor|  0.0| 0.6044595910413112|
      |         6.9|        3.1|         4.9|        1.5|versicolor|  0.0| 0.7916340858281998|
      |         5.5|        2.3|         4.0|        1.3|versicolor|  0.0|0.16080518180591158|
      |         6.5|        2.8|         4.6|        1.5|versicolor|  0.0| 0.6112229217050189|
      |         5.7|        2.8|         4.5|        1.3|versicolor|  0.0| 0.2555087295500885|
      |         6.3|        3.3|         4.7|        1.6|versicolor|  0.0| 0.5681507664364834|
      |         4.9|        2.4|         3.3|        1.0|versicolor|  0.0|0.05990570219972002|
      |         6.6|        2.9|         4.6|        1.3|versicolor|  0.0| 0.6644434078306246|
      |         5.2|        2.7|         3.9|        1.4|versicolor|  0.0|0.11293577405862379|
      |         5.0|        2.0|         3.5|        1.0|versicolor|  0.0|0.06152372321585971|
      |         5.9|        3.0|         4.2|        1.5|versicolor|  0.0|0.35250697207602555|
      |         6.0|        2.2|         4.0|        1.0|versicolor|  0.0|0.32267018290814303|
      |         6.1|        2.9|         4.7|        1.4|versicolor|  0.0|  0.433391153814592|
      |         5.6|        2.9|         3.6|        1.3|versicolor|  0.0| 0.2280744262436993|
      |         6.7|        3.1|         4.4|        1.4|versicolor|  0.0| 0.7219848389339459|
      |         5.6|        3.0|         4.5|        1.5|versicolor|  0.0|0.23527698971404695|
      |         5.8|        2.7|         4.1|        1.0|versicolor|  0.0|  0.285024533520016|
      |         6.2|        2.2|         4.5|        1.5|versicolor|  0.0| 0.4107047877447493|
      |         5.6|        2.5|         3.9|        1.1|versicolor|  0.0|0.20083561961645083|
      +------------+-----------+------------+-----------+----------+-----+-------------------+
      ```
      After this change:
      ```
      +------------+-----------+------------+-----------+----------+-----+----------+
      |Sepal_Length|Sepal_Width|Petal_Length|Petal_Width|   Species|label|prediction|
      +------------+-----------+------------+-----------+----------+-----+----------+
      |         7.0|        3.2|         4.7|        1.4|versicolor|  0.0| virginica|
      |         6.4|        3.2|         4.5|        1.5|versicolor|  0.0| virginica|
      |         6.9|        3.1|         4.9|        1.5|versicolor|  0.0| virginica|
      |         5.5|        2.3|         4.0|        1.3|versicolor|  0.0|versicolor|
      |         6.5|        2.8|         4.6|        1.5|versicolor|  0.0| virginica|
      |         5.7|        2.8|         4.5|        1.3|versicolor|  0.0|versicolor|
      |         6.3|        3.3|         4.7|        1.6|versicolor|  0.0| virginica|
      |         4.9|        2.4|         3.3|        1.0|versicolor|  0.0|versicolor|
      |         6.6|        2.9|         4.6|        1.3|versicolor|  0.0| virginica|
      |         5.2|        2.7|         3.9|        1.4|versicolor|  0.0|versicolor|
      |         5.0|        2.0|         3.5|        1.0|versicolor|  0.0|versicolor|
      |         5.9|        3.0|         4.2|        1.5|versicolor|  0.0|versicolor|
      |         6.0|        2.2|         4.0|        1.0|versicolor|  0.0|versicolor|
      |         6.1|        2.9|         4.7|        1.4|versicolor|  0.0|versicolor|
      |         5.6|        2.9|         3.6|        1.3|versicolor|  0.0|versicolor|
      |         6.7|        3.1|         4.4|        1.4|versicolor|  0.0| virginica|
      |         5.6|        3.0|         4.5|        1.5|versicolor|  0.0|versicolor|
      |         5.8|        2.7|         4.1|        1.0|versicolor|  0.0|versicolor|
      |         6.2|        2.2|         4.5|        1.5|versicolor|  0.0|versicolor|
      |         5.6|        2.5|         3.9|        1.1|versicolor|  0.0|versicolor|
      +------------+-----------+------------+-----------+----------+-----+----------+
      ```
      
      Author: Yanbo Liang <ybliang8@gmail.com>
      
      Closes #15788 from yanboliang/spark-18291.
      daa975f4
    • Liang-Chi Hsieh's avatar
      [SPARK-18125][SQL] Fix a compilation error in codegen due to splitExpression · a814eeac
      Liang-Chi Hsieh authored
      ## What changes were proposed in this pull request?
      
      As reported in the jira, sometimes the generated java code in codegen will cause compilation error.
      
      Code snippet to test it:
      
          case class Route(src: String, dest: String, cost: Int)
          case class GroupedRoutes(src: String, dest: String, routes: Seq[Route])
      
          val ds = sc.parallelize(Array(
            Route("a", "b", 1),
            Route("a", "b", 2),
            Route("a", "c", 2),
            Route("a", "d", 10),
            Route("b", "a", 1),
            Route("b", "a", 5),
            Route("b", "c", 6))
          ).toDF.as[Route]
      
          val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
            .groupByKey(r => (r.src, r.dest))
            .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) =>
              GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes)
            }.map(_._2)
      
      The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error.
      
      ## How was this patch tested?
      
      Jenkins tests.
      
      Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.
      
      Author: Liang-Chi Hsieh <viirya@gmail.com>
      
      Closes #15693 from viirya/fix-codege-compilation-error.
      a814eeac
    • gatorsmile's avatar
      [SPARK-16904][SQL] Removal of Hive Built-in Hash Functions and TestHiveFunctionRegistry · 57626a55
      gatorsmile authored
      ### What changes were proposed in this pull request?
      
      Currently, the Hive built-in `hash` function is not being used in Spark since Spark 2.0. The public interface does not allow users to unregister the Spark built-in functions. Thus, users will never use Hive's built-in `hash` function.
      
      The only exception here is `TestHiveFunctionRegistry`, which allows users to unregister the built-in functions. Thus, we can load Hive's hash function in the test cases. If we disable it, 10+ test cases will fail because the results are different from the Hive golden answer files.
      
      This PR is to remove `hash` from the list of `hiveFunctions` in `HiveSessionCatalog`. It will also remove `TestHiveFunctionRegistry`. This removal makes us easier to remove `TestHiveSessionState` in the future.
      ### How was this patch tested?
      N/A
      
      Author: gatorsmile <gatorsmile@gmail.com>
      
      Closes #14498 from gatorsmile/removeHash.
      57626a55
    • Reynold Xin's avatar
      [SPARK-18296][SQL] Use consistent naming for expression test suites · 9db06c44
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.
      
      ## How was this patch tested?
      This is a test-only naming change.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #15793 from rxin/SPARK-18296.
      9db06c44
    • Reynold Xin's avatar
  2. Nov 06, 2016
    • Wenchen Fan's avatar
      [SPARK-18173][SQL] data source tables should support truncating partition · 46b2e499
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
      ## How was this patch tested?
      
      existing tests
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #15688 from cloud-fan/truncate.
      46b2e499
    • hyukjinkwon's avatar
      [SPARK-18269][SQL] CSV datasource should read null properly when schema is lager than parsed tokens · 556a3b7d
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      Currently, there are the three cases when reading CSV by datasource when it is `PERMISSIVE` parse mode.
      
      - schema == parsed tokens (from each line)
        No problem to cast the value in the tokens to the field in the schema as they are equal.
      
      - schema < parsed tokens (from each line)
        It slices the tokens into the number of fields in schema.
      
      - schema > parsed tokens (from each line)
        It appends `null` into parsed tokens so that safely values can be casted with the schema.
      
      However, when `null` is appended in the third case, we should take `null` into account when casting the values.
      
      In case of `StringType`, it is fine as `UTF8String.fromString(datum)` produces `null` when the input is `null`. Therefore, this case will happen only when schema is explicitly given and schema includes data types that are not `StringType`.
      
      The codes below:
      
      ```scala
      val path = "/tmp/a"
      Seq("1").toDF().write.text(path.getAbsolutePath)
      val schema = StructType(
        StructField("a", IntegerType, true) ::
        StructField("b", IntegerType, true) :: Nil)
      spark.read.schema(schema).option("header", "false").csv(path).show()
      ```
      
      prints
      
      **Before**
      
      ```
      java.lang.NumberFormatException: null
      at java.lang.Integer.parseInt(Integer.java:542)
      at java.lang.Integer.parseInt(Integer.java:615)
      at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
      at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
      at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:24)
      ```
      
      **After**
      
      ```
      +---+----+
      |  a|   b|
      +---+----+
      |  1|null|
      +---+----+
      ```
      
      ## How was this patch tested?
      
      Unit test in `CSVSuite.scala` and `CSVTypeCastSuite.scala`
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #15767 from HyukjinKwon/SPARK-18269.
      556a3b7d
    • Wojciech Szymanski's avatar
      [SPARK-18210][ML] Pipeline.copy does not create an instance with the same UID · b89d0556
      Wojciech Szymanski authored
      ## What changes were proposed in this pull request?
      
      Motivation:
      `org.apache.spark.ml.Pipeline.copy(extra: ParamMap)` does not create an instance with the same UID. It does not conform to the method specification from its base class `org.apache.spark.ml.param.Params.copy(extra: ParamMap)`
      
      Solution:
      - fix for Pipeline UID
      - introduced new tests for `org.apache.spark.ml.Pipeline.copy`
      - minor improvements in test for `org.apache.spark.ml.PipelineModel.copy`
      
      ## How was this patch tested?
      
      Introduced new unit test: `org.apache.spark.ml.PipelineSuite."Pipeline.copy"`
      Improved existing unit test: `org.apache.spark.ml.PipelineSuite."PipelineModel.copy"`
      
      Author: Wojciech Szymanski <wk.szymanski@gmail.com>
      
      Closes #15759 from wojtek-szymanski/SPARK-18210.
      b89d0556
    • hyukjinkwon's avatar
      [SPARK-17854][SQL] rand/randn allows null/long as input seed · 340f09d1
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.
      
      So, this PR includes both changes below:
      - `null` support
      
        It seems MySQL also accepts this.
      
        ``` sql
        mysql> select rand(0);
        +---------------------+
        | rand(0)             |
        +---------------------+
        | 0.15522042769493574 |
        +---------------------+
        1 row in set (0.00 sec)
      
        mysql> select rand(NULL);
        +---------------------+
        | rand(NULL)          |
        +---------------------+
        | 0.15522042769493574 |
        +---------------------+
        1 row in set (0.00 sec)
        ```
      
        and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)
      
        So the codes below:
      
        ``` scala
        spark.range(1).selectExpr("rand(null)").show()
        ```
      
        prints..
      
        **Before**
      
        ```
          Input argument to rand must be an integer literal.;; line 1 pos 0
        org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
        at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
        at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
        ```
      
        **After**
      
        ```
          +-----------------------+
          |rand(CAST(NULL AS INT))|
          +-----------------------+
          |    0.13385709732307427|
          +-----------------------+
        ```
      - `LongType` support in SQL.
      
        In addition, it make the function allows to take `LongType` consistently within Scala/SQL.
      
        In more details, the codes below:
      
        ``` scala
        spark.range(1).select(rand(1), rand(1L)).show()
        spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
        ```
      
        prints..
      
        **Before**
      
        ```
        +------------------+------------------+
        |           rand(1)|           rand(1)|
        +------------------+------------------+
        |0.2630967864682161|0.2630967864682161|
        +------------------+------------------+
      
        Input argument to rand must be an integer literal.;; line 1 pos 0
        org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
        at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
        at
        ```
      
        **After**
      
        ```
        +------------------+------------------+
        |           rand(1)|           rand(1)|
        +------------------+------------------+
        |0.2630967864682161|0.2630967864682161|
        +------------------+------------------+
      
        +------------------+------------------+
        |           rand(1)|           rand(1)|
        +------------------+------------------+
        |0.2630967864682161|0.2630967864682161|
        +------------------+------------------+
        ```
      ## How was this patch tested?
      
      Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #15432 from HyukjinKwon/SPARK-17854.
      Unverified
      340f09d1
    • sethah's avatar
      [SPARK-18276][ML] ML models should copy the training summary and set parent · 23ce0d1e
      sethah authored
      ## What changes were proposed in this pull request?
      
      Only some of the models which contain a training summary currently set the summaries in the copy method. Linear/Logistic regression do, GLR, GMM, KM, and BKM do not. Additionally, these copy methods did not set the parent pointer of the copied model. This patch modifies the copy methods of the four models mentioned above to copy the training summary and set the parent.
      
      ## How was this patch tested?
      
      Add unit tests in Linear/Logistic/GeneralizedLinear regression and GaussianMixture/KMeans/BisectingKMeans to check the parent pointer of the copied model and check that the copied model has a summary.
      
      Author: sethah <seth.hendrickson16@gmail.com>
      
      Closes #15773 from sethah/SPARK-18276.
      23ce0d1e
  3. Nov 05, 2016
  4. Nov 04, 2016
    • Josh Rosen's avatar
      [SPARK-18256] Improve the performance of event log replay in HistoryServer · 0e3312ee
      Josh Rosen authored
      ## What changes were proposed in this pull request?
      
      This patch significantly improves the performance of event log replay in the HistoryServer via two simple changes:
      
      - **Don't use `extractOpt`**: it turns out that `json4s`'s `extractOpt` method uses exceptions for control flow, causing huge performance bottlenecks due to the overhead of initializing exceptions. To avoid this overhead, we can simply use our own` Utils.jsonOption` method. This patch replaces all uses of `extractOpt` with `Utils.jsonOption` and adds a style checker rule to ban the use of the slow `extractOpt` method.
      - **Don't call `Utils.getFormattedClassName` for every event**: the old code called` Utils.getFormattedClassName` dozens of times per replayed event in order to match up class names in events with SparkListener event names. By simply storing the results of these calls in constants rather than recomputing them, we're able to eliminate a huge performance hotspot by removing thousands of expensive `Class.getSimpleName` calls.
      
      ## How was this patch tested?
      
      Tested by profiling the replay of a long event log using YourKit. For an event log containing 1000+ jobs, each of which had thousands of tasks, the changes in this patch cut the replay time in half:
      
      ![image](https://cloud.githubusercontent.com/assets/50748/19980953/31154622-a1bd-11e6-9be4-21fbb9b3f9a7.png)
      
      Prior to this patch's changes, the two slowest methods in log replay were internal exceptions thrown by `Json4S` and calls to `Class.getSimpleName()`:
      
      ![image](https://cloud.githubusercontent.com/assets/50748/19981052/87416cce-a1bd-11e6-9f25-06a7cd391822.png)
      
      After this patch, these hotspots are completely eliminated.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #15756 from JoshRosen/speed-up-jsonprotocol.
      0e3312ee
    • Eric Liang's avatar
      [SPARK-18167] Re-enable the non-flaky parts of SQLQuerySuite · 4cee2ce2
      Eric Liang authored
      ## What changes were proposed in this pull request?
      
      It seems the proximate cause of the test failures is that `cast(str as decimal)` in derby will raise an exception instead of returning NULL. This is a problem since Hive sometimes inserts `__HIVE_DEFAULT_PARTITION__` entries into the partition table as documented here: https://github.com/apache/hive/blob/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java#L1034
      
      Basically, when these special default partitions are present, partition pruning pushdown using the SQL-direct mode will fail due this cast exception. As commented on in `MetaStoreDirectSql.java` above, this is normally fine since Hive falls back to JDO pruning, however when the pruning predicate contains an unsupported operator such as `>`, that will fail as well.
      
      The only remaining question is why this behavior is nondeterministic. We know that when the test flakes, retries do not help, therefore the cause must be environmental. The current best hypothesis is that some config is different between different jenkins runs, which is why this PR prints out the Spark SQL and Hive confs for the test. The hope is that by comparing the config state for failure vs success we can isolate the root cause of the flakiness.
      
      **Update:** we could not isolate the issue. It does not seem to be due to configuration differences. As such, I'm going to enable the non-flaky parts of the test since we are fairly confident these issues only occur with Derby (which is not used in production).
      
      ## How was this patch tested?
      
      N/A
      
      Author: Eric Liang <ekl@databricks.com>
      
      Closes #15725 from ericl/print-confs-out.
      4cee2ce2
    • Herman van Hovell's avatar
      [SPARK-17337][SQL] Do not pushdown predicates through filters with predicate subqueries · 550cd56e
      Herman van Hovell authored
      ## What changes were proposed in this pull request?
      The `PushDownPredicate` rule can create a wrong result if we try to push a filter containing a predicate subquery through a project when the subquery and the project share attributes (have the same source).
      
      The current PR fixes this by making sure that we do not push down when there is a predicate subquery that outputs the same attributes as the filters new child plan.
      
      ## How was this patch tested?
      Added a test to `SubquerySuite`. nsyca has done previous work this. I have taken test from his initial PR.
      
      Author: Herman van Hovell <hvanhovell@databricks.com>
      
      Closes #15761 from hvanhovell/SPARK-17337.
      550cd56e
    • Adam Roberts's avatar
      [SPARK-18197][CORE] Optimise AppendOnlyMap implementation · a42d738c
      Adam Roberts authored
      ## What changes were proposed in this pull request?
      This improvement works by using the fastest comparison test first and we observed a 1% throughput performance improvement on PageRank (HiBench large profile) with this change.
      
      We used tprof and before the change in AppendOnlyMap.changeValue (where the optimisation occurs) this method was being used for 8053 profiling ticks representing 0.72% of the overall application time.
      
      After this change we observed this method only occurring for 2786 ticks and for 0.25% of the overall time.
      
      ## How was this patch tested?
      Existing unit tests and for performance we used HiBench large, profiling with tprof and IBM Healthcenter.
      
      Author: Adam Roberts <aroberts@uk.ibm.com>
      
      Closes #15714 from a-roberts/patch-9.
      a42d738c
    • Reynold Xin's avatar
      Closing some stale/invalid pull requests · 14f235d5
      Reynold Xin authored
      Closes #15758
      Closes #15753
      Closes #12708
      14f235d5
    • Dongjoon Hyun's avatar
      [SPARK-18200][GRAPHX][FOLLOW-UP] Support zero as an initial capacity in OpenHashSet · 27602c33
      Dongjoon Hyun authored
      ## What changes were proposed in this pull request?
      
      This is a follow-up PR of #15741 in order to keep `nextPowerOf2` consistent.
      
      **Before**
      ```
      nextPowerOf2(0) => 2
      nextPowerOf2(1) => 1
      nextPowerOf2(2) => 2
      nextPowerOf2(3) => 4
      nextPowerOf2(4) => 4
      nextPowerOf2(5) => 8
      ```
      
      **After**
      ```
      nextPowerOf2(0) => 1
      nextPowerOf2(1) => 1
      nextPowerOf2(2) => 2
      nextPowerOf2(3) => 4
      nextPowerOf2(4) => 4
      nextPowerOf2(5) => 8
      ```
      
      ## How was this patch tested?
      
      N/A
      
      Author: Dongjoon Hyun <dongjoon@apache.org>
      
      Closes #15754 from dongjoon-hyun/SPARK-18200-2.
      27602c33
    • Felix Cheung's avatar
      [SPARK-14393][SQL][DOC] update doc for python and R · a08463b1
      Felix Cheung authored
      ## What changes were proposed in this pull request?
      
      minor doc update that should go to master & branch-2.1
      
      ## How was this patch tested?
      
      manual
      
      Author: Felix Cheung <felixcheung_m@hotmail.com>
      
      Closes #15747 from felixcheung/pySPARK-14393.
      a08463b1
  5. Nov 03, 2016
    • Herman van Hovell's avatar
      [SPARK-18259][SQL] Do not capture Throwable in QueryExecution · aa412c55
      Herman van Hovell authored
      ## What changes were proposed in this pull request?
      `QueryExecution.toString` currently captures `java.lang.Throwable`s; this is far from a best practice and can lead to confusing situation or invalid application states. This PR fixes this by only capturing `AnalysisException`s.
      
      ## How was this patch tested?
      Added a `QueryExecutionSuite`.
      
      Author: Herman van Hovell <hvanhovell@databricks.com>
      
      Closes #15760 from hvanhovell/SPARK-18259.
      aa412c55
    • Sean Owen's avatar
      [SPARK-18138][DOCS] Document that Java 7, Python 2.6, Scala 2.10, Hadoop < 2.6... · dc4c6009
      Sean Owen authored
      [SPARK-18138][DOCS] Document that Java 7, Python 2.6, Scala 2.10, Hadoop < 2.6 are deprecated in Spark 2.1.0
      
      ## What changes were proposed in this pull request?
      
      Document that Java 7, Python 2.6, Scala 2.10, Hadoop < 2.6 are deprecated in Spark 2.1.0. This does not actually implement any of the change in SPARK-18138, just peppers the documentation with notices about it.
      
      ## How was this patch tested?
      
      Doc build
      
      Author: Sean Owen <sowen@cloudera.com>
      
      Closes #15733 from srowen/SPARK-18138.
      dc4c6009
    • Reynold Xin's avatar
      [SPARK-18257][SS] Improve error reporting for FileStressSuite · f22954ad
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      This patch improves error reporting for FileStressSuite, when there is an error in Spark itself (not user code). This works by simply tightening the exception verification, and gets rid of the unnecessary thread for starting the stream.
      
      Also renamed the class FileStreamStressSuite to make it more obvious it is a streaming suite.
      
      ## How was this patch tested?
      This is a test only change and I manually verified error reporting by injecting some bug in the addBatch code for FileStreamSink.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #15757 from rxin/SPARK-18257.
      f22954ad
    • wm624@hotmail.com's avatar
      [SPARKR][TEST] remove unnecessary suppressWarnings · e8920252
      wm624@hotmail.com authored
      ## What changes were proposed in this pull request?
      
      In test_mllib.R, there are two unnecessary suppressWarnings. This PR just removes them.
      
      ## How was this patch tested?
      
      Existing unit tests.
      
      Author: wm624@hotmail.com <wm624@hotmail.com>
      
      Closes #15697 from wangmiao1981/rtest.
      e8920252
    • cody koeninger's avatar
      [SPARK-18212][SS][KAFKA] increase executor poll timeout · 67659c9a
      cody koeninger authored
      ## What changes were proposed in this pull request?
      
      Increase poll timeout to try and address flaky test
      
      ## How was this patch tested?
      
      Ran existing unit tests
      
      Author: cody koeninger <cody@koeninger.org>
      
      Closes #15737 from koeninger/SPARK-18212.
      67659c9a
    • Kishor Patil's avatar
      [SPARK-18099][YARN] Fail if same files added to distributed cache for --files and --archives · 098e4ca9
      Kishor Patil authored
      ## What changes were proposed in this pull request?
      
      During spark-submit, if yarn dist cache is instructed to add same file under --files and --archives, This code change ensures the spark yarn distributed cache behaviour is retained i.e. to warn and fail if same files is mentioned in both --files and --archives.
      ## How was this patch tested?
      
      Manually tested:
      1. if same jar is mentioned in --jars and --files it will continue to submit the job.
      - basically functionality [SPARK-14423] #12203 is unchanged
        1. if same file is mentioned in --files and --archives it will fail to submit the job.
      
      Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.
      
      … under archives and files
      
      Author: Kishor Patil <kpatil@yahoo-inc.com>
      
      Closes #15627 from kishorvpatil/spark18099.
      098e4ca9
    • 福星's avatar
      [SPARK-18237][HIVE] hive.exec.stagingdir have no effect · 16293311
      福星 authored
      hive.exec.stagingdir have no effect in spark2.0.1,
      Hive confs in hive-site.xml will be loaded in `hadoopConf`, so we should use `hadoopConf` in `InsertIntoHiveTable` instead of `SessionState.conf`
      
      Author: 福星 <fuxing@wacai.com>
      
      Closes #15744 from ClassNotFoundExp/master.
      16293311
    • Reynold Xin's avatar
      [SPARK-18244][SQL] Rename partitionProviderIsHive -> tracksPartitionsInCatalog · b17057c0
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.
      
      ## How was this patch tested?
      Should be covered by existing tests.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #15750 from rxin/SPARK-18244.
      b17057c0
Loading