- Apr 25, 2017
-
-
Sameer Agarwal authored
## What changes were proposed in this pull request? In `randomSplit`, It is possible that the underlying dataset doesn't guarantee the ordering of rows in its constituent partitions each time a split is materialized which could result in overlapping splits. To prevent this, as part of SPARK-12662, we explicitly sort each input partition to make the ordering deterministic. Given that `MapTypes` cannot be sorted this patch explicitly prunes them out from the sort order. Additionally, if the resulting sort order is empty, this patch then materializes the dataset to guarantee determinism. ## How was this patch tested? Extended `randomSplit on reordered partitions` in `DataFrameStatSuite` to also test for dataframes with mapTypes nested mapTypes. Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #17751 from sameeragarwal/randomsplit2.
-
- Apr 24, 2017
-
-
Josh Rosen authored
This patch bumps the master branch version to `2.3.0-SNAPSHOT`. Author: Josh Rosen <joshrosen@databricks.com> Closes #17753 from JoshRosen/SPARK-20453.
-
Xiao Li authored
### What changes were proposed in this pull request? `spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType) ### How was this patch tested? Added a test case Author: Xiao Li <gatorsmile@gmail.com> Closes #17730 from gatorsmile/listTables.
-
- Apr 22, 2017
-
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr initialised `RangeExec` parameters in a driver side. In the current master, a query below throws `NullPointerException`; ``` sql("SET spark.sql.codegen.wholeStage=false") sql("SELECT * FROM range(1)").show 17/04/20 17:11:05 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.NullPointerException at org.apache.spark.sql.execution.SparkPlan.sparkContext(SparkPlan.scala:54) at org.apache.spark.sql.execution.RangeExec.numSlices(basicPhysicalOperators.scala:343) at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:506) at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:505) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:320) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) ``` ## How was this patch tested? Added a test in `DataFrameRangeSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17717 from maropu/SPARK-20430.
-
- Apr 21, 2017
-
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR avoids an exception in the case where `scala.math.BigInt` has a value that does not fit into long value range (e.g. `Long.MAX_VALUE+1`). When we run the following code by using the current Spark, the following exception is thrown. This PR keeps the value using `BigDecimal` if we detect such an overflow case by catching `ArithmeticException`. Sample program: ``` case class BigIntWrapper(value:scala.math.BigInt)``` spark.createDataset(BigIntWrapper(scala.math.BigInt("10000000000000000002"))::Nil).show ``` Exception: ``` Error while encoding: java.lang.ArithmeticException: BigInteger out of long range staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0 java.lang.RuntimeException: Error while encoding: java.lang.ArithmeticException: BigInteger out of long range staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0 at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:290) at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454) at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.SparkSession.createDataset(SparkSession.scala:454) at org.apache.spark.sql.Agg$$anonfun$18.apply$mcV$sp(MySuite.scala:192) at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192) at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) ... Caused by: java.lang.ArithmeticException: BigInteger out of long range at java.math.BigInteger.longValueExact(BigInteger.java:4531) at org.apache.spark.sql.types.Decimal.set(Decimal.scala:140) at org.apache.spark.sql.types.Decimal$.apply(Decimal.scala:434) at org.apache.spark.sql.types.Decimal.apply(Decimal.scala) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source) at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:287) ... 59 more ``` ## How was this patch tested? Add new test suite into `DecimalSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17684 from kiszk/SPARK-20341.
-
Juliusz Sompolski authored
[SPARK-20412] Throw ParseException from visitNonOptionalPartitionSpec instead of returning null values. ## What changes were proposed in this pull request? If a partitionSpec is supposed to not contain optional values, a ParseException should be thrown, and not nulls returned. The nulls can later cause NullPointerExceptions in places not expecting them. ## How was this patch tested? A query like "SHOW PARTITIONS tbl PARTITION(col1='val1', col2)" used to throw a NullPointerException. Now it throws a ParseException. Author: Juliusz Sompolski <julek@databricks.com> Closes #17707 from juliuszsompolski/SPARK-20412.
-
Herman van Hovell authored
## What changes were proposed in this pull request? It is often useful to be able to track changes to the `ExternalCatalog`. This PR makes the `ExternalCatalog` emit events when a catalog object is changed. Events are fired before and after the change. The following events are fired per object: - Database - CreateDatabasePreEvent: event fired before the database is created. - CreateDatabaseEvent: event fired after the database has been created. - DropDatabasePreEvent: event fired before the database is dropped. - DropDatabaseEvent: event fired after the database has been dropped. - Table - CreateTablePreEvent: event fired before the table is created. - CreateTableEvent: event fired after the table has been created. - RenameTablePreEvent: event fired before the table is renamed. - RenameTableEvent: event fired after the table has been renamed. - DropTablePreEvent: event fired before the table is dropped. - DropTableEvent: event fired after the table has been dropped. - Function - CreateFunctionPreEvent: event fired before the function is created. - CreateFunctionEvent: event fired after the function has been created. - RenameFunctionPreEvent: event fired before the function is renamed. - RenameFunctionEvent: event fired after the function has been renamed. - DropFunctionPreEvent: event fired before the function is dropped. - DropFunctionPreEvent: event fired after the function has been dropped. The current events currently only contain the names of the object modified. We add more events, and more details at a later point. A user can monitor changes to the external catalog by adding a listener to the Spark listener bus checking for `ExternalCatalogEvent`s using the `SparkListener.onOtherEvent` hook. A more direct approach is add listener directly to the `ExternalCatalog`. ## How was this patch tested? Added the `ExternalCatalogEventSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17710 from hvanhovell/SPARK-20420.
-
- Apr 20, 2017
-
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? This pr modified code to print the identical `Range` parameters of SparkContext APIs and SQL in `explain` output. In the current master, they internally use `defaultParallelism` for `splits` by default though, they print different strings in explain output; ``` scala> spark.range(4).explain == Physical Plan == *Range (0, 4, step=1, splits=Some(8)) scala> sql("select * from range(4)").explain == Physical Plan == *Range (0, 4, step=1, splits=None) ``` ## How was this patch tested? Added tests in `SQLQuerySuite` and modified some results in the existing tests. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17670 from maropu/SPARK-20281.
-
Herman van Hovell authored
## What changes were proposed in this pull request? A cast expression with a resolved time zone is not equal to a cast expression without a resolved time zone. The `ResolveAggregateFunction` assumed that these expression were the same, and would fail to resolve `HAVING` clauses which contain a `Cast` expression. This is in essence caused by the fact that a `TimeZoneAwareExpression` can be resolved without a set time zone. This PR fixes this, and makes a `TimeZoneAwareExpression` unresolved as long as it has no TimeZone set. ## How was this patch tested? Added a regression test to the `SQLQueryTestSuite.having` file. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17641 from hvanhovell/SPARK-20329.
-
Juliusz Sompolski authored
## What changes were proposed in this pull request? When infering partitioning schema from paths, the column in parsePartitionColumn should be unescaped with unescapePathName, just like it is being done in e.g. parsePathFragmentAsSeq. ## How was this patch tested? Added a test to FileIndexSuite. Author: Juliusz Sompolski <julek@databricks.com> Closes #17703 from juliuszsompolski/SPARK-20367.
-
Herman van Hovell authored
## What changes were proposed in this pull request? It is kind of annoying that `SharedSQLContext.sparkConf` is a val when overriding test cases, because you cannot call `super` on it. This PR makes it a function. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17705 from hvanhovell/SPARK-20410.
-
Dilip Biswal authored
[SPARK-20334][SQL] Return a better error message when correlated predicates contain aggregate expression that has mixture of outer and local references. ## What changes were proposed in this pull request? Address a follow up in [comment](https://github.com/apache/spark/pull/16954#discussion_r105718880) Currently subqueries with correlated predicates containing aggregate expression having mixture of outer references and local references generate a codegen error like following : ```SQL SELECT t1a FROM t1 GROUP BY 1 HAVING EXISTS (SELECT 1 FROM t2 WHERE t2a < min(t1a + t2a)); ``` Exception snippet. ``` Cannot evaluate expression: min((input[0, int, false] + input[4, int, false])) at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:226) at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:106) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:103) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:103) ``` After this PR, a better error message is issued. ``` org.apache.spark.sql.AnalysisException Error in query: Found an aggregate expression in a correlated predicate that has both outer and local references, which is not supported yet. Aggregate expression: min((t1.`t1a` + t2.`t2a`)), Outer references: t1.`t1a`, Local references: t2.`t2a`.; ``` ## How was this patch tested? Added tests in SQLQueryTestSuite. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #17636 from dilipbiswal/subquery_followup1.
-
Bogdan Raducanu authored
## What changes were proposed in this pull request? SharedSQLContext.afterEach now calls DebugFilesystem.assertNoOpenStreams inside eventually. SQLTestUtils withTempDir calls waitForTasksToFinish before deleting the directory. ## How was this patch tested? Added new test in ParquetQuerySuite based on the flaky test Author: Bogdan Raducanu <bogdan@databricks.com> Closes #17701 from bogdanrdc/SPARK-20407.
-
Wenchen Fan authored
## What changes were proposed in this pull request? It's illegal to have aggregate function in GROUP BY, and we should fail at analysis phase, if this happens. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17704 from cloud-fan/minor.
-
Reynold Xin authored
## What changes were proposed in this pull request? Dataset.withNewExecutionId is only used in Dataset itself and should be private. ## How was this patch tested? N/A - this is a simple visibility change. Author: Reynold Xin <rxin@databricks.com> Closes #17699 from rxin/SPARK-20405.
-
Xiao Li authored
[SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs ### What changes were proposed in this pull request? Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _. When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). ### How was this patch tested? Added a test case Author: Xiao Li <gatorsmile@gmail.com> Closes #17655 from gatorsmile/locale.
-
- Apr 19, 2017
-
-
Eric Liang authored
## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-19820 adds a reason field for why tasks were killed. However, for backwards compatibility it left the old TaskKilledException constructor which defaults to "unknown reason". The range() operator should use the constructor that fills in the reason rather than dropping it on task kill. ## How was this patch tested? Existing tests, and I tested this manually. Author: Eric Liang <ekl@databricks.com> Closes #17692 from ericl/fix-kill-reason-in-range.
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Also went through the same file to ensure other string concatenation are correct. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17691 from zsxwing/fix-error-message.
-
ptkool authored
## What changes were proposed in this pull request? Apply Complementation Laws during boolean expression simplification. ## How was this patch tested? Tested using unit tests, integration tests, and manual tests. Author: ptkool <michael.styles@shopify.com> Author: Michael Styles <michael.styles@shopify.com> Closes #17650 from ptkool/apply_complementation_laws.
-
Liang-Chi Hsieh authored
[SPARK-20356][SQL] Pruned InMemoryTableScanExec should have correct output partitioning and ordering ## What changes were proposed in this pull request? The output of `InMemoryTableScanExec` can be pruned and mismatch with `InMemoryRelation` and its child plan's output. This causes wrong output partitioning and ordering. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #17679 from viirya/SPARK-20356.
-
Koert Kuipers authored
[SPARK-20359][SQL] Avoid unnecessary execution in EliminateOuterJoin optimization that can lead to NPE Avoid necessary execution that can lead to NPE in EliminateOuterJoin and add test in DataFrameSuite to confirm NPE is no longer thrown ## What changes were proposed in this pull request? Change leftHasNonNullPredicate and rightHasNonNullPredicate to lazy so they are only executed when needed. ## How was this patch tested? Added test in DataFrameSuite that failed before this fix and now succeeds. Note that a test in catalyst project would be better but i am unsure how to do this. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Koert Kuipers <koert@tresata.com> Closes #17660 from koertkuipers/feat-catch-npe-in-eliminate-outer-join.
-
- Apr 18, 2017
-
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR elminates unnecessary data conversion, which is introduced by SPARK-19716, for Dataset with primitve array in the generated Java code. When we run the following example program, now we get the Java code "Without this PR". In this code, lines 56-82 are unnecessary since the primitive array in ArrayData can be converted into Java primitive array by using ``toDoubleArray()`` method. ``GenericArrayData`` is not required. ```java val ds = sparkContext.parallelize(Seq(Array(1.1, 2.2)), 1).toDS.cache ds.count ds.map(e => e).show ``` Without this PR ``` == Parsed Logical Plan == 'SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25] +- 'MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D +- 'DeserializeToObject unresolveddeserializer(unresolvedmapobjects(<function1>, getcolumnbyordinal(0, ArrayType(DoubleType,false)), None).toDoubleArray), obj#23: [D +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2] +- ExternalRDD [obj#1] == Analyzed Logical Plan == value: array<double> SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25] +- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2] +- ExternalRDD [obj#1] == Optimized Logical Plan == SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25] +- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2] +- Scan ExternalRDDScan[obj#1] == Physical Plan == *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25] +- *MapElements <function1>, obj#24: [D +- *DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D +- InMemoryTableScan [value#2] +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas) +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2] +- Scan ExternalRDDScan[obj#1] ``` ```java /* 050 */ protected void processNext() throws java.io.IOException { /* 051 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 052 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 053 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 054 */ ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0)); /* 055 */ /* 056 */ ArrayData deserializetoobject_value1 = null; /* 057 */ /* 058 */ if (!inputadapter_isNull) { /* 059 */ int deserializetoobject_dataLength = inputadapter_value.numElements(); /* 060 */ /* 061 */ Double[] deserializetoobject_convertedArray = null; /* 062 */ deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength]; /* 063 */ /* 064 */ int deserializetoobject_loopIndex = 0; /* 065 */ while (deserializetoobject_loopIndex < deserializetoobject_dataLength) { /* 066 */ MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex)); /* 067 */ MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex); /* 068 */ /* 069 */ if (MapObjects_loopIsNull2) { /* 070 */ throw new RuntimeException(((java.lang.String) references[0])); /* 071 */ } /* 072 */ if (false) { /* 073 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null; /* 074 */ } else { /* 075 */ deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2; /* 076 */ } /* 077 */ /* 078 */ deserializetoobject_loopIndex += 1; /* 079 */ } /* 080 */ /* 081 */ deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/ /* 082 */ } /* 083 */ boolean deserializetoobject_isNull = true; /* 084 */ double[] deserializetoobject_value = null; /* 085 */ if (!inputadapter_isNull) { /* 086 */ deserializetoobject_isNull = false; /* 087 */ if (!deserializetoobject_isNull) { /* 088 */ Object deserializetoobject_funcResult = null; /* 089 */ deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray(); /* 090 */ if (deserializetoobject_funcResult == null) { /* 091 */ deserializetoobject_isNull = true; /* 092 */ } else { /* 093 */ deserializetoobject_value = (double[]) deserializetoobject_funcResult; /* 094 */ } /* 095 */ /* 096 */ } /* 097 */ deserializetoobject_isNull = deserializetoobject_value == null; /* 098 */ } /* 099 */ /* 100 */ boolean mapelements_isNull = true; /* 101 */ double[] mapelements_value = null; /* 102 */ if (!false) { /* 103 */ mapelements_resultIsNull = false; /* 104 */ /* 105 */ if (!mapelements_resultIsNull) { /* 106 */ mapelements_resultIsNull = deserializetoobject_isNull; /* 107 */ mapelements_argValue = deserializetoobject_value; /* 108 */ } /* 109 */ /* 110 */ mapelements_isNull = mapelements_resultIsNull; /* 111 */ if (!mapelements_isNull) { /* 112 */ Object mapelements_funcResult = null; /* 113 */ mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue); /* 114 */ if (mapelements_funcResult == null) { /* 115 */ mapelements_isNull = true; /* 116 */ } else { /* 117 */ mapelements_value = (double[]) mapelements_funcResult; /* 118 */ } /* 119 */ /* 120 */ } /* 121 */ mapelements_isNull = mapelements_value == null; /* 122 */ } /* 123 */ /* 124 */ serializefromobject_resultIsNull = false; /* 125 */ /* 126 */ if (!serializefromobject_resultIsNull) { /* 127 */ serializefromobject_resultIsNull = mapelements_isNull; /* 128 */ serializefromobject_argValue = mapelements_value; /* 129 */ } /* 130 */ /* 131 */ boolean serializefromobject_isNull = serializefromobject_resultIsNull; /* 132 */ final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue); /* 133 */ serializefromobject_isNull = serializefromobject_value == null; /* 134 */ serializefromobject_holder.reset(); /* 135 */ /* 136 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 137 */ /* 138 */ if (serializefromobject_isNull) { /* 139 */ serializefromobject_rowWriter.setNullAt(0); /* 140 */ } else { /* 141 */ // Remember the current cursor so that we can calculate how many bytes are /* 142 */ // written later. /* 143 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 144 */ /* 145 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 146 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 147 */ // grow the global buffer before writing data. /* 148 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 149 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 150 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 151 */ /* 152 */ } else { /* 153 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 154 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8); /* 155 */ /* 156 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 157 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 158 */ serializefromobject_arrayWriter.setNullDouble(serializefromobject_index); /* 159 */ } else { /* 160 */ final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index); /* 161 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 162 */ } /* 163 */ } /* 164 */ } /* 165 */ /* 166 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 167 */ } /* 168 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 169 */ append(serializefromobject_result); /* 170 */ if (shouldStop()) return; /* 171 */ } /* 172 */ } ``` With this PR (eliminated lines 56-62 in the above code) ```java /* 047 */ protected void processNext() throws java.io.IOException { /* 048 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 049 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 050 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 051 */ ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0)); /* 052 */ /* 053 */ boolean deserializetoobject_isNull = true; /* 054 */ double[] deserializetoobject_value = null; /* 055 */ if (!inputadapter_isNull) { /* 056 */ deserializetoobject_isNull = false; /* 057 */ if (!deserializetoobject_isNull) { /* 058 */ Object deserializetoobject_funcResult = null; /* 059 */ deserializetoobject_funcResult = inputadapter_value.toDoubleArray(); /* 060 */ if (deserializetoobject_funcResult == null) { /* 061 */ deserializetoobject_isNull = true; /* 062 */ } else { /* 063 */ deserializetoobject_value = (double[]) deserializetoobject_funcResult; /* 064 */ } /* 065 */ /* 066 */ } /* 067 */ deserializetoobject_isNull = deserializetoobject_value == null; /* 068 */ } /* 069 */ /* 070 */ boolean mapelements_isNull = true; /* 071 */ double[] mapelements_value = null; /* 072 */ if (!false) { /* 073 */ mapelements_resultIsNull = false; /* 074 */ /* 075 */ if (!mapelements_resultIsNull) { /* 076 */ mapelements_resultIsNull = deserializetoobject_isNull; /* 077 */ mapelements_argValue = deserializetoobject_value; /* 078 */ } /* 079 */ /* 080 */ mapelements_isNull = mapelements_resultIsNull; /* 081 */ if (!mapelements_isNull) { /* 082 */ Object mapelements_funcResult = null; /* 083 */ mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue); /* 084 */ if (mapelements_funcResult == null) { /* 085 */ mapelements_isNull = true; /* 086 */ } else { /* 087 */ mapelements_value = (double[]) mapelements_funcResult; /* 088 */ } /* 089 */ /* 090 */ } /* 091 */ mapelements_isNull = mapelements_value == null; /* 092 */ } /* 093 */ /* 094 */ serializefromobject_resultIsNull = false; /* 095 */ /* 096 */ if (!serializefromobject_resultIsNull) { /* 097 */ serializefromobject_resultIsNull = mapelements_isNull; /* 098 */ serializefromobject_argValue = mapelements_value; /* 099 */ } /* 100 */ /* 101 */ boolean serializefromobject_isNull = serializefromobject_resultIsNull; /* 102 */ final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue); /* 103 */ serializefromobject_isNull = serializefromobject_value == null; /* 104 */ serializefromobject_holder.reset(); /* 105 */ /* 106 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 107 */ /* 108 */ if (serializefromobject_isNull) { /* 109 */ serializefromobject_rowWriter.setNullAt(0); /* 110 */ } else { /* 111 */ // Remember the current cursor so that we can calculate how many bytes are /* 112 */ // written later. /* 113 */ final int serializefromobject_tmpCursor = serializefromobject_holder.cursor; /* 114 */ /* 115 */ if (serializefromobject_value instanceof UnsafeArrayData) { /* 116 */ final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes(); /* 117 */ // grow the global buffer before writing data. /* 118 */ serializefromobject_holder.grow(serializefromobject_sizeInBytes); /* 119 */ ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor); /* 120 */ serializefromobject_holder.cursor += serializefromobject_sizeInBytes; /* 121 */ /* 122 */ } else { /* 123 */ final int serializefromobject_numElements = serializefromobject_value.numElements(); /* 124 */ serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8); /* 125 */ /* 126 */ for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) { /* 127 */ if (serializefromobject_value.isNullAt(serializefromobject_index)) { /* 128 */ serializefromobject_arrayWriter.setNullDouble(serializefromobject_index); /* 129 */ } else { /* 130 */ final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index); /* 131 */ serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element); /* 132 */ } /* 133 */ } /* 134 */ } /* 135 */ /* 136 */ serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor); /* 137 */ } /* 138 */ serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize()); /* 139 */ append(serializefromobject_result); /* 140 */ if (shouldStop()) return; /* 141 */ } /* 142 */ } ``` ## How was this patch tested? Add test suites into `DatasetPrimitiveSuite` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17568 from kiszk/SPARK-20254.
-
wangzhenhua authored
## What changes were proposed in this pull request? If a plan has multi-level successive joins, e.g.: ``` Join / \ Union t5 / \ Join t4 / \ Join t3 / \ t1 t2 ``` Currently we fail to reorder the inside joins, i.e. t1, t2, t3. In join reorder, we use `OrderedJoin` to indicate a join has been ordered, such that when transforming down the plan, these joins don't need to be rerodered again. But there's a problem in the definition of `OrderedJoin`: The real join node is a parameter, but not a child. This breaks the transform procedure because `mapChildren` applies transform function on parameters which should be children. In this patch, we change `OrderedJoin` to a class having the same structure as a join node. ## How was this patch tested? Add a corresponding test case. Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17668 from wzhfy/recursiveReorder.
-
Felix Cheung authored
## What changes were proposed in this pull request? fix typo ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17663 from felixcheung/likedoctypo.
-
- Apr 17, 2017
-
-
Jacek Laskowski authored
## What changes were proposed in this pull request? Replace non-existent `repartitionBy` with `distribute` in `CollapseRepartitionSuite`. ## How was this patch tested? local build and `catalyst/testOnly *CollapseRepartitionSuite` Author: Jacek Laskowski <jacek@japila.pl> Closes #17657 from jaceklaskowski/CollapseRepartitionSuite.
-
Jakob Odersky authored
## What changes were proposed in this pull request? This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping. A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead. --- Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features. | RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity | | --- | --- | --- | --- | | [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no | | [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes | | [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes | | [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no | | [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes | | [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes | | Current Spark | _, % | \ | yes | [1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket. The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed. 1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`. PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern. According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid. _Proposed new behaviour in Spark: throw AnalysisException_ 2. [x] Empty input, e.g. `'' like ''` Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule. 3. [x] Escape before a non-special character, e.g. `'a' like '\a'`. Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same. According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign. _Proposed new behaviour in Spark: throw AnalysisException_ The current specification is also described in the operator's source code in this patch. ## How was this patch tested? Extra case in regex unit tests. Author: Jakob Odersky <jakob@odersky.com> This patch had conflicts when merged, resolved by Committer: Reynold Xin <rxin@databricks.com> Closes #15398 from jodersky/SPARK-17647.
-
Xiao Li authored
### What changes were proposed in this pull request? The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it. It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names. ### How was this patch tested? Added test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17646 from gatorsmile/showFunctions.
-
- Apr 15, 2017
-
-
Xiao Li authored
### What changes were proposed in this pull request? ```JAVA /** * Certain optimizations should not be applied if UDF is not deterministic. * Deterministic UDF returns same result each time it is invoked with a * particular input. This determinism just needs to hold within the context of * a query. * * return true if the UDF is deterministic */ boolean deterministic() default true; ``` Based on the definition of [UDFType](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java#L42-L50), when Hive UDF's children are non-deterministic, Hive UDF is also non-deterministic. ### How was this patch tested? Added test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17635 from gatorsmile/udfDeterministic.
-
Wenchen Fan authored
## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable. Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #17639 from cloud-fan/minor.
-
ouyangxiaochen authored
## What changes were proposed in this pull request? val and var should strictly follow the Scala syntax ## How was this patch tested? manual test and exisiting test cases Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn> Closes #17628 from ouyangxiaochen/spark-413.
-
- Apr 14, 2017
-
-
wangzhenhua authored
## What changes were proposed in this pull request? Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation. To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore. This pr also fixes a bug for boolean type in `IN` condition. ## How was this patch tested? The changes for ColumnStat are covered by existing tests. For bug fix, a new test for boolean type in IN condition is added Author: wangzhenhua <wangzhenhua@huawei.com> Closes #17630 from wzhfy/refactorColumnStat.
-
- Apr 13, 2017
-
-
Steve Loughran authored
[SPARK-20038][SQL] FileFormatWriter.ExecuteWriteTask.releaseResources() implementations to be re-entrant ## What changes were proposed in this pull request? have the`FileFormatWriter.ExecuteWriteTask.releaseResources()` implementations set `currentWriter=null` in a finally clause. This guarantees that if the first call to `currentWriter()` throws an exception, the second releaseResources() call made during the task cancel process will not trigger a second attempt to close the stream. ## How was this patch tested? Tricky. I've been fixing the underlying cause when I saw the problem [HADOOP-14204](https://issues.apache.org/jira/browse/HADOOP-14204), but SPARK-10109 shows I'm not the first to have seen this. I can't replicate it locally any more, my code no longer being broken. code review, however, should be straightforward Author: Steve Loughran <stevel@hortonworks.com> Closes #17364 from steveloughran/stevel/SPARK-20038-close.
-
Ioana Delaney authored
## What changes were proposed in this pull request? Implements star-join filter to reduce the search space for dynamic programming join enumeration. Consider the following join graph: ``` T1 D1 - T2 - T3 \ / F1 | D2 star-join: {F1, D1, D2} non-star: {T1, T2, T3} ``` The following join combinations will be generated: ``` level 0: (F1), (D1), (D2), (T1), (T2), (T3) level 1: {F1, D1}, {F1, D2}, {T2, T3} level 2: {F1, D1, D2} level 3: {F1, D1, D2, T1}, {F1, D1, D2, T2} level 4: {F1, D1, D2, T1, T2}, {F1, D1, D2, T2, T3 } level 6: {F1, D1, D2, T1, T2, T3} ``` ## How was this patch tested? New test suite ```StarJOinCostBasedReorderSuite.scala```. Author: Ioana Delaney <ioanamdelaney@gmail.com> Closes #17546 from ioana-delaney/starSchemaCBOv3.
-
- Apr 12, 2017
-
-
Burak Yavuz authored
## What changes were proposed in this pull request? Some Structured Streaming tests show flakiness such as: ``` [info] - prune results by current_date, complete mode - 696 *** FAILED *** (10 seconds, 937 milliseconds) [info] Timed out while stopping and waiting for microbatchthread to terminate.: The code passed to failAfter did not complete within 10 seconds. ``` This happens when we wait for the stream to stop, but it doesn't. The reason it doesn't stop is that we interrupt the microBatchThread, but Hadoop's `Shell.runCommand` swallows the interrupt exception, and the exception is not propagated upstream to the microBatchThread. Then this thread continues to run, only to start blocking on the `streamManualClock`. ## How was this patch tested? Thousand retries locally and [Jenkins](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75720/testReport) of the flaky tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #17613 from brkyvz/flaky-stream-agg.
-
Reynold Xin authored
## What changes were proposed in this pull request? AssertNotNull's toString/simpleString dumps the entire walkedTypePath. walkedTypePath is used for error message reporting and shouldn't be part of the output. ## How was this patch tested? Manually tested. Author: Reynold Xin <rxin@databricks.com> Closes #17616 from rxin/SPARK-20304.
-
Xiao Li authored
### What changes were proposed in this pull request? Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere. In the next PRs, the remaining Function-related APIs also need cleanups. ### How was this patch tested? Existing test cases. Author: Xiao Li <gatorsmile@gmail.com> Closes #17615 from gatorsmile/cleanupCreateTempFunction.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable. There are several problems with it: - It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?". - > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up. (see joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627)) To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above. There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013 Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings. ## How was this patch tested? Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`. This was tested via manually adding `time.time()` as below: ```diff profiles_and_goals = build_profiles + sbt_goals print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ", " ".join(profiles_and_goals)) + import time + st = time.time() exec_sbt(profiles_and_goals) + print("Elapsed :[%s]" % str(time.time() - st)) ``` produces ``` ... ======================================================================== Building Unidoc API Documentation ======================================================================== ... [info] Main Java API documentation successful. ... Elapsed :[94.8746569157] ... Author: hyukjinkwon <gurwls223@gmail.com> Closes #17477 from HyukjinKwon/SPARK-18692.
-
jtoka authored
## What changes were proposed in this pull request? Update count distinct error message for streaming datasets/dataframes to match current behavior. These aggregations are not yet supported, regardless of whether the dataset/dataframe is aggregated. Author: jtoka <jason.tokayer@gmail.com> Closes #17609 from jtoka/master.
-
Reynold Xin authored
## What changes were proposed in this pull request? When we perform a cast expression and the from and to types are structurally the same (having the same structure but different field names), we should be able to skip the actual cast. ## How was this patch tested? Added unit tests for the newly introduced functions. Author: Reynold Xin <rxin@databricks.com> Closes #17614 from rxin/SPARK-20302.
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes corrections related to JSON APIs as below: - Rendering links in Python documentation - Replacing `RDD` to `Dataset` in programing guide - Adding missing description about JSON Lines consistently in `DataFrameReader.json` in Python API - De-duplicating little bit of `DataFrameReader.json` in Scala/Java API ## How was this patch tested? Manually build the documentation via `jekyll build`. Corresponding snapstops will be left on the codes. Note that currently there are Javadoc8 breaks in several places. These are proposed to be handled in https://github.com/apache/spark/pull/17477. So, this PR does not fix those. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17602 from HyukjinKwon/minor-json-documentation.
-