Skip to content
Snippets Groups Projects
  1. May 30, 2017
  2. May 29, 2017
  3. May 19, 2017
  4. May 17, 2017
  5. May 16, 2017
    • Yash Sharma's avatar
      [SPARK-20140][DSTREAM] Remove hardcoded kinesis retry wait and max retries · 38f4e869
      Yash Sharma authored
      ## What changes were proposed in this pull request?
      
      The pull requests proposes to remove the hardcoded values for Amazon Kinesis - MIN_RETRY_WAIT_TIME_MS, MAX_RETRIES.
      
      This change is critical for kinesis checkpoint recovery when the kinesis backed rdd is huge.
      Following happens in a typical kinesis recovery :
      - kinesis throttles large number of requests while recovering
      - retries in case of throttling are not able to recover due to the small wait period
      - kinesis throttles per second, the wait period should be configurable for recovery
      
      The patch picks the spark kinesis configs from:
      - spark.streaming.kinesis.retry.wait.time
      - spark.streaming.kinesis.retry.max.attempts
      
      Jira : https://issues.apache.org/jira/browse/SPARK-20140
      
      ## How was this patch tested?
      
      Modified the KinesisBackedBlockRDDSuite.scala to run kinesis tests with the modified configurations. Wasn't able to test the patch with actual throttling.
      
      Author: Yash Sharma <ysharma@atlassian.com>
      
      Closes #17467 from yssharma/ysharma/spark-kinesis-retries.
      38f4e869
  6. May 11, 2017
  7. May 10, 2017
    • Xianyang Liu's avatar
      [MINOR][BUILD] Fix lint-java breaks. · fcb88f92
      Xianyang Liu authored
      ## What changes were proposed in this pull request?
      
      This PR proposes to fix the lint-breaks as below:
      ```
      [ERROR] src/main/java/org/apache/spark/unsafe/Platform.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[45,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[62,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[78,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[92,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[102,25] (naming) MethodName: Method name 'Once' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
      [ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.api.java.JavaDStream.
      ```
      
      after:
      ```
      dev/lint-java
      Checkstyle checks passed.
      ```
      [Test Result](https://travis-ci.org/ConeyLiu/spark/jobs/229666169)
      
      ## How was this patch tested?
      
      Travis CI
      
      Author: Xianyang Liu <xianyang.liu@intel.com>
      
      Closes #17890 from ConeyLiu/codestyle.
      fcb88f92
  8. May 07, 2017
    • Xiao Li's avatar
      [SPARK-20557][SQL] Support JDBC data type Time with Time Zone · cafca54c
      Xiao Li authored
      ### What changes were proposed in this pull request?
      
      This PR is to support JDBC data type TIME WITH TIME ZONE. It can be converted to TIMESTAMP
      
      In addition, before this PR, for unsupported data types, we simply output the type number instead of the type name.
      
      ```
      java.sql.SQLException: Unsupported type 2014
      ```
      After this PR, the message is like
      ```
      java.sql.SQLException: Unsupported type TIMESTAMP_WITH_TIMEZONE
      ```
      
      - Also upgrade the H2 version to `1.4.195` which has the type fix for "TIMESTAMP WITH TIMEZONE". However, it is not fully supported. Thus, we capture the exception, but we still need it to partially test the support of "TIMESTAMP WITH TIMEZONE", because Docker tests are not regularly run.
      
      ### How was this patch tested?
      Added test cases.
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #17835 from gatorsmile/h2.
      cafca54c
  9. May 05, 2017
  10. Apr 28, 2017
  11. Apr 27, 2017
    • Shixiong Zhu's avatar
      [SPARK-20452][SS][KAFKA] Fix a potential ConcurrentModificationException for batch Kafka DataFrame · 823baca2
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      Cancel a batch Kafka query but one of task cannot be cancelled, and rerun the same DataFrame may cause ConcurrentModificationException because it may launch two tasks sharing the same group id.
      
      This PR always create a new consumer when `reuseKafkaConsumer = false` to avoid ConcurrentModificationException. It also contains other minor fixes.
      
      ## How was this patch tested?
      
      Jenkins.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #17752 from zsxwing/kafka-fix.
      823baca2
    • Shixiong Zhu's avatar
      [SPARK-20461][CORE][SS] Use UninterruptibleThread for Executor and fix the... · 01c999e7
      Shixiong Zhu authored
      [SPARK-20461][CORE][SS] Use UninterruptibleThread for Executor and fix the potential hang in CachedKafkaConsumer
      
      ## What changes were proposed in this pull request?
      
      This PR changes Executor's threads to `UninterruptibleThread` so that we can use `runUninterruptibly` in `CachedKafkaConsumer`. However, this is just best effort to avoid hanging forever. If the user uses`CachedKafkaConsumer` in another thread (e.g., create a new thread or Future), the potential hang may still happen.
      
      ## How was this patch tested?
      
      The new added test.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #17761 from zsxwing/int.
      01c999e7
  12. Apr 24, 2017
  13. Apr 13, 2017
    • Yash Sharma's avatar
      [SPARK-20189][DSTREAM] Fix spark kinesis testcases to remove deprecated... · ec68d8f8
      Yash Sharma authored
      [SPARK-20189][DSTREAM] Fix spark kinesis testcases to remove deprecated 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.
      ec68d8f8
  14. Apr 10, 2017
    • Sean Owen's avatar
      [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish... · a26e3ed5
      Sean Owen authored
      [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish 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.
      a26e3ed5
  15. Apr 05, 2017
    • Tathagata Das's avatar
      [SPARK-20209][SS] Execute next trigger immediately if previous batch took... · dad499f3
      Tathagata Das authored
      [SPARK-20209][SS] Execute next trigger immediately if previous batch took 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.
      dad499f3
  16. Mar 24, 2017
    • Adam Budde's avatar
      [SPARK-19911][STREAMING] Add builder interface for Kinesis DStreams · 707e5018
      Adam Budde authored
      ## 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.
      707e5018
  17. Mar 23, 2017
    • Tyson Condie's avatar
      [SPARK-19876][SS][WIP] OneTime Trigger Executor · 746a558d
      Tyson Condie authored
      ## 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.
      746a558d
  18. Mar 16, 2017
    • Liwei Lin's avatar
      [SPARK-19721][SS] Good error message for version mismatch in log files · 2ea214dd
      Liwei Lin authored
      ## Problem
      
      There are several places where we write out version identifiers in various logs for structured streaming (usually `v1`). However, in the places where we check for this, we throw a confusing error message.
      
      ## What changes were proposed in this pull request?
      
      This patch made two major changes:
      1. added a `parseVersion(...)` method, and based on this method, fixed the following places the way they did version checking (no other place needed to do this checking):
      ```
      HDFSMetadataLog
        - CompactibleFileStreamLog  ------------> fixed with this patch
          - FileStreamSourceLog  ---------------> inherited the fix of `CompactibleFileStreamLog`
          - FileStreamSinkLog  -----------------> inherited the fix of `CompactibleFileStreamLog`
        - OffsetSeqLog  ------------------------> fixed with this patch
        - anonymous subclass in KafkaSource  ---> fixed with this patch
      ```
      
      2. changed the type of `FileStreamSinkLog.VERSION`, `FileStreamSourceLog.VERSION` etc. from `String` to `Int`, so that we can identify newer versions via `version > 1` instead of `version != "v1"`
          - note this didn't break any backwards compatibility -- we are still writing out `"v1"` and reading back `"v1"`
      
      ## Exception message with this patch
      ```
      java.lang.IllegalStateException: Failed to read log file /private/var/folders/nn/82rmvkk568sd8p3p8tb33trw0000gn/T/spark-86867b65-0069-4ef1-b0eb-d8bd258ff5b8/0. UnsupportedLogVersion: maximum supported log version is v1, but encountered v99. The log file was produced by a newer version of Spark and cannot be read by this version. Please upgrade.
      	at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:202)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:78)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:75)
      	at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:133)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite.withTempDir(OffsetSeqLogSuite.scala:26)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply$mcV$sp(OffsetSeqLogSuite.scala:75)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
      	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
      	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
      	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
      ```
      
      ## How was this patch tested?
      
      unit tests
      
      Author: Liwei Lin <lwlin7@gmail.com>
      
      Closes #17070 from lw-lin/better-msg.
      2ea214dd
  19. Mar 12, 2017
    • uncleGen's avatar
      [SPARK-19853][SS] uppercase kafka topics fail when startingOffsets are SpecificOffsets · 0a4d06a7
      uncleGen authored
      When using the KafkaSource with Structured Streaming, consumer assignments are not what the user expects if startingOffsets is set to an explicit set of topics/partitions in JSON where the topic(s) happen to have uppercase characters. When StartingOffsets is constructed, the original string value from options is transformed toLowerCase to make matching on "earliest" and "latest" case insensitive. However, the toLowerCase JSON is passed to SpecificOffsets for the terminal condition, so topic names may not be what the user intended by the time assignments are made with the underlying KafkaConsumer.
      
      KafkaSourceProvider.scala:
      ```
      val startingOffsets = caseInsensitiveParams.get(STARTING_OFFSETS_OPTION_KEY).map(_.trim.toLowerCase) match {
          case Some("latest") => LatestOffsets
          case Some("earliest") => EarliestOffsets
          case Some(json) => SpecificOffsets(JsonUtils.partitionOffsets(json))
          case None => LatestOffsets
        }
      ```
      
      Thank cbowden for reporting.
      
      Jenkins
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #17209 from uncleGen/SPARK-19853.
      0a4d06a7
  20. Mar 09, 2017
  21. Mar 06, 2017
    • Tyson Condie's avatar
      [SPARK-19719][SS] Kafka writer for both structured streaming and batch queires · b0a5cd89
      Tyson Condie authored
      ## 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.
      b0a5cd89
    • Gaurav's avatar
      [SPARK-19304][STREAMING][KINESIS] fix kinesis slow checkpoint recovery · 46a64d1e
      Gaurav authored
      ## 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.
      46a64d1e
  22. Feb 27, 2017
    • hyukjinkwon's avatar
      [MINOR][BUILD] Fix lint-java breaks in Java · 4ba9c6c4
      hyukjinkwon authored
      ## 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.
      4ba9c6c4
  23. Feb 22, 2017
    • Adam Budde's avatar
      [SPARK-19405][STREAMING] Support for cross-account Kinesis reads via STS · e4065376
      Adam Budde authored
      - 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.
      e4065376
  24. Feb 20, 2017
    • hyukjinkwon's avatar
      [SPARK-18922][TESTS] Fix new test failures on Windows due to path and resource not closed · 17b93b5f
      hyukjinkwon authored
      ## 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.
      Unverified
      17b93b5f
  25. Feb 19, 2017
  26. Feb 17, 2017
  27. Feb 16, 2017
    • Sean Owen's avatar
      [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support · 0e240549
      Sean Owen authored
      - 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.
      Unverified
      0e240549
  28. Feb 14, 2017
    • sureshthalamati's avatar
      [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the... · f48c5a57
      sureshthalamati authored
      [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the 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.
      f48c5a57
  29. Feb 13, 2017
    • Liwei Lin's avatar
      [SPARK-19564][SPARK-19559][SS][KAFKA] KafkaOffsetReader's consumers should not be in the same group · 2bdbc870
      Liwei Lin authored
      ## 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-.
      2bdbc870
  30. Feb 07, 2017
    • Tyson Condie's avatar
      [SPARK-18682][SS] Batch Source for Kafka · 8df44440
      Tyson Condie authored
      ## 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.
      8df44440
  31. Jan 25, 2017
    • Takeshi YAMAMURO's avatar
      [SPARK-18020][STREAMING][KINESIS] Checkpoint SHARD_END to finish reading closed shards · 256a3a80
      Takeshi YAMAMURO authored
      ## 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.
      256a3a80
  32. Jan 20, 2017
    • sureshthalamati's avatar
      [SPARK-14536][SQL] fix to handle null value in array type column for postgres. · f174cdc7
      sureshthalamati authored
      ## What changes were proposed in this pull request?
      
      JDBC  read  is failing with  NPE due to missing null value check for array data type if the source table has null values in the array type column.  For null values Resultset.getArray()  returns null.
      This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object.
      ## How was this patch tested?
      
      Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop.
      
      Author: sureshthalamati <suresh.thalamati@gmail.com>
      
      Closes #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
      f174cdc7
  33. Jan 18, 2017
  34. Jan 15, 2017
  35. Jan 10, 2017
    • hyukjinkwon's avatar
      [SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed due... · 4e27578f
      hyukjinkwon authored
      [SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed 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.
      Unverified
      4e27578f
  36. Jan 04, 2017
    • Niranjan Padmanabhan's avatar
      [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo · a1e40b1f
      Niranjan Padmanabhan authored
      ## 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.
      Unverified
      a1e40b1f
Loading