- Mar 28, 2017
-
-
Patrick Wendell authored
-
sureshthalamati authored
## What changes were proposed in this pull request? JDBC read is failing with NPE due to missing null value check for array data type if the source table has null values in the array type column. For null values Resultset.getArray() returns null. This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object ## How was this patch tested? Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop. Author: sureshthalamati <suresh.thalamati@gmail.com> Closes #17460 from sureshthalamati/jdbc_array_null_fix_spark_2.1-SPARK-14536.
-
Wenchen Fan authored
When we build the deserializer expression for map type, we will use `StaticInvoke` to call `ArrayBasedMapData.toScalaMap`, and declare the return type as `scala.collection.immutable.Map`. If the map is inside an Option, we will wrap this `StaticInvoke` with `WrapOption`, which requires the input to be `scala.collect.Map`. Ideally this should be fine, as `scala.collection.immutable.Map` extends `scala.collect.Map`, but our `ObjectType` is too strict about this, this PR fixes it. new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17454 from cloud-fan/map. (cherry picked from commit d4fac410) Signed-off-by:
Cheng Lian <lian@databricks.com>
-
jerryshao authored
[SPARK-19995][YARN] Register tokens to current UGI to avoid re-issuing of tokens in yarn client mode ## What changes were proposed in this pull request? In the current Spark on YARN code, we will obtain tokens from provided services, but we're not going to add these tokens to the current user's credentials. This will make all the following operations to these services still require TGT rather than delegation tokens. This is unnecessary since we already got the tokens, also this will lead to failure in user impersonation scenario, because the TGT is granted by real user, not proxy user. So here changing to put all the tokens to the current UGI, so that following operations to these services will honor tokens rather than TGT, and this will further handle the proxy user issue mentioned above. ## How was this patch tested? Local verified in secure cluster. vanzin tgravescs mridulm dongjoon-hyun please help to review, thanks a lot. Author: jerryshao <sshao@hortonworks.com> Closes #17335 from jerryshao/SPARK-19995. (cherry picked from commit 17eddb35) Signed-off-by:
Marcelo Vanzin <vanzin@cloudera.com>
-
- Mar 27, 2017
-
-
Josh Rosen authored
## What changes were proposed in this pull request? The master snapshot publisher builds are currently broken due to two minor build issues: 1. For unknown reasons, the LFTP `mkdir -p` command began throwing errors when the remote directory already exists. This change of behavior might have been caused by configuration changes in the ASF's SFTP server, but I'm not entirely sure of that. To work around this problem, this patch updates the script to ignore errors from the `lftp mkdir -p` commands. 2. The PySpark `setup.py` file references a non-existent `pyspark.ml.stat` module, causing Python packaging to fail by complaining about a missing directory. The fix is to simply drop that line from the setup script. ## How was this patch tested? The LFTP fix was tested by manually running the failing commands on AMPLab Jenkins against the ASF SFTP server. The PySpark fix was tested locally. Author: Josh Rosen <joshrosen@databricks.com> Closes #17437 from JoshRosen/spark-20102. (cherry picked from commit 314cf51d) Signed-off-by:
Josh Rosen <joshrosen@databricks.com>
-
- Mar 26, 2017
-
-
Herman van Hovell authored
## What changes were proposed in this pull request? The `CollapseWindow` is currently to aggressive when collapsing adjacent windows. It also collapses windows in the which the parent produces a column that is consumed by the child; this creates an invalid window which will fail at runtime. This PR fixes this by adding a check for dependent adjacent windows to the `CollapseWindow` rule. ## How was this patch tested? Added a new test case to `CollapseWindowSuite` Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17432 from hvanhovell/SPARK-20086. (cherry picked from commit 617ab644) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
- Mar 25, 2017
-
-
Carson Wang authored
[SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the execution when merging all accumulator updates N.B. This is a backport to branch-2.1 of #17009. ## What changes were proposed in this pull request? In SQLListener.getExecutionMetrics, driver accumulator updates don't belong to the execution should be ignored when merging all accumulator updates to prevent NoSuchElementException. ## How was this patch tested? Updated unit test. Author: Carson Wang <carson.wangintel.com> Author: Carson Wang <carson.wang@intel.com> Closes #17418 from mallman/spark-19674-backport_2.1.
-
- Mar 23, 2017
-
-
Kazuaki Ishizaki authored
## What changes were proposed in this pull request? This PR fixes `NullPointerException` in the generated code by Catalyst. When we run the following code, we get the following `NullPointerException`. This is because there is no null checks for `inputadapter_value` while `java.lang.Long inputadapter_value` at Line 30 may have `null`. This happen when a type of DataFrame is nullable primitive type such as `java.lang.Long` and the wholestage codegen is used. While the physical plan keeps `nullable=true` in `input[0, java.lang.Long, true].longValue`, `BoundReference.doGenCode` ignores `nullable=true`. Thus, nullcheck code will not be generated and `NullPointerException` will occur. This PR checks the nullability and correctly generates nullcheck if needed. ```java sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF.collect ``` ```java Caused by: java.lang.NullPointerException at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(generated.java:37) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:393) ... ``` Generated code without this PR ```java /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private UnsafeRow serializefromobject_result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 012 */ /* 013 */ public GeneratedIterator(Object[] references) { /* 014 */ this.references = references; /* 015 */ } /* 016 */ /* 017 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 018 */ partitionIndex = index; /* 019 */ this.inputs = inputs; /* 020 */ inputadapter_input = inputs[0]; /* 021 */ serializefromobject_result = new UnsafeRow(1); /* 022 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0); /* 023 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 024 */ /* 025 */ } /* 026 */ /* 027 */ protected void processNext() throws java.io.IOException { /* 028 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 029 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 030 */ java.lang.Long inputadapter_value = (java.lang.Long)inputadapter_row.get(0, null); /* 031 */ /* 032 */ boolean serializefromobject_isNull = true; /* 033 */ long serializefromobject_value = -1L; /* 034 */ if (!false) { /* 035 */ serializefromobject_isNull = false; /* 036 */ if (!serializefromobject_isNull) { /* 037 */ serializefromobject_value = inputadapter_value.longValue(); /* 038 */ } /* 039 */ /* 040 */ } /* 041 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 042 */ /* 043 */ if (serializefromobject_isNull) { /* 044 */ serializefromobject_rowWriter.setNullAt(0); /* 045 */ } else { /* 046 */ serializefromobject_rowWriter.write(0, serializefromobject_value); /* 047 */ } /* 048 */ append(serializefromobject_result); /* 049 */ if (shouldStop()) return; /* 050 */ } /* 051 */ } /* 052 */ } ``` Generated code with this PR ```java /* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator { /* 006 */ private Object[] references; /* 007 */ private scala.collection.Iterator[] inputs; /* 008 */ private scala.collection.Iterator inputadapter_input; /* 009 */ private UnsafeRow serializefromobject_result; /* 010 */ private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder; /* 011 */ private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter; /* 012 */ /* 013 */ public GeneratedIterator(Object[] references) { /* 014 */ this.references = references; /* 015 */ } /* 016 */ /* 017 */ public void init(int index, scala.collection.Iterator[] inputs) { /* 018 */ partitionIndex = index; /* 019 */ this.inputs = inputs; /* 020 */ inputadapter_input = inputs[0]; /* 021 */ serializefromobject_result = new UnsafeRow(1); /* 022 */ this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0); /* 023 */ this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1); /* 024 */ /* 025 */ } /* 026 */ /* 027 */ protected void processNext() throws java.io.IOException { /* 028 */ while (inputadapter_input.hasNext() && !stopEarly()) { /* 029 */ InternalRow inputadapter_row = (InternalRow) inputadapter_input.next(); /* 030 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 031 */ java.lang.Long inputadapter_value = inputadapter_isNull ? null : ((java.lang.Long)inputadapter_row.get(0, null)); /* 032 */ /* 033 */ boolean serializefromobject_isNull = true; /* 034 */ long serializefromobject_value = -1L; /* 035 */ if (!inputadapter_isNull) { /* 036 */ serializefromobject_isNull = false; /* 037 */ if (!serializefromobject_isNull) { /* 038 */ serializefromobject_value = inputadapter_value.longValue(); /* 039 */ } /* 040 */ /* 041 */ } /* 042 */ serializefromobject_rowWriter.zeroOutNullBytes(); /* 043 */ /* 044 */ if (serializefromobject_isNull) { /* 045 */ serializefromobject_rowWriter.setNullAt(0); /* 046 */ } else { /* 047 */ serializefromobject_rowWriter.write(0, serializefromobject_value); /* 048 */ } /* 049 */ append(serializefromobject_result); /* 050 */ if (shouldStop()) return; /* 051 */ } /* 052 */ } /* 053 */ } ``` ## How was this patch tested? Added new test suites in `DataFrameSuites` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #17302 from kiszk/SPARK-19959. (cherry picked from commit bb823ca4) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Dongjoon Hyun authored
[SPARK-19970][SQL][BRANCH-2.1] Table owner should be USER instead of PRINCIPAL in kerberized clusters ## What changes were proposed in this pull request? In the kerberized hadoop cluster, when Spark creates tables, the owner of tables are filled with PRINCIPAL strings instead of USER names. This is inconsistent with Hive and causes problems when using [ROLE](https://cwiki.apache.org/confluence/display/Hive/SQL+Standard+Based+Hive+Authorization) in Hive. We had better to fix this. **BEFORE** ```scala scala> sql("create table t(a int)").show scala> sql("desc formatted t").show(false) ... |Owner: |sparkEXAMPLE.COM | | ``` **AFTER** ```scala scala> sql("create table t(a int)").show scala> sql("desc formatted t").show(false) ... |Owner: |spark | | ``` ## How was this patch tested? Manually do `create table` and `desc formatted` because this happens in Kerberized clusters. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #17363 from dongjoon-hyun/SPARK-19970-2.
-
- Mar 22, 2017
-
-
uncleGen authored
## What changes were proposed in this pull request? Add backslash for line continuation in python code. ## How was this patch tested? Jenkins. Author: uncleGen <hustyugm@gmail.com> Author: dylon <hustyugm@gmail.com> Closes #17352 from uncleGen/python-example-doc. (cherry picked from commit facfd608) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
Yanbo Liang authored
## What changes were proposed in this pull request? SparkR ```spark.getSparkFiles``` fails when it was called on executors, see details at [SPARK-19925](https://issues.apache.org/jira/browse/SPARK-19925 ). ## How was this patch tested? Add unit tests, and verify this fix at standalone and yarn cluster. Author: Yanbo Liang <ybliang8@gmail.com> Closes #17274 from yanboliang/spark-19925. (cherry picked from commit 478fbc86) Signed-off-by:
Yanbo Liang <ybliang8@gmail.com>
-
- Mar 21, 2017
-
-
Patrick Wendell authored
-
Patrick Wendell authored
-
Takeshi Yamamuro authored
## What changes were proposed in this pull request? A Bean serializer in `ExpressionEncoder` could change values when Beans having NULL. A concrete example is as follows; ``` scala> :paste class Outer extends Serializable { private var cls: Inner = _ def setCls(c: Inner): Unit = cls = c def getCls(): Inner = cls } class Inner extends Serializable { private var str: String = _ def setStr(s: String): Unit = str = str def getStr(): String = str } scala> Seq("""{"cls":null}""", """{"cls": {"str":null}}""").toDF().write.text("data") scala> val encoder = Encoders.bean(classOf[Outer]) scala> val schema = encoder.schema scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder) scala> df.show +------+ | cls| +------+ |[null]| | null| +------+ scala> df.map(x => x)(encoder).show() +------+ | cls| +------+ |[null]| |[null]| // <-- Value changed +------+ ``` This is because the Bean serializer does not have the NULL-check expressions that the serializer of Scala's product types has. Actually, this value change does not happen in Scala's product types; ``` scala> :paste case class Outer(cls: Inner) case class Inner(str: String) scala> val encoder = Encoders.product[Outer] scala> val schema = encoder.schema scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder) scala> df.show +------+ | cls| +------+ |[null]| | null| +------+ scala> df.map(x => x)(encoder).show() +------+ | cls| +------+ |[null]| | null| +------+ ``` This pr added the NULL-check expressions in Bean serializer along with the serializer of Scala's product types. ## How was this patch tested? Added tests in `JavaDatasetSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17372 from maropu/SPARK-19980-BACKPORT2.1.
-
Will Manning authored
## What changes were proposed in this pull request? The description in the comment for array_contains is vague/incomplete (i.e., doesn't mention that it returns `null` if the array is `null`); this PR fixes that. ## How was this patch tested? No testing, since it merely changes a comment. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Will Manning <lwwmanning@gmail.com> Closes #17380 from lwwmanning/patch-1. (cherry picked from commit a04dcde8) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Felix Cheung authored
## What changes were proposed in this pull request? When SparkR is installed as a R package there might not be any java runtime. If it is not there SparkR's `sparkR.session()` will block waiting for the connection timeout, hanging the R IDE/shell, without any notification or message. ## How was this patch tested? manually - [x] need to test on Windows Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16596 from felixcheung/rcheckjava. (cherry picked from commit a8877bdb) Signed-off-by:
Shivaram Venkataraman <shivaram@cs.berkeley.edu>
-
zhaorongsheng authored
## What changes were proposed in this pull request? Change the nullability of function `StringToMap` from `false` to `true`. Author: zhaorongsheng <334362872@qq.com> Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE. (cherry picked from commit 7dbc162f) Signed-off-by:
Xiao Li <gatorsmile@gmail.com>
-
- Mar 20, 2017
-
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Since current `HiveShim`'s `convertFilters` does not escape the string literals. There exists the following correctness issues. This PR aims to return the correct result and also shows the more clear exception message. **BEFORE** ```scala scala> Seq((1, "p1", "q1"), (2, "p1\" and q=\"q1", "q2")).toDF("a", "p", "q").write.partitionBy("p", "q").saveAsTable("t1") scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show +---+ | a| +---+ +---+ scala> spark.table("t1").filter($"p" === "'\"").select($"a").show java.lang.RuntimeException: Caught Hive MetaException attempting to get partition metadata by filter from ... ``` **AFTER** ```scala scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show +---+ | a| +---+ | 2| +---+ scala> spark.table("t1").filter($"p" === "'\"").select($"a").show java.lang.UnsupportedOperationException: Partition filter cannot have both `"` and `'` characters ``` ## How was this patch tested? Pass the Jenkins test with new test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #17266 from dongjoon-hyun/SPARK-19912. (cherry picked from commit 21e366ae) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Michael Allman authored
(Jira: https://issues.apache.org/jira/browse/SPARK-17204 ) ## What changes were proposed in this pull request? There are a couple of bugs in the `BlockManager` with respect to support for replicated off-heap storage. First, the locally-stored off-heap byte buffer is disposed of when it is replicated. It should not be. Second, the replica byte buffers are stored as heap byte buffers instead of direct byte buffers even when the storage level memory mode is off-heap. This PR addresses both of these problems. ## How was this patch tested? `BlockManagerReplicationSuite` was enhanced to fill in the coverage gaps. It now fails if either of the bugs in this PR exist. Author: Michael Allman <michael@videoamp.com> Closes #16499 from mallman/spark-17204-replicated_off_heap_storage. (cherry picked from commit 7fa116f8) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
wangzhenhua authored
## What changes were proposed in this pull request? For right outer join, values of the left key will be filled with nulls if it can't match the value of the right key, so `nullOrdering` of the left key can't be guaranteed. We should output right key order instead of left key order. For full outer join, neither left key nor right key guarantees `nullOrdering`. We should not output any ordering. In tests, besides adding three test cases for left/right/full outer sort merge join, this patch also reorganizes code in `PlannerSuite` by putting together tests for `Sort`, and also extracts common logic in Sort tests into a method. ## How was this patch tested? Corresponding test cases are added. Author: wangzhenhua <wangzhenhua@huawei.com> Author: Zhenhua Wang <wzh_zju@163.com> Closes #17331 from wzhfy/wrongOrdering. (cherry picked from commit 965a5abc) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
- Mar 19, 2017
-
-
Felix Cheung authored
## What changes were proposed in this pull request? Passes R `tempdir()` (this is the R session temp dir, shared with other temp files/dirs) to JVM, set System.Property for derby home dir to move derby.log ## How was this patch tested? Manually, unit tests With this, these are relocated to under /tmp ``` # ls /tmp/RtmpG2M0cB/ derby.log ``` And they are removed automatically when the R session is ended. Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16330 from felixcheung/rderby. (cherry picked from commit 422aa67d) Signed-off-by:
Felix Cheung <felixcheung@apache.org>
-
- Mar 17, 2017
-
-
Jacek Laskowski authored
## What changes were proposed in this pull request? Fix scaladoc for UDFRegistration ## How was this patch tested? local build Author: Jacek Laskowski <jacek@japila.pl> Closes #17337 from jaceklaskowski/udfregistration-scaladoc. (cherry picked from commit 6326d406) Signed-off-by:
Reynold Xin <rxin@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? Sometimes, CheckpointTests will hang on a busy machine because the streaming jobs are too slow and cannot catch up. I observed the scheduled delay was keeping increasing for dozens of seconds locally. This PR increases the batch interval from 0.5 seconds to 2 seconds to generate less Spark jobs. It should make `pyspark.streaming.tests.CheckpointTests` more stable. I also replaced `sleep` with `awaitTerminationOrTimeout` so that if the streaming job fails, it will also fail the test. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17323 from zsxwing/SPARK-19986. (cherry picked from commit 376d7821) Signed-off-by:
Tathagata Das <tathagata.das1565@gmail.com>
-
Liwei Lin authored
## Problem There are several places where we write out version identifiers in various logs for structured streaming (usually `v1`). However, in the places where we check for this, we throw a confusing error message. ## What changes were proposed in this pull request? This patch made two major changes: 1. added a `parseVersion(...)` method, and based on this method, fixed the following places the way they did version checking (no other place needed to do this checking): ``` HDFSMetadataLog - CompactibleFileStreamLog ------------> fixed with this patch - FileStreamSourceLog ---------------> inherited the fix of `CompactibleFileStreamLog` - FileStreamSinkLog -----------------> inherited the fix of `CompactibleFileStreamLog` - OffsetSeqLog ------------------------> fixed with this patch - anonymous subclass in KafkaSource ---> fixed with this patch ``` 2. changed the type of `FileStreamSinkLog.VERSION`, `FileStreamSourceLog.VERSION` etc. from `String` to `Int`, so that we can identify newer versions via `version > 1` instead of `version != "v1"` - note this didn't break any backwards compatibility -- we are still writing out `"v1"` and reading back `"v1"` ## Exception message with this patch ``` java.lang.IllegalStateException: Failed to read log file /private/var/folders/nn/82rmvkk568sd8p3p8tb33trw0000gn/T/spark-86867b65-0069-4ef1-b0eb-d8bd258ff5b8/0. UnsupportedLogVersion: maximum supported log version is v1, but encountered v99. The log file was produced by a newer version of Spark and cannot be read by this version. Please upgrade. at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:202) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:78) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:75) at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:133) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite.withTempDir(OffsetSeqLogSuite.scala:26) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply$mcV$sp(OffsetSeqLogSuite.scala:75) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75) at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) ``` ## How was this patch tested? unit tests Author: Liwei Lin <lwlin7@gmail.com> Closes #17327 from lw-lin/good-msg-2.1.
-
- Mar 16, 2017
-
-
Xiao Li authored
[SPARK-19765][SPARK-18549][SPARK-19093][SPARK-19736][BACKPORT-2.1][SQL] Backport Three Cache-related PRs to Spark 2.1 ### What changes were proposed in this pull request? Backport a few cache related PRs: --- [[SPARK-19093][SQL] Cached tables are not used in SubqueryExpression](https://github.com/apache/spark/pull/16493) Consider the plans inside subquery expressions while looking up cache manager to make use of cached data. Currently CacheManager.useCachedData does not consider the subquery expressions in the plan. --- [[SPARK-19736][SQL] refreshByPath should clear all cached plans with the specified path](https://github.com/apache/spark/pull/17064) Catalog.refreshByPath can refresh the cache entry and the associated metadata for all dataframes (if any), that contain the given data source path. However, CacheManager.invalidateCachedPath doesn't clear all cached plans with the specified path. It causes some strange behaviors reported in SPARK-15678. --- [[SPARK-19765][SPARK-18549][SQL] UNCACHE TABLE should un-cache all cached plans that refer to this table](https://github.com/apache/spark/pull/17097) When un-cache a table, we should not only remove the cache entry for this table, but also un-cache any other cached plans that refer to this table. The following commands trigger the table uncache: `DropTableCommand`, `TruncateTableCommand`, `AlterTableRenameCommand`, `UncacheTableCommand`, `RefreshTable` and `InsertIntoHiveTable` This PR also includes some refactors: - use java.util.LinkedList to store the cache entries, so that it's safer to remove elements while iterating - rename invalidateCache to recacheByPlan, which is more obvious about what it does. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #17319 from gatorsmile/backport-17097.
-
windpiger authored
[SPARK-19329][SQL][BRANCH-2.1] Reading from or writing to a datasource table with a non pre-existing location should succeed ## What changes were proposed in this pull request? This is a backport pr of https://github.com/apache/spark/pull/16672 into branch-2.1. ## How was this patch tested? Existing tests. Author: windpiger <songjun@outlook.com> Closes #17317 from windpiger/backport-insertnotexists.
-
- Mar 15, 2017
-
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one. with the file, `text.txt` below: ``` a b d e f g h i j k l ``` - Before ```python >>> sc.textFile('text.txt').repartition(1).collect() ``` ``` UTF8Deserializer(True) Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/rdd.py", line 811, in collect return list(_load_from_socket(port, self._jrdd_deserializer)) File ".../spark/python/pyspark/serializers.py", line 549, in load_stream yield self.loads(stream) File ".../spark/python/pyspark/serializers.py", line 544, in loads return s.decode("utf-8") if self.use_unicode else s File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode return codecs.utf_8_decode(input, errors, True) UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte ``` - After ```python >>> sc.textFile('text.txt').repartition(1).collect() ``` ``` [u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u''] ``` ## How was this patch tested? Unit test in `python/pyspark/tests.py`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17282 from HyukjinKwon/SPARK-19872. (cherry picked from commit 7387126f) Signed-off-by:
Davies Liu <davies.liu@gmail.com>
-
Reynold Xin authored
## What changes were proposed in this pull request? This patch moves SQLConf from sql/core to sql/catalyst. To minimize the changes, the patch used type alias to still keep CatalystConf (as a type alias) and SimpleCatalystConf (as a concrete class that extends SQLConf). Motivation for the change is that it is pretty weird to have SQLConf only in sql/core and then we have to duplicate config options that impact optimizer/analyzer in sql/catalyst using CatalystConf. This is a backport into branch-2.1 to minimize merge conflicts. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #17301 from rxin/branch-2.1-conf.
-
- Mar 14, 2017
-
-
Wenchen Fan authored
When dynamic partition value is null or empty string, we should write the data to a directory like `a=__HIVE_DEFAULT_PARTITION__`, when we read the data back, we should respect this special directory name and treat it as null. This is the same behavior of impala, see https://issues.apache.org/jira/browse/IMPALA-252 new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17277 from cloud-fan/partition. (cherry picked from commit dacc382f) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Herman van Hovell authored
## What changes were proposed in this pull request? The `RemoveRedundantAlias` rule can change the output attributes (the expression id's to be precise) of a query by eliminating the redundant alias producing them. This is no problem for a regular query, but can cause problems for correlated subqueries: The attributes produced by the subquery are used in the parent plan; changing them will break the parent plan. This PR fixes this by wrapping a subquery in a `Subquery` top level node when it gets optimized. The `RemoveRedundantAlias` rule now recognizes `Subquery` and makes sure that the output attributes of the `Subquery` node are retained. ## How was this patch tested? Added a test case to `RemoveRedundantAliasAndProjectSuite` and added a regression test to `SubquerySuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17278 from hvanhovell/SPARK-19933. (cherry picked from commit e04c05cf) Signed-off-by:
Herman van Hovell <hvanhovell@databricks.com>
-
- Mar 12, 2017
-
-
uncleGen authored
When using the KafkaSource with Structured Streaming, consumer assignments are not what the user expects if startingOffsets is set to an explicit set of topics/partitions in JSON where the topic(s) happen to have uppercase characters. When StartingOffsets is constructed, the original string value from options is transformed toLowerCase to make matching on "earliest" and "latest" case insensitive. However, the toLowerCase JSON is passed to SpecificOffsets for the terminal condition, so topic names may not be what the user intended by the time assignments are made with the underlying KafkaConsumer. KafkaSourceProvider.scala: ``` val startingOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match { case Some("latest") => LatestOffsets case Some("earliest") => EarliestOffsets case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json)) case None => LatestOffsets } ``` Thank cbowden for reporting. Jenkins Author: uncleGen <hustyugm@gmail.com> Closes #17209 from uncleGen/SPARK-19853. (cherry picked from commit 0a4d06a7) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
uncleGen authored
## What changes were proposed in this pull request? - SS python example: `TypeError: 'xxx' object is not callable` - some other doc issue. ## How was this patch tested? Jenkins. Author: uncleGen <hustyugm@gmail.com> Closes #17257 from uncleGen/docs-ss-python. (cherry picked from commit e29a74d5) Signed-off-by:
Sean Owen <sowen@cloudera.com>
-
- Mar 10, 2017
-
-
Budde authored
Add a new configuration option that allows Spark SQL to infer a case-sensitive schema from a Hive Metastore table's data files when a case-sensitive schema can't be read from the table properties. - Add spark.sql.hive.caseSensitiveInferenceMode param to SQLConf - Add schemaPreservesCase field to CatalogTable (set to false when schema can't successfully be read from Hive table props) - Perform schema inference in HiveMetastoreCatalog if schemaPreservesCase is false, depending on spark.sql.hive.caseSensitiveInferenceMode - Add alterTableSchema() method to the ExternalCatalog interface - Add HiveSchemaInferenceSuite tests - Refactor and move ParquetFileForamt.meregeMetastoreParquetSchema() as HiveMetastoreCatalog.mergeWithMetastoreSchema - Move schema merging tests from ParquetSchemaSuite to HiveSchemaInferenceSuite [JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611) The tests in ```HiveSchemaInferenceSuite``` should verify that schema inference is working as expected. ```ExternalCatalogSuite``` has also been extended to cover the new ```alterTableSchema()``` API. Author: Budde <budde@amazon.com> Closes #17229 from budde/SPARK-19611-2.1.
-
Wenchen Fan authored
In spark SQL, map type can't be used in equality test/comparison, and `Intersect`/`Except`/`Distinct` do need equality test for all columns, we should not allow map type in `Intersect`/`Except`/`Distinct`. new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #17236 from cloud-fan/map. (cherry picked from commit fb9beda5) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
Tyson Condie authored
## What changes were proposed in this pull request? We need to notify the await batch lock when the stream exits early e.g., when an exception has been thrown. ## How was this patch tested? Current tests that throw exceptions at runtime will finish faster as a result of this update. zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Closes #17231 from tcondie/kafka-writer. (cherry picked from commit 501b7111) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
- Mar 09, 2017
-
-
Burak Yavuz authored
## What changes were proposed in this pull request? Fix the `throw new IllegalStateException` if statement part. ## How is this patch tested Regression test Author: Burak Yavuz <brkyvz@gmail.com> Closes #17228 from brkyvz/kafka-cause-fix. (cherry picked from commit 82138e09) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
uncleGen authored
## What changes were proposed in this pull request? `watermark` should not be negative. This behavior is invalid, check it before real run. ## How was this patch tested? add new unit test. Author: uncleGen <hustyugm@gmail.com> Author: dylon <hustyugm@gmail.com> Closes #17202 from uncleGen/SPARK-19861. (cherry picked from commit 30b18e69) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Jason White authored
## What changes were proposed in this pull request? Add handling of input of type `Int` for dataType `TimestampType` to `EvaluatePython.scala`. Py4J serializes ints smaller than MIN_INT or larger than MAX_INT to Long, which are handled correctly already, but values between MIN_INT and MAX_INT are serialized to Int. These range limits correspond to roughly half an hour on either side of the epoch. As a result, PySpark doesn't allow TimestampType values to be created in this range. Alternatives attempted: patching the `TimestampType.toInternal` function to cast return values to `long`, so Py4J would always serialize them to Scala Long. Python3 does not have a `long` type, so this approach failed on Python3. ## How was this patch tested? Added a new PySpark-side test that fails without the change. The contribution is my original work and I license the work to the project under the project’s open source license. Resubmission of https://github.com/apache/spark/pull/16896 . The original PR didn't go through Jenkins and broke the build. davies dongjoon-hyun cloud-fan Could you kick off a Jenkins run for me? It passed everything for me locally, but it's possible something has changed in the last few weeks. Author: Jason White <jason.white@shopify.com> Closes #17200 from JasonMWhite/SPARK-19561. (cherry picked from commit 206030bd) Signed-off-by:
Wenchen Fan <wenchen@databricks.com>
-
uncleGen authored
## What changes were proposed in this pull request? A follow up to SPARK-19859: - extract the calculation of `delayMs` and reuse it. - update EventTimeWatermarkExec - use the correct `delayMs` in EventTimeWatermark ## How was this patch tested? Jenkins. Author: uncleGen <hustyugm@gmail.com> Closes #17221 from uncleGen/SPARK-19859. (cherry picked from commit eeb1d6db) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-
Shixiong Zhu authored
## What changes were proposed in this pull request? The API docs should not include the "org.apache.spark.sql.internal" package because they are internal private APIs. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #17217 from zsxwing/SPARK-19874. (cherry picked from commit 029e40b4) Signed-off-by:
Shixiong Zhu <shixiong@databricks.com>
-