Skip to content
Snippets Groups Projects
  1. Feb 23, 2017
    • Ron Hu's avatar
      [SPARK-17075][SQL] implemented filter estimation · d7e43b61
      Ron Hu authored
      ## What changes were proposed in this pull request?
      
      We traverse predicate and evaluate the logical expressions to compute the selectivity of a FILTER operator.
      
      ## How was this patch tested?
      
      We add a new test suite to test various logical operators.
      
      Author: Ron Hu <ron.hu@huawei.com>
      
      Closes #16395 from ron8hu/filterSelectivity.
      d7e43b61
    • Bryan Cutler's avatar
      [SPARK-14772][PYTHON][ML] Fixed Params.copy method to match Scala implementation · 2f69e3f6
      Bryan Cutler authored
      ## What changes were proposed in this pull request?
      Fixed the PySpark Params.copy method to behave like the Scala implementation.  The main issue was that it did not account for the _defaultParamMap and merged it into the explicitly created param map.
      
      ## How was this patch tested?
      Added new unit test to verify the copy method behaves correctly for copying uid, explicitly created params, and default params.
      
      Author: Bryan Cutler <cutlerb@gmail.com>
      
      Closes #16772 from BryanCutler/pyspark-ml-param_copy-Scala_sync-SPARK-14772.
      2f69e3f6
    • uncleGen's avatar
      [SPARK-16122][DOCS] application environment rest api · d0276245
      uncleGen authored
      ## What changes were proposed in this pull request?
      
      follow up pr of #16949.
      
      ## How was this patch tested?
      
      jenkins
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #17033 from uncleGen/doc-restapi-environment.
      d0276245
    • Carson Wang's avatar
      [SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the... · eff7b408
      Carson Wang authored
      [SPARK-19674][SQL] Ignore driver accumulator updates don't belong to the execution when merging all accumulator updates
      
      ## What changes were proposed in this pull request?
      In SQLListener.getExecutionMetrics, driver accumulator updates don't belong to the execution should be ignored when merging all accumulator updates to prevent NoSuchElementException.
      
      ## How was this patch tested?
      Updated unit test.
      
      Author: Carson Wang <carson.wang@intel.com>
      
      Closes #17009 from carsonwang/FixSQLMetrics.
      eff7b408
    • Kay Ousterhout's avatar
      [SPARK-19684][DOCS] Remove developer info from docs. · f87a6a59
      Kay Ousterhout authored
      This commit moves developer-specific information from the release-
      specific documentation in this repo to the developer tools page on
      the main Spark website. This commit relies on this PR on the
      Spark website: https://github.com/apache/spark-website/pull/33.
      
      srowen
      
      Author: Kay Ousterhout <kayousterhout@gmail.com>
      
      Closes #17018 from kayousterhout/SPARK-19684.
      f87a6a59
    • Wenchen Fan's avatar
      [SPARK-19706][PYSPARK] add Column.contains in pyspark · 4fa4cf1d
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      to be consistent with the scala API, we should also add `contains` to `Column` in pyspark.
      
      ## How was this patch tested?
      
      updated unit test
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #17036 from cloud-fan/pyspark.
      4fa4cf1d
    • Takeshi Yamamuro's avatar
      [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV data · 09ed6e77
      Takeshi Yamamuro authored
      ## What changes were proposed in this pull request?
      This pr added a logic to put malformed tokens into a new field when parsing CSV data  in case of permissive modes. In the current master, if the CSV parser hits these malformed ones, it throws an exception below (and then a job fails);
      ```
      Caused by: java.lang.IllegalArgumentException
      	at java.sql.Date.valueOf(Date.java:143)
      	at org.apache.spark.sql.catalyst.util.DateTimeUtils$.stringToTime(DateTimeUtils.scala:137)
      	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply$mcJ$sp(CSVInferSchema.scala:272)
      	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
      	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
      	at scala.util.Try.getOrElse(Try.scala:79)
      	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:269)
      	at
      ```
      In case that users load large CSV-formatted data, the job failure makes users get some confused. So, this fix set NULL for original columns and put malformed tokens in a new field.
      
      ## How was this patch tested?
      Added tests in `CSVSuite`.
      
      Author: Takeshi Yamamuro <yamamuro@apache.org>
      
      Closes #16928 from maropu/SPARK-18699-2.
      09ed6e77
    • Shixiong Zhu's avatar
      [SPARK-19497][SS] Implement streaming deduplication · 9bf4e2ba
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.
      
      The following cases are supported:
      
      - one or multiple `dropDuplicates()` without aggregation (with or without watermark)
      - `dropDuplicates` before aggregation
      
      Not supported cases:
      
      - `dropDuplicates` after aggregation
      
      Breaking changes:
      - `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.
      
      ## How was this patch tested?
      
      The new unit tests.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #16970 from zsxwing/dedup.
      9bf4e2ba
    • actuaryzhang's avatar
      [SPARK-19682][SPARKR] Issue warning (or error) when subset method "[[" takes vector index · 7bf09433
      actuaryzhang authored
      ## What changes were proposed in this pull request?
      The `[[` method is supposed to take a single index and return a column. This is different from base R which takes a vector index.  We should check for this and issue warning or error when vector index is supplied (which is very likely given the behavior in base R).
      
      Currently I'm issuing a warning message and just take the first element of the vector index. We could change this to an error it that's better.
      
      ## How was this patch tested?
      new tests
      
      Author: actuaryzhang <actuaryzhang10@gmail.com>
      
      Closes #17017 from actuaryzhang/sparkRSubsetter.
      7bf09433
    • Herman van Hovell's avatar
      [SPARK-19459] Support for nested char/varchar fields in ORC · 78eae7e6
      Herman van Hovell authored
      ## What changes were proposed in this pull request?
      This PR is a small follow-up on https://github.com/apache/spark/pull/16804. This PR also adds support for nested char/varchar fields in orc.
      
      ## How was this patch tested?
      I have added a regression test to the OrcSourceSuite.
      
      Author: Herman van Hovell <hvanhovell@databricks.com>
      
      Closes #17030 from hvanhovell/SPARK-19459-follow-up.
      78eae7e6
    • Takeshi Yamamuro's avatar
      [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of decimal column · 93aa4271
      Takeshi Yamamuro authored
      ## What changes were proposed in this pull request?
      This pr fixed a class-cast exception below;
      ```
      scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect()
       java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number
      	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141)
      	at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58)
      	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514)
      	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
      	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171)
      	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187)
      	at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181)
      	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151)
      	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
      	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109)
      	at
      ```
      This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`.
      
      ## How was this patch tested?
      Added a test in `DataFrameSuite`.
      
      Author: Takeshi Yamamuro <yamamuro@apache.org>
      
      Closes #17028 from maropu/SPARK-19691.
      93aa4271
  2. Feb 22, 2017
    • Takeshi Yamamuro's avatar
      [SPARK-19695][SQL] Throw an exception if a `columnNameOfCorruptRecord` field... · 769aa0f1
      Takeshi Yamamuro authored
      [SPARK-19695][SQL] Throw an exception if a `columnNameOfCorruptRecord` field violates requirements in json formats
      
      ## What changes were proposed in this pull request?
      This pr comes from #16928 and fixed a json behaviour along with the CSV one.
      
      ## How was this patch tested?
      Added tests in `JsonSuite`.
      
      Author: Takeshi Yamamuro <yamamuro@apache.org>
      
      Closes #17023 from maropu/SPARK-19695.
      769aa0f1
    • uncleGen's avatar
      [SPARK-16122][CORE] Add rest api for job environment · 66c4b79a
      uncleGen authored
      ## What changes were proposed in this pull request?
      
      add rest api for job environment.
      
      ## How was this patch tested?
      
      existing ut.
      
      Author: uncleGen <hustyugm@gmail.com>
      
      Closes #16949 from uncleGen/SPARK-16122.
      66c4b79a
    • pj.fanning's avatar
      [SPARK-15615][SQL] Add an API to load DataFrame from Dataset[String] storing JSON · d3147502
      pj.fanning authored
      ## What changes were proposed in this pull request?
      
      SPARK-15615 proposes replacing the sqlContext.read.json(rdd) with a dataset equivalent.
      SPARK-15463 adds a CSV API for reading from Dataset[String] so this keeps the API consistent.
      I am deprecating the existing RDD based APIs.
      
      ## How was this patch tested?
      
      There are existing tests. I left most tests to use the existing APIs as they delegate to the new json API.
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: pj.fanning <pj.fanning@workday.com>
      Author: PJ Fanning <pjfanning@users.noreply.github.com>
      
      Closes #16895 from pjfanning/SPARK-15615.
      d3147502
    • Xiao Li's avatar
      [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In Parser · dc005ed5
      Xiao Li authored
      ### What changes were proposed in this pull request?
      
      Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization.
      
      ### How was this patch tested?
      
      Added a test case.
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #16988 from gatorsmile/resolveRepartition.
      dc005ed5
    • Marcelo Vanzin's avatar
      [SPARK-19554][UI,YARN] Allow SHS URL to be used for tracking in YARN RM. · 4661d30b
      Marcelo Vanzin authored
      Allow an application to use the History Server URL as the tracking
      URL in the YARN RM, so there's still a link to the web UI somewhere
      in YARN even if the driver's UI is disabled. This is useful, for
      example, if an admin wants to disable the driver UI by default for
      applications, since it's harder to secure it (since it involves non
      trivial ssl certificate and auth management that admins may not want
      to expose to user apps).
      
      This needs to be opt-in, because of the way the YARN proxy works, so
      a new configuration was added to enable the option.
      
      The YARN RM will proxy requests to live AMs instead of redirecting
      the client, so pages in the SHS UI will not render correctly since
      they'll reference invalid paths in the RM UI. The proxy base support
      in the SHS cannot be used since that would prevent direct access to
      the SHS.
      
      So, to solve this problem, for the feature to work end-to-end, a new
      YARN-specific filter was added that detects whether the requests come
      from the proxy and redirects the client appropriatly. The SHS admin has
      to add this filter manually if they want the feature to work.
      
      Tested with new unit test, and by running with the documented configuration
      set in a test cluster. Also verified the driver UI is used when it's
      enabled.
      
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #16946 from vanzin/SPARK-19554.
      4661d30b
    • hyukjinkwon's avatar
      [SPARK-19666][SQL] Skip a property without getter in Java schema inference and... · 37112fcf
      hyukjinkwon authored
      [SPARK-19666][SQL] Skip a property without getter in Java schema inference and allow empty bean in encoder creation
      
      ## What changes were proposed in this pull request?
      
      This PR proposes to fix two.
      
      **Skip a property without a getter in beans**
      
      Currently, if we use a JavaBean without the getter as below:
      
      ```java
      public static class BeanWithoutGetter implements Serializable {
        private String a;
      
        public void setA(String a) {
          this.a = a;
        }
      }
      
      BeanWithoutGetter bean = new BeanWithoutGetter();
      List<BeanWithoutGetter> data = Arrays.asList(bean);
      spark.createDataFrame(data, BeanWithoutGetter.class).show();
      ```
      
      - Before
      
      It throws an exception as below:
      
      ```
      java.lang.NullPointerException
      	at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465)
      	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126)
      	at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125)
      ```
      
      - After
      
      ```
      ++
      ||
      ++
      ||
      ++
      ```
      
      **Supports empty bean in encoder creation**
      
      ```java
      public static class EmptyBean implements Serializable {}
      
      EmptyBean bean = new EmptyBean();
      List<EmptyBean> data = Arrays.asList(bean);
      spark.createDataset(data, Encoders.bean(EmptyBean.class)).show();
      ```
      
      - Before
      
      throws an exception as below:
      
      ```
      java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant
      	at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436)
      	at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341)
      ```
      
      - After
      
      ```
      ++
      ||
      ++
      ||
      ++
      ```
      
      ## How was this patch tested?
      
      Unit test in `JavaDataFrameSuite`.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #17013 from HyukjinKwon/SPARK-19666.
      37112fcf
    • wm624@hotmail.com's avatar
      [SPARK-19616][SPARKR] weightCol and aggregationDepth should be improved for some SparkR APIs · 1f86e795
      wm624@hotmail.com authored
      ## What changes were proposed in this pull request?
      
      This is a follow-up PR of #16800
      
      When doing SPARK-19456, we found that "" should be consider a NULL column name and should not be set. aggregationDepth should be exposed as an expert parameter.
      
      ## How was this patch tested?
      Existing tests.
      
      Author: wm624@hotmail.com <wm624@hotmail.com>
      
      Closes #16945 from wangmiao1981/svc.
      1f86e795
    • 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
    • Bogdan Raducanu's avatar
      [SPARK-13721][SQL] Make GeneratorOuter unresolved. · 10c566cc
      Bogdan Raducanu authored
      ## What changes were proposed in this pull request?
      
      This is a small change to make GeneratorOuter always unresolved. It is mostly no-op change but makes it more clear since GeneratorOuter shouldn't survive analysis phase.
      This requires also handling in ResolveAliases rule.
      
      ## How was this patch tested?
      Existing generator tests.
      
      Author: Bogdan Raducanu <bogdan@databricks.com>
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #17026 from bogdanrdc/PR16958.
      10c566cc
    • Zheng RuiFeng's avatar
      [SPARK-19679][ML] Destroy broadcasted object without blocking · bf7bb497
      Zheng RuiFeng authored
      ## What changes were proposed in this pull request?
      Destroy broadcasted object without blocking
      use `find mllib -name '*.scala' | xargs -i bash -c 'egrep "destroy" -n {} && echo {}'`
      
      ## How was this patch tested?
      existing tests
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #17016 from zhengruifeng/destroy_without_block.
      bf7bb497
    • Zheng RuiFeng's avatar
      [SPARK-19694][ML] Add missing 'setTopicDistributionCol' for LDAModel · ef3c7353
      Zheng RuiFeng authored
      ## What changes were proposed in this pull request?
      Add missing 'setTopicDistributionCol' for LDAModel
      ## How was this patch tested?
      existing tests
      
      Author: Zheng RuiFeng <ruifengz@foxmail.com>
      
      Closes #17021 from zhengruifeng/lda_outputCol.
      ef3c7353
  3. Feb 21, 2017
    • Xiao Li's avatar
      [SPARK-19670][SQL][TEST] Enable Bucketed Table Reading and Writing Testing Without Hive Support · 1a45d2b2
      Xiao Li authored
      ### What changes were proposed in this pull request?
      Bucketed table reading and writing does not need Hive support. We can move the test cases from `sql/hive` to `sql/core`. After this PR, we can improve the test case coverage. Bucket table reading and writing can be tested with and without Hive support.
      
      ### How was this patch tested?
      N/A
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #17004 from gatorsmile/mvTestCaseForBuckets.
      1a45d2b2
    • Marcelo Vanzin's avatar
      [SPARK-19652][UI] Do auth checks for REST API access. · 17d83e1e
      Marcelo Vanzin authored
      The REST API has a security filter that performs auth checks
      based on the UI root's security manager. That works fine when
      the UI root is the app's UI, but not when it's the history server.
      
      In the SHS case, all users would be allowed to see all applications
      through the REST API, even if the UI itself wouldn't be available
      to them.
      
      This change adds auth checks for each app access through the API
      too, so that only authorized users can see the app's data.
      
      The change also modifies the existing security filter to use
      `HttpServletRequest.getRemoteUser()`, which is used in other
      places. That is not necessarily the same as the principal's
      name; for example, when using Hadoop's SPNEGO auth filter,
      the remote user strips the realm information, which then matches
      the user name registered as the owner of the application.
      
      I also renamed the UIRootFromServletContext trait to a more generic
      name since I'm using it to store more context information now.
      
      Tested manually with an authentication filter enabled.
      
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #16978 from vanzin/SPARK-19652.
      17d83e1e
    • Kent Yao's avatar
      [SPARK-19626][YARN] Using the correct config to set credentials update time · 7363dde6
      Kent Yao authored
      ## What changes were proposed in this pull request?
      
      In https://github.com/apache/spark/pull/14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` .
      
      This PR fixes this mistake.
      
      ## How was this patch tested?
      
      existing test
      
      cc jerryshao vanzin
      
      Author: Kent Yao <yaooqinn@hotmail.com>
      
      Closes #16955 from yaooqinn/cred_update.
      7363dde6
    • Yuhao Yang's avatar
      [SPARK-19337][ML][DOC] Documentation and examples for LinearSVC · 280afe0e
      Yuhao Yang authored
      ## What changes were proposed in this pull request?
      
      Documentation and examples (Java, scala, python, R) for LinearSVC
      
      ## How was this patch tested?
      local doc generation
      
      Author: Yuhao Yang <yuhao.yang@intel.com>
      
      Closes #16968 from hhbyyh/mlsvmdoc.
      280afe0e
  4. 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
    • Liang-Chi Hsieh's avatar
      [SPARK-19508][CORE] Improve error message when binding service fails · 33941914
      Liang-Chi Hsieh authored
      ## What changes were proposed in this pull request?
      
      Utils provides a helper function to bind service on port. This function can bind the service to a random free port. However, if the binding fails on a random free port, the retrying and final exception messages look confusing.
      
          17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1.
          17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1.
          17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1.
          17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1.
          17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1.
          ...
          17/02/06 16:25:43 ERROR SparkContext: Error initializing SparkContext.
          java.net.BindException: Can't assign requested address: Service 'sparkDriver' failed after 16 retries (starting from 0)! Consider explicitly setting the appropriate port for the service 'sparkDriver' (for example spark.ui.port for SparkUI) to an available port or increasing spark.port.maxRetries.
      
      ## How was this patch tested?
      
      Jenkins tests.
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: Liang-Chi Hsieh <viirya@gmail.com>
      
      Closes #16851 from viirya/better-log-message.
      Unverified
      33941914
    • windpiger's avatar
      [SPARK-19669][HOTFIX][SQL] sessionState access privileges compiled failed in TestSQLContext · 73f06556
      windpiger authored
      ## What changes were proposed in this pull request?
      
      In [SPARK-19669](https://github.com/apache/spark/commit/0733a54a4517b82291efed9ac7f7407d9044593c) change the sessionState access privileges from private to public, this lead to the compile failed in TestSQLContext
      
      this pr is a hotfix for this.
      
      ## How was this patch tested?
      N/A
      
      Author: windpiger <songjun@outlook.com>
      
      Closes #17008 from windpiger/hotfixcompile.
      73f06556
    • Reynold Xin's avatar
      [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a few other functions · 0733a54a
      Reynold Xin authored
      ## What changes were proposed in this pull request?
      To ease debugging, most of Spark SQL internals have public level visibility. Two of the most important internal states, sharedState and sessionState, however, are package private. It would make more sense to open these up as well with clear documentation that they are internal.
      
      In addition, users currently have way to set active/default SparkSession, but no way to actually get them back. We should open those up as well.
      
      ## How was this patch tested?
      N/A - only visibility change.
      
      Author: Reynold Xin <rxin@databricks.com>
      
      Closes #17002 from rxin/SPARK-19669.
      0733a54a
    • Xiao Li's avatar
      [SPARK-15453][SQL][FOLLOW-UP] FileSourceScanExec to extract `outputOrdering` information · ead4ba0e
      Xiao Li authored
      ### What changes were proposed in this pull request?
      `outputOrdering` is also dependent on whether the bucket has more than one files. The test cases fail when we try to move them to sql/core. This PR is to fix the test cases introduced in https://github.com/apache/spark/pull/14864 and add a test case to verify [the related logics](https://github.com/tejasapatil/spark/blob/070c24994747c0479fb2520774ede27ff1cf8cac/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L197-L206).
      
      ### How was this patch tested?
      N/A
      
      Author: Xiao Li <gatorsmile@gmail.com>
      
      Closes #16994 from gatorsmile/bucketingTS.
      ead4ba0e
    • Sean Owen's avatar
      [SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala API · d0ecca60
      Sean Owen authored
      ## What changes were proposed in this pull request?
      
      Use `BytesWritable.copyBytes`, not `getBytes`, because `getBytes` returns the underlying array, which may be reused when repeated reads don't need a different size, as is the case with binaryRecords APIs
      
      ## How was this patch tested?
      
      Existing tests
      
      Author: Sean Owen <sowen@cloudera.com>
      
      Closes #16974 from srowen/SPARK-19646.
      Unverified
      d0ecca60
  5. Feb 19, 2017
  6. Feb 18, 2017
    • Ala Luszczak's avatar
      [SPARK-19447] Make Range operator generate "recordsRead" metric · b486ffc8
      Ala Luszczak authored
      ## What changes were proposed in this pull request?
      
      The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite.
      
      ## How was this patch tested?
      
      Unit tests.
      
      Author: Ala Luszczak <ala@databricks.com>
      
      Closes #16960 from ala/range-records-read.
      b486ffc8
    • jinxing's avatar
      [SPARK-19263] DAGScheduler should avoid sending conflicting task set. · 729ce370
      jinxing authored
      In current `DAGScheduler handleTaskCompletion` code, when event.reason is `Success`, it will first do `stage.pendingPartitions -= task.partitionId`, which maybe a bug when `FetchFailed` happens.
      
      **Think about below**
      
      1.  Stage 0 runs and generates shuffle output data.
      2. Stage 1 reads the output from stage 0 and generates more shuffle data. It has two tasks: ShuffleMapTask1 and ShuffleMapTask2, and these tasks are launched on executorA.
      3. ShuffleMapTask1 fails to fetch blocks locally and sends a FetchFailed to the driver. The driver marks executorA as lost and updates failedEpoch;
      4. The driver resubmits stage 0 so the missing output can be re-generated, and then once it completes, resubmits stage 1 with ShuffleMapTask1x and ShuffleMapTask2x.
      5. ShuffleMapTask2 (from the original attempt of stage 1) successfully finishes on executorA and sends Success back to driver. This causes DAGScheduler::handleTaskCompletion to remove partition 2 from stage.pendingPartitions (line 1149), but it does not add the partition to the set of output locations (line 1192), because the task’s epoch is less than the failure epoch for the executor (because of the earlier failure on executor A)
      6. ShuffleMapTask1x successfully finishes on executorB, causing the driver to remove partition 1 from stage.pendingPartitions. Combined with the previous step, this means that there are no more pending partitions for the stage, so the DAGScheduler marks the stage as finished (line 1196). However, the shuffle stage is not available (line 1215) because the completion for ShuffleMapTask2 was ignored because of its epoch, so the DAGScheduler resubmits the stage.
      7. ShuffleMapTask2x is still running, so when TaskSchedulerImpl::submitTasks is called for the re-submitted stage, it throws an error, because there’s an existing active task set
      
      **In this fix**
      
      If a task completion is from a previous stage attempt and the epoch is too low
      (i.e., it was from a failed executor), don't remove the corresponding partition
      from pendingPartitions.
      
      Author: jinxing <jinxing@meituan.com>
      Author: jinxing <jinxing6042@126.com>
      
      Closes #16620 from jinxing64/SPARK-19263.
      729ce370
Loading