aboutsummaryrefslogtreecommitdiff
path: root/external
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-20189][DSTREAM] Fix spark kinesis testcases to remove deprecated ↵Yash Sharma2017-04-132-22/+38
| | | | | | | | | | | | | | | | | | | createStream and use Builders ## What changes were proposed in this pull request? The spark-kinesis testcases use the KinesisUtils.createStream which are deprecated now. Modify the testcases to use the recommended KinesisInputDStream.builder instead. This change will also enable the testcases to automatically use the session tokens automatically. ## How was this patch tested? All the existing testcases work fine as expected with the changes. https://issues.apache.org/jira/browse/SPARK-20189 Author: Yash Sharma <ysharma@atlassian.com> Closes #17506 from yssharma/ysharma/cleanup_kinesis_testcases.
* [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish ↵Sean Owen2017-04-106-30/+38
| | | | | | | | | | | | | | | | | | locale bug" causes Spark problems ## What changes were proposed in this pull request? Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods. ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #17527 from srowen/SPARK-20156.
* [SPARK-20209][SS] Execute next trigger immediately if previous batch took ↵Tathagata Das2017-04-041-0/+1
| | | | | | | | | | | | | | | | | longer than trigger interval ## What changes were proposed in this pull request? For large trigger intervals (e.g. 10 minutes), if a batch takes 11 minutes, then it will wait for 9 mins before starting the next batch. This does not make sense. The processing time based trigger policy should be to do process batches as fast as possible, but no faster than 1 in every trigger interval. If batches are taking longer than trigger interval anyways, then no point waiting extra trigger interval. In this PR, I modified the ProcessingTimeExecutor to do so. Another minor change I did was to extract our StreamManualClock into a separate class so that it can be used outside subclasses of StreamTest. For example, ProcessingTimeExecutorSuite does not need to create any context for testing, just needs the StreamManualClock. ## How was this patch tested? Added new unit tests to comprehensively test this behavior. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17525 from tdas/SPARK-20209.
* [SPARK-19911][STREAMING] Add builder interface for Kinesis DStreamsAdam Budde2017-03-2411-149/+749
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Add new KinesisDStream.scala containing KinesisDStream.Builder class - Add KinesisDStreamBuilderSuite test suite - Make KinesisInputDStream ctor args package private for testing - Add JavaKinesisDStreamBuilderSuite test suite - Add args to KinesisInputDStream and KinesisReceiver for optional service-specific auth (Kinesis, DynamoDB and CloudWatch) ## How was this patch tested? Added ```KinesisDStreamBuilderSuite``` to verify builder class works as expected Author: Adam Budde <budde@amazon.com> Closes #17250 from budde/KinesisStreamBuilder.
* [SPARK-19876][SS][WIP] OneTime Trigger ExecutorTyson Condie2017-03-231-2/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
* [SPARK-19721][SS] Good error message for version mismatch in log filesLiwei Lin2017-03-162-9/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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 #17070 from lw-lin/better-msg.
* [SPARK-19853][SS] uppercase kafka topics fail when startingOffsets are ↵uncleGen2017-03-122-34/+54
| | | | | | | | | | | | | | | | | | | | | | | | SpecificOffsets 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.
* [SPARK-19886] Fix reportDataLoss if statement in SS KafkaSourceBurak Yavuz2017-03-092-13/+54
| | | | | | | | | | | | | | ## 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.
* [SPARK-19719][SS] Kafka writer for both structured streaming and batch queiresTyson Condie2017-03-065-5/+753
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a new Kafka Sink and Kafka Relation for writing streaming and batch queries, respectively, to Apache Kafka. ### Streaming Kafka Sink - When addBatch is called -- If batchId is great than the last written batch --- Write batch to Kafka ---- Topic will be taken from the record, if present, or from a topic option, which overrides topic in record. -- Else ignore ### Batch Kafka Sink - KafkaSourceProvider will implement CreatableRelationProvider - CreatableRelationProvider#createRelation will write the passed in Dataframe to a Kafka - Topic will be taken from the record, if present, or from topic option, which overrides topic in record. - Save modes Append and ErrorIfExist supported under identical semantics. Other save modes result in an AnalysisException tdas zsxwing ## How was this patch tested? ### The following unit tests will be included - write to stream with topic field: valid stream write with data that includes an existing topic in the schema - write structured streaming aggregation w/o topic field, with default topic: valid stream write with data that does not include a topic field, but the configuration includes a default topic - write data with bad schema: various cases of writing data that does not conform to a proper schema e.g., 1. no topic field or default topic, and 2. no value field - write data with valid schema but wrong types: data with a complete schema but wrong types e.g., key and value types are integers. - write to non-existing topic: write a stream to a topic that does not exist in Kafka, which has been configured to not auto-create topics. - write batch to kafka: simple write batch to Kafka, which goes through the same code path as streaming scenario, so validity checks will not be redone here. ### Examples ```scala // Structured Streaming val writer = inputStringStream.map(s => s.get(0).toString.getBytes()).toDF("value") .selectExpr("value as key", "value as value") .writeStream .format("kafka") .option("checkpointLocation", checkpointDir) .outputMode(OutputMode.Append) .option("kafka.bootstrap.servers", brokerAddress) .option("topic", topic) .queryName("kafkaStream") .start() // Batch val df = spark .sparkContext .parallelize(Seq("1", "2", "3", "4", "5")) .map(v => (topic, v)) .toDF("topic", "value") df.write .format("kafka") .option("kafka.bootstrap.servers",brokerAddress) .option("topic", topic) .save() ``` Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Closes #17043 from tcondie/kafka-writer.
* [SPARK-19304][STREAMING][KINESIS] fix kinesis slow checkpoint recoveryGaurav2017-03-064-11/+25
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? added a limit to getRecords api call call in KinesisBackedBlockRdd. This helps reduce the amount of data returned by kinesis api call making the recovery considerably faster As we are storing the `fromSeqNum` & `toSeqNum` in checkpoint metadata, we can also store the number of records. Which can later be used for api call. ## How was this patch tested? The patch was manually tested Apologies for any silly mistakes, opening first pull request Author: Gaurav <gaurav@techtinium.com> Closes #16842 from Gauravshah/kinesis_checkpoint_recovery_fix_2_1_0.
* [MINOR][BUILD] Fix lint-java breaks in Javahyukjinkwon2017-02-272-3/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to fix the lint-breaks as below: ``` [ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer. [ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers. [ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105). [ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed. [ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121). [ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113). [ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122). [ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time. [ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101). [ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102). [ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102). [ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101). [ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110). [ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114). [ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101). [ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils. [ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils. [ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103). [ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1. [ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104). [ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101). [ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD. [ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext. ``` ## How was this patch tested? Manually via ```bash ./dev/lint-java ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #17072 from HyukjinKwon/java-lint.
* [SPARK-19405][STREAMING] Support for cross-account Kinesis reads via STSAdam Budde2017-02-2215-80/+394
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | - Add dependency on aws-java-sdk-sts - Replace SerializableAWSCredentials with new SerializableCredentialsProvider interface - Make KinesisReceiver take SerializableCredentialsProvider as argument and pass credential provider to KCL - Add new implementations of KinesisUtils.createStream() that take STS arguments - Make JavaKinesisStreamSuite test the entire KinesisUtils Java API - Update KCL/AWS SDK dependencies to 1.7.x/1.11.x ## What changes were proposed in this pull request? [JIRA link with detailed description.](https://issues.apache.org/jira/browse/SPARK-19405) * Replace SerializableAWSCredentials with new SerializableKCLAuthProvider class that takes 5 optional config params for configuring AWS auth and returns the appropriate credential provider object * Add new public createStream() APIs for specifying these parameters in KinesisUtils ## How was this patch tested? * Manually tested using explicit keypair and instance profile to read data from Kinesis stream in separate account (difficult to write a test orchestrating creation and assumption of IAM roles across separate accounts) * Expanded JavaKinesisStreamSuite to test the entire Java API in KinesisUtils ## License acknowledgement This contribution is my original work and that I license the work to the project under the project’s open source license. Author: Budde <budde@amazon.com> Closes #16744 from budde/master.
* [SPARK-18922][TESTS] Fix new test failures on Windows due to path and ↵hyukjinkwon2017-02-202-9/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | resource not closed ## What changes were proposed in this pull request? This PR proposes to fix new test failures on WIndows as below: **Before** ``` KafkaRelationSuite: - test late binding start offsets *** FAILED *** (7 seconds, 679 milliseconds) Cause: java.nio.file.FileSystemException: C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log -> C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process. KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 *** FAILED *** (3 seconds, 542 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-97ef64fc-ae61-4ce3-ac59-287fd38bd824 - deserialization of initial offset written by Spark 2.1.0 *** FAILED *** (60 milliseconds) java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/external/kafka-0-10-sql/target/scala-2.11/test-classes/kafka-source-initial-offset-version-2.1.0.b HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema *** FAILED *** (657 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f1b83d09-850a-4bba-8e43-a2a28dfaa757; DDLSuite: - create a data source table without schema *** FAILED *** (94 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-a3f3c161-afae-4d6f-9182-e8642f77062b; - SET LOCATION for managed table *** FAILED *** (219 milliseconds) org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: Exchange SinglePartit +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#99367L]) +- *FileScan parquet default.tbl[] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/C:projectsspark arget mpspark-15be2f2f-4ea9-4c47-bfee-1b7b49363033], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<> - insert data to a data source table which has a not existed location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-34987671-e8d1-4624-ba5b-db1012e1246b; - insert into a data source table with no existed partition location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4c6ccfbf-4091-4032-9fbc-3d40c58267d5; - read data from a data source table which has a not existed location should succeed *** FAILED *** (0 milliseconds) - read data from a data source table with no existed partition location should succeed *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-6af39e37-abd1-44e8-ac68-e2dfcf67a2f3; InputOutputMetricsSuite: - output metrics on records written *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-cd69ee77-88f2-4202-bed6-19c0ee05ef55\InputOutputMetricsSuite, expected: file:/// - output metrics on records written - new Hadoop API *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-b69e8fcb-047b-4de8-9cdf-5f026efb6762\InputOutputMetricsSuite, expected: file:/// ``` **After** ``` KafkaRelationSuite: - test late binding start offsets !!! CANCELED !!! (62 milliseconds) KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 (5 seconds, 341 milliseconds) - deserialization of initial offset written by Spark 2.1.0 (910 milliseconds) HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema (2 seconds) DDLSuite: - create a data source table without schema (828 milliseconds) - SET LOCATION for managed table (406 milliseconds) - insert data to a data source table which has a not existed location should succeed (406 milliseconds) - insert into a data source table with no existed partition location should succeed (453 milliseconds) - read data from a data source table which has a not existed location should succeed (94 milliseconds) - read data from a data source table with no existed partition location should succeed (265 milliseconds) InputOutputMetricsSuite: - output metrics on records written (172 milliseconds) - output metrics on records written - new Hadoop API (297 milliseconds) ``` ## How was this patch tested? Fixed tests in `InputOutputMetricsSuite`, `KafkaRelationSuite`, `KafkaSourceSuite`, `DDLSuite.scala` and `HiveDDLSuite`. Manually tested via AppVeyor as below: `InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/ex8nvwa6tsh7rmto `KafkaRelationSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/h8dlcowew52y8ncw `KafkaSourceSuite`: https://ci.appveyor.com/project/spark-test/spark/build/634-20170219-windows-test/job/9ybgjl7yeubxcre4 `DDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/635-20170219-windows-test `HiveDDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/up6o9n47er087ltb Author: hyukjinkwon <gurwls223@gmail.com> Closes #16999 from HyukjinKwon/windows-fix.
* [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-191-12/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #16964 from srowen/SPARK-19534.
* [SPARK-19617][SS] Fix the race condition when starting and stopping a query ↵Shixiong Zhu2017-02-171-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | quickly ## What changes were proposed in this pull request? The streaming thread in StreamExecution uses the following ways to check if it should exit: - Catch an InterruptException. - `StreamExecution.state` is TERMINATED. When starting and stopping a query quickly, the above two checks may both fail: - Hit [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) and swallow InterruptException - StreamExecution.stop is called before `state` becomes `ACTIVE`. Then [runBatches](https://github.com/apache/spark/blob/dcc2d540a53f0bd04baead43fdee1c170ef2b9f3/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L252) changes the state from `TERMINATED` to `ACTIVE`. If the above cases both happen, the query will hang forever. This PR changes `state` to `AtomicReference` and uses`compareAndSet` to make sure we only change the state from `INITIALIZING` to `ACTIVE`. It also removes the `runUninterruptibly` hack from ``HDFSMetadata`, because HADOOP-14084 won't cause any problem after we fix the race condition. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16947 from zsxwing/SPARK-19617.
* [SPARK-19517][SS] KafkaSource fails to initialize partition offsetsRoberto Agostino Vitillo2017-02-173-7/+130
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a bug in `KafkaSource` with the (de)serialization of the length of the JSON string that contains the initial partition offsets. ## How was this patch tested? I ran the test suite for spark-sql-kafka-0-10. Author: Roberto Agostino Vitillo <ra.vitillo@gmail.com> Closes #16857 from vitillo/kafka_source_fix.
* [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 supportSean Owen2017-02-169-1519/+2
| | | | | | | | | | | | | | | | | | | | | | | | - Move external/java8-tests tests into core, streaming, sql and remove - Remove MaxPermGen and related options - Fix some reflection / TODOs around Java 8+ methods - Update doc references to 1.7/1.8 differences - Remove Java 7/8 related build profiles - Update some plugins for better Java 8 compatibility - Fix a few Java-related warnings For the future: - Update Java 8 examples to fully use Java 8 - Update Java tests to use lambdas for simplicity - Update Java internal implementations to use lambdas ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16871 from srowen/SPARK-19493.
* [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the ↵sureshthalamati2017-02-141-0/+36
| | | | | | | | | | | | | | | | | | | | user in case-sensitive manner. ## What changes were proposed in this pull request? The reason for test failure is that the property “oracle.jdbc.mapDateToTimestamp” set by the test was getting converted into all lower case. Oracle database expects this property in case-sensitive manner. This test was passing in previous releases because connection properties were sent as user specified for the test case scenario. Fixes to handle all option uniformly in case-insensitive manner, converted the JDBC connection properties also to lower case. This PR enhances CaseInsensitiveMap to keep track of input case-sensitive keys , and uses those when creating connection properties that are passed to the JDBC connection. Alternative approach PR https://github.com/apache/spark/pull/16847 is to pass original input keys to JDBC data source by adding check in the Data source class and handle case-insensitivity in the JDBC source code. ## How was this patch tested? Added new test cases to JdbcSuite , and OracleIntegrationSuite. Ran docker integration tests passed on my laptop, all tests passed successfully. Author: sureshthalamati <suresh.thalamati@gmail.com> Closes #16891 from sureshthalamati/jdbc_case_senstivity_props_fix-SPARK-19318.
* [SPARK-19564][SPARK-19559][SS][KAFKA] KafkaOffsetReader's consumers should ↵Liwei Lin2017-02-121-4/+7
| | | | | | | | | | | | | | | | | | | | not be in the same group ## What changes were proposed in this pull request? In `KafkaOffsetReader`, when error occurs, we abort the existing consumer and create a new consumer. In our current implementation, the first consumer and the second consumer would be in the same group (which leads to SPARK-19559), **_violating our intention of the two consumers not being in the same group._** The cause is that, in our current implementation, the first consumer is created before `groupId` and `nextId` are initialized in the constructor. Then even if `groupId` and `nextId` are increased during the creation of that first consumer, `groupId` and `nextId` would still be initialized to default values in the constructor for the second consumer. We should make sure that `groupId` and `nextId` are initialized before any consumer is created. ## How was this patch tested? Ran 100 times of `KafkaSourceSuite`; all passed Author: Liwei Lin <lwlin7@gmail.com> Closes #16902 from lw-lin/SPARK-19564-.
* [SPARK-18682][SS] Batch Source for KafkaTyson Condie2017-02-0712-430/+1180
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Today, you can start a stream that reads from kafka. However, given kafka's configurable retention period, it seems like sometimes you might just want to read all of the data that is available now. As such we should add a version that works with spark.read as well. The options should be the same as the streaming kafka source, with the following differences: startingOffsets should default to earliest, and should not allow latest (which would always be empty). endingOffsets should also be allowed and should default to latest. the same assign json format as startingOffsets should also be accepted. It would be really good, if things like .limit(n) were enough to prevent all the data from being read (this might just work). ## How was this patch tested? KafkaRelationSuite was added for testing batch queries via KafkaUtils. Author: Tyson Condie <tcondie@gmail.com> Closes #16686 from tcondie/SPARK-18682.
* [SPARK-18020][STREAMING][KINESIS] Checkpoint SHARD_END to finish reading ↵Takeshi YAMAMURO2017-01-255-7/+116
| | | | | | | | | | | | | | closed shards ## What changes were proposed in this pull request? This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint `SHARD_END` when finishing reading closed shards in `KinesisRecordProcessor#shutdown`. This bug finally leads to stopping subscribing new split (or merged) shards. ## How was this patch tested? Added a test in `KinesisStreamSuite` to check if it works well when splitting/merging shards. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16213 from maropu/SPARK-18020.
* [SPARK-14536][SQL] fix to handle null value in array type column for postgres.sureshthalamati2017-01-201-2/+10
| | | | | | | | | | | | | | ## 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 #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
* [SPARK-19227][SPARK-19251] remove unused imports and outdated commentsuncleGen2017-01-186-7/+5
| | | | | | | | | | | | ## What changes were proposed in this pull request? remove ununsed imports and outdated comments, and fix some minor code style issue. ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16591 from uncleGen/SPARK-19227.
* [SPARK-19206][DOC][DSTREAM] Fix outdated parameter descriptions in kafka010uncleGen2017-01-153-25/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix outdated parameter descriptions in kafka010 ## How was this patch tested? cc koeninger zsxwing Author: uncleGen <hustyugm@gmail.com> Closes #16569 from uncleGen/SPARK-19206.
* [SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed ↵hyukjinkwon2017-01-106-29/+91
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | due to path and resource-not-closed problems on Windows ## What changes were proposed in this pull request? This PR proposes to fix all the test failures identified by testing with AppVeyor. **Scala - aborted tests** ``` WindowQuerySuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilespart_tiny.txt; OrcSourceSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 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); ParquetMetastoreSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 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); ParquetSourceSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-581a6575-454f-4f21-a516-a07f95266143; KafkaRDDSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807 DirectKafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) ReliableKafkaStreamSuite Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888 KafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c KafkaClusterSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d DirectKafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6 KafkaRDDSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2 ``` **Java - failed tests** ``` Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec ``` **Scala - failed tests** ``` PartitionProviderCompatibilitySuite: - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 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-18635 special chars in partition values - partition management false *** FAILED *** (141 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); ``` ``` UtilsSuite: - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0 ``` ``` StatisticsSuite: - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds) org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default'; ``` ``` SQLQuerySuite: - permanent UDTF *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24 - describe functions - user defined functions *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 - CTAS without serde with location *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds) java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark arget mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r* ``` ``` HiveDDLSuite: - drop external tables in default database *** FAILED *** (16 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); - add/drop partitions - external table *** FAILED *** (16 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); - create/drop database - location without pre-created directory *** FAILED *** (16 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); - create/drop database - location with pre-created directory *** FAILED *** (32 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); - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop an empty database - CASCADE *** FAILED *** (63 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888; - CREATE TABLE LIKE an external Hive serde table *** FAILED *** (16 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); - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e8bf5bf5-721a-4cbe-9d6 at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d; ``` ``` MetastoreDataSourcesSuite - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ``` ``` ShowCreateTableSuite: - simple external hive table *** FAILED *** (0 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); ``` ``` PartitionedTablePerfStatsSuite: - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 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); - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-311f45f8-d064-4023-a4bb-e28235bff64d; - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 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); - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957; - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 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); - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e6d20183-dd68-4145-acbe-4a509849accd; - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 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); - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-8b2c9651-2adf-4d58-874f-659007e21463; - hive table: file status cache respects size limit *** FAILED *** (219 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); - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4; - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2; - hive table: table setup does not scan filesystem *** FAILED *** (266 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); - hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 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); - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383; - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 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); - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12; - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 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); ``` ``` HiveSparkSubmitSuite: - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified ``` ``` UtilsSuite: - resolveURIs with multiple paths *** FAILED *** (0 milliseconds) ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468) ``` ``` CheckpointSuite: - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds) The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1 \ ^. (CheckpointSuite.scala:680) ``` ## How was this patch tested? Manually via AppVeyor as below: **Scala - aborted tests** ``` WindowQuerySuite - all passed OrcSourceSuite: - SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds) org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625) ParquetMetastoreSuite - all passed ParquetSourceSuite - all passed KafkaRDDSuite - all passed DirectKafkaStreamSuite - all passed ReliableKafkaStreamSuite - all passed KafkaStreamSuite - all passed KafkaClusterSuite - all passed DirectKafkaStreamSuite - all passed KafkaRDDSuite - all passed ``` **Java - failed tests** ``` org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed ``` **Scala - failed tests** ``` PartitionProviderCompatibilitySuite: - insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds) - SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds) - SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds) ``` ``` UtilsSuite: - reading offset bytes of a file (compressed) (203 milliseconds) - reading offset bytes across multiple files (compressed) (0 milliseconds) ``` ``` StatisticsSuite: - MetastoreRelations fallback to HDFS for size estimation (94 milliseconds) ``` ``` SQLQuerySuite: - permanent UDTF (407 milliseconds) - describe functions - user defined functions (441 milliseconds) - CTAS without serde with location (2 seconds, 831 milliseconds) - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds) - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds) - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds) ``` ``` HiveDDLSuite: - drop external tables in default database (3 seconds, 5 milliseconds) - add/drop partitions - external table (2 seconds, 750 milliseconds) - create/drop database - location without pre-created directory (500 milliseconds) - create/drop database - location with pre-created directory (407 milliseconds) - drop database containing tables - CASCADE (453 milliseconds) - drop an empty database - CASCADE (375 milliseconds) - drop database containing tables - RESTRICT (328 milliseconds) - drop an empty database - RESTRICT (391 milliseconds) - CREATE TABLE LIKE an external data source table (953 milliseconds) - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds) - desc table for data source table - no user-defined schema (1 second, 150 milliseconds) ``` ``` MetastoreDataSourcesSuite - CTAS: persisted bucketed data source table (875 milliseconds) ``` ``` ShowCreateTableSuite: - simple external hive table (78 milliseconds) ``` ``` PartitionedTablePerfStatsSuite: - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds) - datasource table: partitioned pruned table reports only selected files (860 milliseconds) - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds) - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds) - hive table: lazy partition pruning with file status caching enabled (875 milliseconds) - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds) - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds) - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds) - hive table: file status cache respects size limit (469 milliseconds) - datasource table: file status cache respects size limit (453 milliseconds) - datasource table: table setup does not scan filesystem (328 milliseconds) - hive table: table setup does not scan filesystem (313 milliseconds) - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds) - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds) - hive table: files read and cached when filesource partition management is off (656 milliseconds) - datasource table: all partition data cached in memory when partition management is off (484 milliseconds) - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds) ``` ``` HiveSparkSubmitSuite: - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds) - permanent Hive UDF: define a UDF and use it (406 milliseconds) - permanent Hive UDF: use a already defined permanent function (375 milliseconds) - SPARK-8368: includes jars passed in through --jars (391 milliseconds) - SPARK-8020: set sql conf in spark conf (156 milliseconds) - SPARK-8489: MissingRequirementError during reflection (187 milliseconds) - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds) - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds) - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds) - set spark.sql.warehouse.dir (172 milliseconds) - set hive.metastore.warehouse.dir (156 milliseconds) - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds) - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds) ``` ``` UtilsSuite: - resolveURIs with multiple paths (0 milliseconds) ``` ``` CheckpointSuite: - recovery with file input stream (4 seconds, 452 milliseconds) ``` Note: after resolving the aborted tests, there is a test failure identified as below: ``` OrcSourceSuite: - SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds) org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625) ``` This does not look due to this problem so this PR does not fix it here. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16451 from HyukjinKwon/all-path-resource-fixes.
* [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark RepoNiranjan Padmanabhan2017-01-043-3/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words. ## How was this patch tested? N/A since only docs or comments were updated. Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com> Closes #16455 from neurons/np.structure_streaming_doc.
* [SPARK-17807][CORE] split test-tags into test-JARRyan Williams2016-12-218-1/+85
| | | | | | | | | | Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR. Alternative to #16303. Author: Ryan Williams <ryan.blake.williams@gmail.com> Closes #16311 from ryan-williams/tt.
* [SPARK-18588][SS][KAFKA] Create a new KafkaConsumer when error happens to ↵Shixiong Zhu2016-12-213-31/+50
| | | | | | | | | | | | | | | | | | fix the flaky test ## What changes were proposed in this pull request? When KafkaSource fails on Kafka errors, we should create a new consumer to retry rather than using the existing broken one because it's possible that the broken one will fail again. This PR also assigns a new group id to the new created consumer for a possible race condition: the broken consumer cannot talk with the Kafka cluster in `close` but the new consumer can talk to Kafka cluster. I'm not sure if this will happen or not. Just for safety to avoid that the Kafka cluster thinks there are two consumers with the same group id in a short time window. (Note: CachedKafkaConsumer doesn't need this fix since `assign` never uses the group id.) ## How was this patch tested? In https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70370/console , it ran this flaky test 120 times and all passed. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16282 from zsxwing/kafka-fix.
* [SPARK-18588][TESTS] Ignore KafkaSourceStressForDontFailOnDataLossSuiteShixiong Zhu2016-12-131-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Disable KafkaSourceStressForDontFailOnDataLossSuite for now. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16275 from zsxwing/ignore-flaky-test.
* [SPARK-18620][STREAMING][KINESIS] Flatten input rates in timeline for ↵Takeshi YAMAMURO2016-12-103-2/+35
| | | | | | | | | | | | | | | | | | | | | streaming + kinesis ## What changes were proposed in this pull request? This pr is to make input rates in timeline more flat for spark streaming + kinesis. Since kinesis workers fetch records and push them into block generators in bulk, timeline in web UI has many spikes when `maxRates` applied (See a Figure.1 below). This fix splits fetched input records into multiple `adRecords` calls. Figure.1 Apply `maxRates=500` in vanilla Spark <img width="1084" alt="apply_limit in_vanilla_spark" src="https://cloud.githubusercontent.com/assets/692303/20823861/4602f300-b89b-11e6-95f3-164a37061305.png"> Figure.2 Apply `maxRates=500` in Spark with my patch <img width="1056" alt="apply_limit in_spark_with_my_patch" src="https://cloud.githubusercontent.com/assets/692303/20823882/6c46352c-b89b-11e6-81ab-afd8abfe0cfe.png"> ## How was this patch tested? Add tests to check to split input records into multiple `addRecords` calls. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16114 from maropu/SPARK-18620.
* [SPARK-18776][SS] Make Offset for FileStreamSource corrected formatted in jsonTathagata Das2016-12-081-1/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Changed FileStreamSource to use new FileStreamSourceOffset rather than LongOffset. The field is named as `logOffset` to make it more clear that this is a offset in the file stream log. - Fixed bug in FileStreamSourceLog, the field endId in the FileStreamSourceLog.get(startId, endId) was not being used at all. No test caught it earlier. Only my updated tests caught it. Other minor changes - Dont use batchId in the FileStreamSource, as calling it batch id is extremely miss leading. With multiple sources, it may happen that a new batch has no new data from a file source. So offset of FileStreamSource != batchId after that batch. ## How was this patch tested? Updated unit test. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16205 from tdas/SPARK-18776.
* [SPARK-18754][SS] Rename recentProgresses to recentProgressMichael Armbrust2016-12-071-1/+1
| | | | | | | | Based on an informal survey, users find this option easier to understand / remember. Author: Michael Armbrust <michael@databricks.com> Closes #16182 from marmbrus/renameRecentProgress.
* [SPARK-18588][TESTS] Fix flaky test: KafkaSourceStressForDontFailOnDataLossSuiteShixiong Zhu2016-12-074-37/+90
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixed the following failures: ``` org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 3745 times over 1.0000790851666665 minutes. Last failure message: assertion failed: failOnDataLoss-0 not deleted after timeout. ``` ``` sbt.ForkMain$ForkError: org.apache.spark.sql.streaming.StreamingQueryException: Query query-66 terminated with exception: null at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:252) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:146) Caused by: sbt.ForkMain$ForkError: java.lang.NullPointerException: null at java.util.ArrayList.addAll(ArrayList.java:577) at org.apache.kafka.clients.Metadata.getClusterForCurrentTopics(Metadata.java:257) at org.apache.kafka.clients.Metadata.update(Metadata.java:177) at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.handleResponse(NetworkClient.java:605) at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeHandleCompletedReceive(NetworkClient.java:582) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:450) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.awaitPendingRequests(ConsumerNetworkClient.java:260) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:222) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.ensurePartitionAssignment(ConsumerCoordinator.java:366) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:978) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:938) at ... ``` ## How was this patch tested? Tested in #16048 by running many times. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16109 from zsxwing/fix-kafka-flaky-test.
* [SPARK-18671][SS][TEST-MAVEN] Follow up PR to fix test for MavenTathagata Das2016-12-062-1/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Maven compilation seem to not allow resource is sql/test to be easily referred to in kafka-0-10-sql tests. So moved the kafka-source-offset-version-2.1.0 from sql test resources to kafka-0-10-sql test resources. ## How was this patch tested? Manually ran maven test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16183 from tdas/SPARK-18671-1.
* [SPARK-18671][SS][TEST] Added tests to ensure stability of that all ↵Tathagata Das2016-12-062-1/+20
| | | | | | | | | | | | | | | Structured Streaming log formats ## What changes were proposed in this pull request? To be able to restart StreamingQueries across Spark version, we have already made the logs (offset log, file source log, file sink log) use json. We should added tests with actual json files in the Spark such that any incompatible changes in reading the logs is immediately caught. This PR add tests for FileStreamSourceLog, FileStreamSinkLog, and OffsetSeqLog. ## How was this patch tested? new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16128 from tdas/SPARK-18671.
* [SPARK-18695] Bump master branch version to 2.2.0-SNAPSHOTReynold Xin2016-12-0213-13/+13
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch bumps master branch version to 2.2.0-SNAPSHOT. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #16126 from rxin/SPARK-18695.
* [SPARK-18516][SQL] Split state and progress in streamingTathagata Das2016-11-291-3/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
* [SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility ↵hyukjinkwon2016-11-295-16/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | in Java API documentation ## What changes were proposed in this pull request? This PR make `sbt unidoc` complete with Java 8. This PR roughly includes several fixes as below: - Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` `` ```diff - * A column that will be computed based on the data in a [[DataFrame]]. + * A column that will be computed based on the data in a `DataFrame`. ``` - Fix throws annotations so that they are recognisable in javadoc - Fix URL links to `<a href="http..."></a>`. ```diff - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning"> + * Decision tree (Wikipedia)</a> model for regression. ``` ```diff - * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic"> + * Receiver operating characteristic (Wikipedia)</a> ``` - Fix < to > to - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable. - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558 - Fix `</p>` complaint ## How was this patch tested? Manually tested by `jekyll build` with Java 7 and 8 ``` java version "1.7.0_80" Java(TM) SE Runtime Environment (build 1.7.0_80-b15) Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode) ``` ``` java version "1.8.0_45" Java(TM) SE Runtime Environment (build 1.8.0_45-b14) Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode) ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.
* [SPARK-18588][SS][KAFKA] Ignore the flaky kafka testShixiong Zhu2016-11-281-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Ignore the flaky test to unblock other PRs while I'm debugging it. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16051 from zsxwing/ignore-flaky-kafka-test.
* [SPARK-18530][SS][KAFKA] Change Kafka timestamp column type to TimestampTypeShixiong Zhu2016-11-222-4/+93
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Changed Kafka timestamp column type to TimestampType. ## How was this patch tested? `test("Kafka column types")`. Author: Shixiong Zhu <shixiong@databricks.com> Closes #15969 from zsxwing/SPARK-18530.
* [SPARK-18373][SPARK-18529][SS][KAFKA] Make failOnDataLoss=false work with ↵Shixiong Zhu2016-11-225-95/+523
| | | | | | | | | | | | | | | | | | | | Spark jobs ## What changes were proposed in this pull request? This PR adds `CachedKafkaConsumer.getAndIgnoreLostData` to handle corner cases of `failOnDataLoss=false`. It also resolves [SPARK-18529](https://issues.apache.org/jira/browse/SPARK-18529) after refactoring codes: Timeout will throw a TimeoutException. ## How was this patch tested? Because I cannot find any way to manually control the Kafka server to clean up logs, it's impossible to write unit tests for each corner case. Therefore, I just created `test("stress test for failOnDataLoss=false")` which should cover most of corner cases. I also modified some existing tests to test for both `failOnDataLoss=false` and `failOnDataLoss=true` to make sure it doesn't break existing logic. Author: Shixiong Zhu <shixiong@databricks.com> Closes #15820 from zsxwing/failOnDataLoss.
* [SPARK-18445][BUILD][DOCS] Fix the markdown for `Note:`/`NOTE:`/`Note ↵hyukjinkwon2016-11-194-38/+34
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | that`/`'''Note:'''` across Scala/Java API documentation ## What changes were proposed in this pull request? It seems in Scala/Java, - `Note:` - `NOTE:` - `Note that` - `'''Note:'''` - `note` This PR proposes to fix those to `note` to be consistent. **Before** - Scala ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png) - Java ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png) **After** - Scala ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png) - Java ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png) ## How was this patch tested? The notes were found via ```bash grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:''' grep -v "// NOTE: " | \ # starting with // does not appear in API documentation. grep -E '.scala|.java' | \ # java/scala files grep -v Suite | \ # exclude tests grep -v Test | \ # exclude tests grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation -e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...` -e 'org.apache.spark.api.r' \ ... ``` ```bash grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:''' grep -v "// Note that " | \ # starting with // does not appear in API documentation. grep -E '.scala|.java' | \ # java/scala files grep -v Suite | \ # exclude tests grep -v Test | \ # exclude tests grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation -e 'org.apache.spark.api.java.function' \ -e 'org.apache.spark.api.r' \ ... ``` ```bash grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:''' grep -v "// Note: " | \ # starting with // does not appear in API documentation. grep -E '.scala|.java' | \ # java/scala files grep -v Suite | \ # exclude tests grep -v Test | \ # exclude tests grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation -e 'org.apache.spark.api.java.function' \ -e 'org.apache.spark.api.r' \ ... ``` ```bash grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:''' grep -v "// '''Note:''' " | \ # starting with // does not appear in API documentation. grep -E '.scala|.java' | \ # java/scala files grep -v Suite | \ # exclude tests grep -v Test | \ # exclude tests grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation -e 'org.apache.spark.api.java.function' \ -e 'org.apache.spark.api.r' \ ... ``` And then fixed one by one comparing with API documentation/access modifiers. After that, manually tested via `jekyll build`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15889 from HyukjinKwon/SPARK-18437.
* [SPARK-18400][STREAMING] NPE when resharding Kinesis StreamSean Owen2016-11-161-19/+23
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid NPE in KinesisRecordProcessor when shutdown happens without successful init ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #15882 from srowen/SPARK-18400.
* [SPARK-17510][STREAMING][KAFKA] config max rate on a per-partition basiscody koeninger2016-11-145-18/+131
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Allow configuration of max rate on a per-topicpartition basis. ## How was this patch tested? Unit tests. The reporter (Jeff Nadler) said he could test on his workload, so let's wait on that report. Author: cody koeninger <cody@koeninger.org> Closes #15132 from koeninger/SPARK-17510.
* [SPARK-14914][CORE] Fix Resource not closed after using, for unit tests and ↵wm624@hotmail.com2016-11-103-0/+5
| | | | | | | | | | | | | | | | | | | example ## What changes were proposed in this pull request? This is a follow-up work of #15618. Close file source; For any newly created streaming context outside the withContext, explicitly close the context. ## How was this patch tested? Existing unit tests. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #15818 from wangmiao1981/rtest.
* [SPARK-17829][SQL] Stable format for offset logTyson Condie2016-11-094-8/+82
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we use java serialization for the WAL that stores the offsets contained in each batch. This has two main issues: It can break across spark releases (though this is not the only thing preventing us from upgrading a running query) It is unnecessarily opaque to the user. I'd propose we require offsets to provide a user readable serialization and use that instead. JSON is probably a good option. ## How was this patch tested? Tests were added for KafkaSourceOffset in [KafkaSourceOffsetSuite](external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala) and for LongOffset in [OffsetSuite](sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala) Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. zsxwing marmbrus Author: Tyson Condie <tcondie@gmail.com> Author: Tyson Condie <tcondie@clash.local> Closes #15626 from tcondie/spark-8360.
* [SPARK-18283][STRUCTURED STREAMING][KAFKA] Added test to check whether ↵Tathagata Das2016-11-071-0/+24
| | | | | | | | | | | | | | | default starting offset in latest ## What changes were proposed in this pull request? Added test to check whether default starting offset in latest ## How was this patch tested? new unit test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15778 from tdas/SPARK-18283.
* [SPARK-18212][SS][KAFKA] increase executor poll timeoutcody koeninger2016-11-032-2/+6
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Increase poll timeout to try and address flaky test ## How was this patch tested? Ran existing unit tests Author: cody koeninger <cody@koeninger.org> Closes #15737 from koeninger/SPARK-18212.
* [SPARK-17813][SQL][KAFKA] Maximum data per triggercody koeninger2016-10-272-27/+151
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? maxOffsetsPerTrigger option for rate limiting, proportionally based on volume of different topicpartitions. ## How was this patch tested? Added unit test Author: cody koeninger <cody@koeninger.org> Closes #15527 from koeninger/SPARK-17813.
* [SPARK-17812][SQL][KAFKA] Assign and specific startingOffsets for structured ↵cody koeninger2016-10-217-45/+369
| | | | | | | | | | | | | | | | | | stream ## What changes were proposed in this pull request? startingOffsets takes specific per-topicpartition offsets as a json argument, usable with any consumer strategy assign with specific topicpartitions as a consumer strategy ## How was this patch tested? Unit tests Author: cody koeninger <cody@koeninger.org> Closes #15504 from koeninger/SPARK-17812.