- Sep 07, 2017
-
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? This is a follow-up of #19050 to deal with `ExistenceJoin` case. ## How was this patch tested? Added test. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #19151 from viirya/SPARK-21835-followup.
-
- Sep 06, 2017
-
-
Jacek Laskowski authored
## What changes were proposed in this pull request? Just `StateOperatorProgress.toString` + few formatting fixes ## How was this patch tested? Local build. Waiting for OK from Jenkins. Author: Jacek Laskowski <jacek@japila.pl> Closes #19112 from jaceklaskowski/SPARK-21901-StateOperatorProgress-toString.
-
Jose Torres authored
## What changes were proposed in this pull request? Add an assert in logical plan optimization that the isStreaming bit stays the same, and fix empty relation rules where that wasn't happening. ## How was this patch tested? new and existing unit tests Author: Jose Torres <joseph.torres@databricks.com> Author: Jose Torres <joseph-torres@databricks.com> Closes #19056 from joseph-torres/SPARK-21765-followup.
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? Correlated predicate subqueries are rewritten into `Join` by the rule `RewritePredicateSubquery` during optimization. It is possibly that the two sides of the `Join` have conflicting attributes. The query plans produced by `RewritePredicateSubquery` become unresolved and break structural integrity. We should check if there are conflicting attributes in the `Join` and de-duplicate them by adding a `Project`. ## How was this patch tested? Added tests. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #19050 from viirya/SPARK-21835.
-
- Sep 05, 2017
-
-
jerryshao authored
Spark ThriftServer doesn't support spnego auth for thrift/http protocol, this mainly used for knox+thriftserver scenario. Since in HiveServer2 CLIService there already has existing codes to support it. So here copy it to Spark ThriftServer to make it support. Related Hive JIRA HIVE-6697. Manual verification. Author: jerryshao <sshao@hortonworks.com> Closes #18628 from jerryshao/SPARK-21407. Change-Id: I61ef0c09f6972bba982475084a6b0ae3a74e385e
-
Xingbo Jiang authored
## What changes were proposed in this pull request? For the given example below, the predicate added by `InferFiltersFromConstraints` is folded by `ConstantPropagation` later, this leads to unconverged optimize iteration: ``` Seq((1, 1)).toDF("col1", "col2").createOrReplaceTempView("t1") Seq(1, 2).toDF("col").createOrReplaceTempView("t2") sql("SELECT * FROM t1, t2 WHERE t1.col1 = 1 AND 1 = t1.col2 AND t1.col1 = t2.col AND t1.col2 = t2.col") ``` We can fix this by adjusting the indent of the optimize rules. ## How was this patch tested? Add test case that would have failed in `SQLQuerySuite`. Author: Xingbo Jiang <xingbo.jiang@databricks.com> Closes #19099 from jiangxb1987/unconverge-optimization.
-
gatorsmile authored
## What changes were proposed in this pull request? We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases. ## How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #19119 from gatorsmile/fallbackCodegen.
-
hyukjinkwon authored
## What changes were proposed in this pull request? There was a bug in Univocity Parser that causes the issue in SPARK-20978. This was fixed as below: ```scala val df = spark.read.schema("a string, b string, unparsed string").option("columnNameOfCorruptRecord", "unparsed").csv(Seq("a").toDS()) df.show() ``` **Before** ``` java.lang.NullPointerException at scala.collection.immutable.StringLike$class.stripLineEnd(StringLike.scala:89) at scala.collection.immutable.StringOps.stripLineEnd(StringOps.scala:29) at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$getCurrentInput(UnivocityParser.scala:56) at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert$1.apply(UnivocityParser.scala:207) at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert$1.apply(UnivocityParser.scala:207) ... ``` **After** ``` +---+----+--------+ | a| b|unparsed| +---+----+--------+ | a|null| a| +---+----+--------+ ``` It was fixed in 2.5.0 and 2.5.4 was released. I guess it'd be safe to upgrade this. ## How was this patch tested? Unit test added in `CSVSuite.scala`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19113 from HyukjinKwon/bump-up-univocity.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? Currently, `withDatabase` fails if the database is not empty. It would be great if we drop cleanly with CASCADE. ## How was this patch tested? This is a change on test util. Pass the existing Jenkins. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19125 from dongjoon-hyun/SPARK-21913.
-
- Sep 04, 2017
-
-
Sean Owen authored
[SPARK-21418][SQL] NoSuchElementException: None.get in DataSourceScanExec with sun.io.serialization.extendedDebugInfo=true ## What changes were proposed in this pull request? If no SparkConf is available to Utils.redact, simply don't redact. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #19123 from srowen/SPARK-21418.
-
- Sep 03, 2017
-
-
Liang-Chi Hsieh authored
## What changes were proposed in this pull request? SQL predicates don't have complete expression description. This patch goes to complement the description by adding arguments, examples. This change also adds related test cases for the SQL predicate expressions. ## How was this patch tested? Existing tests. And added predicate test. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #18869 from viirya/SPARK-21654.
-
- Sep 02, 2017
-
-
gatorsmile authored
## What changes were proposed in this pull request? Add `TBLPROPERTIES` to the DDL statement `CREATE TABLE USING`. After this change, the DDL becomes ``` CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name USING table_provider [OPTIONS table_property_list] [PARTITIONED BY (col_name, col_name, ...)] [CLUSTERED BY (col_name, col_name, ...) [SORTED BY (col_name [ASC|DESC], ...)] INTO num_buckets BUCKETS ] [LOCATION path] [COMMENT table_comment] [TBLPROPERTIES (property_name=property_value, ...)] [[AS] select_statement]; ``` ## How was this patch tested? Add a few tests Author: gatorsmile <gatorsmile@gmail.com> Closes #19100 from gatorsmile/addTablePropsToCreateTableUsing.
-
- Sep 01, 2017
-
-
gatorsmile authored
## What changes were proposed in this pull request? Supporting moving tables across different database in HiveClient `alterTable` ## How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #19104 from gatorsmile/alterTable.
-
Sean Owen authored
[SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation …build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure ## What changes were proposed in this pull request? This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts. In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11. It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release. - Scalatest 2.x -> 3.0.3 - Chill 0.8.0 -> 0.8.4 - Clapper 1.0.x -> 1.1.2 - json4s 3.2.x -> 3.4.2 - Jackson 2.6.x -> 2.7.9 (required by json4s) This change does _not_ fully enable a Scala 2.12 build: - It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here - It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too. What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build. ## How was this patch tested? Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above. Author: Sean Owen <sowen@cloudera.com> Closes #18645 from srowen/SPARK-14280.
-
he.qiao authored
## What changes were proposed in this pull request? As shown below, for example, When the job 5 is running, It was a mistake to think that five jobs were running, So I think it would be more appropriate to change jobs to job id.  ## How was this patch tested? no need Author: he.qiao <he.qiao17@zte.com.cn> Closes #19093 from Geek-He/08_31_sqltable.
-
- Aug 31, 2017
-
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API. In short, the following examples are allowed: ```python >>> df = spark.range(10) >>> df.sample(0.5).count() 7 >>> df.sample(fraction=0.5).count() 3 >>> df.sample(0.5, seed=42).count() 5 >>> df.sample(fraction=0.5, seed=42).count() 5 ``` In addition, this PR also adds some type checking logics as below: ```python >>> df = spark.range(10) >>> df.sample().count() ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got []. >>> df.sample(True).count() ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>]. >>> df.sample(42).count() ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>]. >>> df.sample(fraction=False, seed="a").count() ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>]. >>> df.sample(seed=[1]).count() ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>]. >>> df.sample(withReplacement="a", fraction=0.5, seed=1) ... TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>]. ``` ## How was this patch tested? Manually tested, unit tests added in doc tests and manually checked the built documentation for Python. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18999 from HyukjinKwon/SPARK-21779.
-
Andrew Ray authored
## What changes were proposed in this pull request? Allows `BinaryComparison` operators to work on any data type that actually supports ordering as verified by `TypeUtils.checkForOrderingExpr` instead of relying on the incomplete list `TypeCollection.Ordered` (which is removed by this PR). ## How was this patch tested? Updated unit tests to cover structs and arrays. Author: Andrew Ray <ray.andrew@gmail.com> Closes #18818 from aray/SPARK-21110.
-
gatorsmile authored
## What changes were proposed in this pull request? Also remove useless function `partitionByDeterministic` after the changes of https://github.com/apache/spark/pull/14687 ## How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #19097 from gatorsmile/followupSPARK-17107.
-
Bryan Cutler authored
Removing a check in the ColumnarBatchSuite that depended on a Java assertion. This assertion is being compiled out in the Maven builds causing the test to fail. This part of the test is not specifically from to the functionality that is being tested here. Author: Bryan Cutler <cutlerb@gmail.com> Closes #19098 from BryanCutler/hotfix-ColumnarBatchSuite-assertion.
-
Jacek Laskowski authored
… Dataset with LogicalRDD logical operator ## What changes were proposed in this pull request? Reusing `SparkSession.internalCreateDataFrame` wherever possible (to cut dups) ## How was this patch tested? Local build and waiting for Jenkins Author: Jacek Laskowski <jacek@japila.pl> Closes #19095 from jaceklaskowski/SPARK-21886-internalCreateDataFrame.
-
gatorsmile authored
## What changes were proposed in this pull request? Creates `SQLMetricsTestUtils` for the utility functions of both Hive-specific and the other SQLMetrics test cases. Also, move two SQLMetrics test cases from sql/hive to sql/core. ## How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #19092 from gatorsmile/rewriteSQLMetrics.
-
- Aug 30, 2017
-
-
Bryan Cutler authored
## What changes were proposed in this pull request? This PR allows the creation of a `ColumnarBatch` from `ReadOnlyColumnVectors` where previously a columnar batch could only allocate vectors internally. This is useful for using `ArrowColumnVectors` in a batch form to do row-based iteration. Also added `ArrowConverter.fromPayloadIterator` which converts `ArrowPayload` iterator to `InternalRow` iterator and uses a `ColumnarBatch` internally. ## How was this patch tested? Added a new unit test for creating a `ColumnarBatch` with `ReadOnlyColumnVectors` and a test to verify the roundtrip of rows -> ArrowPayload -> rows, using `toPayloadIterator` and `fromPayloadIterator`. Author: Bryan Cutler <cutlerb@gmail.com> Closes #18787 from BryanCutler/arrow-ColumnarBatch-support-SPARK-21583.
-
Andrew Ash authored
## What changes were proposed in this pull request? Fix Java code style so `./dev/lint-java` succeeds ## How was this patch tested? Run `./dev/lint-java` Author: Andrew Ash <andrew@andrewash.com> Closes #19088 from ash211/spark-21875-lint-java.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? This PR aims to support `spark.sql.orc.compression.codec` like Parquet's `spark.sql.parquet.compression.codec`. Users can use SQLConf to control ORC compression, too. ## How was this patch tested? Pass the Jenkins with new and updated test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19055 from dongjoon-hyun/SPARK-21839.
-
caoxuewen authored
## What changes were proposed in this pull request? igore("shuffle hash join") is to shuffle hash join to test _case class ShuffledHashJoinExec_. But when you 'ignore' -> 'test', the test is _case class BroadcastHashJoinExec_. Before modified, as a result of:canBroadcast is true. Print information in _canBroadcast(plan: LogicalPlan)_ ``` canBroadcast plan.stats.sizeInBytes:6710880 canBroadcast conf.autoBroadcastJoinThreshold:10000000 ``` After modified, plan.stats.sizeInBytes is 11184808. Print information in _canBuildLocalHashMap(plan: LogicalPlan)_ and _muchSmaller(a: LogicalPlan, b: LogicalPlan)_ : ``` canBuildLocalHashMap plan.stats.sizeInBytes:11184808 canBuildLocalHashMap conf.autoBroadcastJoinThreshold:10000000 canBuildLocalHashMap conf.numShufflePartitions:2 ``` ``` muchSmaller a.stats.sizeInBytes * 3:33554424 muchSmaller b.stats.sizeInBytes:33554432 ``` ## How was this patch tested? existing test case. Author: caoxuewen <cao.xuewen@zte.com.cn> Closes #19069 from heary-cao/shuffle_hash_join.
-
gatorsmile authored
This reverts commit 3d0e1742.
-
hyukjinkwon authored
## What changes were proposed in this pull request? `org.apache.spark.deploy.RPackageUtilsSuite` ``` - jars without manifest return false *** FAILED *** (109 milliseconds) java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1500266936418-0\dep1-c.jar ``` `org.apache.spark.deploy.SparkSubmitSuite` ``` - download one file to local *** FAILED *** (16 milliseconds) java.net.URISyntaxException: Illegal character in authority at index 6: s3a://C:\projects\spark\target\tmp\test2630198944759847458.jar - download list of files to local *** FAILED *** (0 milliseconds) java.net.URISyntaxException: Illegal character in authority at index 6: s3a://C:\projects\spark\target\tmp\test2783551769392880031.jar ``` `org.apache.spark.scheduler.ReplayListenerSuite` ``` - Replay compressed inprogress log file succeeding on partial read (156 milliseconds) Exception encountered when attempting to run a suite with class name: org.apache.spark.scheduler.ReplayListenerSuite *** ABORTED *** (1 second, 391 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-8f3cacd6-faad-4121-b901-ba1bba8025a0 - End-to-end replay *** FAILED *** (62 milliseconds) java.io.IOException: No FileSystem for scheme: C - End-to-end replay with compression *** FAILED *** (110 milliseconds) java.io.IOException: No FileSystem for scheme: C ``` `org.apache.spark.sql.hive.StatisticsSuite` ``` - SPARK-21079 - analyze table with location different than that of individual partitions *** FAILED *** (875 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-21079 - analyze partitioned table with only a subset of partitions visible *** FAILED *** (47 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` **Note:** this PR does not fix: `org.apache.spark.deploy.SparkSubmitSuite` ``` - launch simple application with spark-submit with redaction *** FAILED *** (172 milliseconds) java.util.NoSuchElementException: next on empty iterator ``` I can't reproduce this on my Windows machine but looks appearntly consistently failed on AppVeyor. This one is unclear to me yet and hard to debug so I did not include this one for now. **Note:** it looks there are more instances but it is hard to identify them partly due to flakiness and partly due to swarming logs and errors. Will probably go one more time if it is fine. ## How was this patch tested? Manually via AppVeyor: **Before** - `org.apache.spark.deploy.RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/771-windows-fix/job/8t8ra3lrljuir7q4 - `org.apache.spark.deploy.SparkSubmitSuite`: https://ci.appveyor.com/project/spark-test/spark/build/771-windows-fix/job/taquy84yudjjen64 - `org.apache.spark.scheduler.ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/771-windows-fix/job/24omrfn2k0xfa9xq - `org.apache.spark.sql.hive.StatisticsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/771-windows-fix/job/2079y1plgj76dc9l **After** - `org.apache.spark.deploy.RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/775-windows-fix/job/3803dbfn89ne1164 - `org.apache.spark.deploy.SparkSubmitSuite`: https://ci.appveyor.com/project/spark-test/spark/build/775-windows-fix/job/m5l350dp7u9a4xjr - `org.apache.spark.scheduler.ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/775-windows-fix/job/565vf74pp6bfdk18 - `org.apache.spark.sql.hive.StatisticsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/775-windows-fix/job/qm78tsk8c37jb6s4 Jenkins tests are required and AppVeyor tests will be triggered. Author: hyukjinkwon <gurwls223@gmail.com> Closes #18971 from HyukjinKwon/windows-fixes.
-
- Aug 29, 2017
-
-
gatorsmile authored
## What changes were proposed in this pull request? We should make codegen fallback of expressions configurable. So far, it is always on. We might hide it when our codegen have compilation bugs. Thus, we should also disable the codegen fallback when running test cases. ## How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #19062 from gatorsmile/fallbackCodegen.
-
Wenchen Fan authored
## What changes were proposed in this pull request? This is a follow-up for https://github.com/apache/spark/pull/18488, to simplify the code. The major change is, we should map java enum to string type, instead of a struct type with a single string field. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #19066 from cloud-fan/fix.
-
Wang Gengliang authored
## What changes were proposed in this pull request? Add trait UserDefinedExpression to identify user-defined functions. UDF can be expensive. In optimizer we may need to avoid executing UDF multiple times. E.g. ```scala table.select(UDF as 'a).select('a, ('a + 1) as 'b) ``` If UDF is expensive in this case, optimizer should not collapse the project to ```scala table.select(UDF as 'a, (UDF+1) as 'b) ``` Currently UDF classes like PythonUDF, HiveGenericUDF are not defined in catalyst. This PR is to add a new trait to make it easier to identify user-defined functions. ## How was this patch tested? Unit test Author: Wang Gengliang <ltnwgl@gmail.com> Closes #19064 from gengliangwang/UDFType.
-
Takuya UESHIN authored
## What changes were proposed in this pull request? As mentioned at https://github.com/apache/spark/pull/18680#issuecomment-316820409, when we have more `ColumnVector` implementations, it might (or might not) have huge performance implications because it might disable inlining, or force virtual dispatches. As for read path, one of the major paths is the one generated by `ColumnBatchScan`. Currently it refers `ColumnVector` so the penalty will be bigger as we have more classes, but we can know the concrete type from its usage, e.g. vectorized Parquet reader uses `OnHeapColumnVector`. We can use the concrete type in the generated code directly to avoid the penalty. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@databricks.com> Closes #18989 from ueshin/issues/SPARK-21781.
-
- Aug 27, 2017
-
-
iamhumanbeing authored
Signed-off-by: iamhumanbeing <iamhumanbeinggmail.com> ## What changes were proposed in this pull request? testNameNote = "(minNumPostShufflePartitions: 3) is not correct. it should be "(minNumPostShufflePartitions: " + numPartitions + ")" in ExchangeCoordinatorSuite ## How was this patch tested? unit tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: iamhumanbeing <iamhumanbeing@gmail.com> Closes #19058 from iamhumanbeing/testnote.
-
- Aug 25, 2017
-
-
hyukjinkwon authored
## What changes were proposed in this pull request? This PR proposes both: - Add information about Javadoc, SQL docs and few more information in `docs/README.md` and a comment in `docs/_plugins/copy_api_dirs.rb` related with Javadoc. - Adds some commands so that the script always runs the SQL docs build under `./sql` directory (for directly running `./sql/create-docs.sh` in the root directory). ## How was this patch tested? Manual tests with `jekyll build` and `./sql/create-docs.sh` in the root directory. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19019 from HyukjinKwon/minor-doc-build.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? [SPARK-19025](https://github.com/apache/spark/pull/16869) removes SQLBuilder, so we don't need the following in HiveCompatibilitySuite. ```scala // Ensures that the plans generation use metastore relation and not OrcRelation // Was done because SqlBuilder does not work with plans having logical relation TestHive.setConf(HiveUtils.CONVERT_METASTORE_ORC, false) ``` ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19043 from dongjoon-hyun/SPARK-21831.
-
Sean Owen authored
## What changes were proposed in this pull request? Adjust Local UDTs test to assert about results, and fix index of vector column. See JIRA for details. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #19053 from srowen/SPARK-21837.
-
vinodkc authored
## What changes were proposed in this pull request? This patch adds allowUnquotedControlChars option in JSON data source to allow JSON Strings to contain unquoted control characters (ASCII characters with value less than 32, including tab and line feed characters) ## How was this patch tested? Add new test cases Author: vinodkc <vinod.kc.in@gmail.com> Closes #19008 from vinodkc/br_fix_SPARK-21756.
-
Dongjoon Hyun authored
## What changes were proposed in this pull request? After [SPARK-19025](https://github.com/apache/spark/pull/16869), there is no need to keep SQLBuilderTest. ExpressionSQLBuilderSuite is the only place to use it. This PR aims to remove SQLBuilderTest. ## How was this patch tested? Pass the updated `ExpressionSQLBuilderSuite`. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19044 from dongjoon-hyun/SPARK-21832.
-
Sean Owen authored
## What changes were proposed in this pull request? Fix build warnings and Java lint errors. This just helps a bit in evaluating (new) warnings in another PR I have open. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #19051 from srowen/JavaWarnings.
-
mike authored
## What changes were proposed in this pull request? Fixed NPE when creating encoder for enum. When you try to create an encoder for Enum type (or bean with enum property) via Encoders.bean(...), it fails with NullPointerException at TypeToken:495. I did a little research and it turns out, that in JavaTypeInference following code ``` def getJavaBeanReadableProperties(beanClass: Class[_]): Array[PropertyDescriptor] = { val beanInfo = Introspector.getBeanInfo(beanClass) beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") .filter(_.getReadMethod != null) } ``` filters out properties named "class", because we wouldn't want to serialize that. But enum types have another property of type Class named "declaringClass", which we are trying to inspect recursively. Eventually we try to inspect ClassLoader class, which has property "defaultAssertionStatus" with no read method, which leads to NPE at TypeToken:495. I added property name "declaringClass" to filtering to resolve this. ## How was this patch tested? Unit test in JavaDatasetSuite which creates an encoder for enum Author: mike <mike0sv@gmail.com> Author: Mikhail Sveshnikov <mike0sv@gmail.com> Closes #18488 from mike0sv/enum-support.
-
- Aug 24, 2017
-
-
Herman van Hovell authored
## What changes were proposed in this pull request? This PR bumps the ANTLR version to 4.7, and fixes a number of small parser related issues uncovered by the bump. The main reason for upgrading is that in some cases the current version of ANTLR (4.5) can exhibit exponential slowdowns if it needs to parse boolean predicates. For example the following query will take forever to parse: ```sql SELECT * FROM RANGE(1000) WHERE TRUE AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' AND NOT upper(DESCRIPTION) LIKE '%FOO%' ``` This is caused by a know bug in ANTLR (https://github.com/antlr/antlr4/issues/994), which was fixed in version 4.6. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #19042 from hvanhovell/SPARK-21830.
-