Commit graph

23621 commits

Author SHA1 Message Date
Liupengcheng eb42bb493b [SPARK-26529] Add debug logs for confArchive when preparing local resource
## What changes were proposed in this pull request?

Currently, `Client#createConfArchive` do not handle IOException, and some detail info is not provided in logs. Sometimes, this may delay the time of locating the root cause of io error.
This PR will add debug logs for confArchive when preparing local resource.

## How was this patch tested?

unittest

Closes #23444 from liupc/Add-logs-for-IOException-when-preparing-local-resource.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-09 10:39:25 +08:00
Gengliang Wang 311f32f37f [SPARK-26571][SQL] Update Hive Serde mapping with canonical name of Parquet and Orc FileFormat
## What changes were proposed in this pull request?

Currently Spark table maintains Hive catalog storage format, so that Hive client can read it.  In `HiveSerDe.scala`, Spark uses a mapping from its data source to HiveSerde. The mapping is old, we need to update with latest canonical name of Parquet and Orc FileFormat.

Otherwise the following queries will result in wrong Serde value in Hive table(default value `org.apache.hadoop.mapred.SequenceFileInputFormat`), and Hive client will fail to read the output table:
```
df.write.format("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat").saveAsTable(..)
```

```
df.write.format("org.apache.spark.sql.execution.datasources.orc.OrcFileFormat").saveAsTable(..)
```

This minor PR is to fix the mapping.

## How was this patch tested?

Unit test.

Closes #23491 from gengliangwang/fixHiveSerdeMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-09 10:18:33 +08:00
Imran Rashid 32515d205a [SPARK-26349][PYSPARK] Forbid insecure py4j gateways
Spark always creates secure py4j connections between java and python,
but it also allows users to pass in their own connection. This ensures
that even passed in connections are secure.

Added test cases verifying the failure with a (mocked) insecure gateway.

This is closely related to SPARK-26019, but this entirely forbids the
insecure connection, rather than creating the "escape-hatch".

Closes #23441 from squito/SPARK-26349.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-01-08 11:26:36 -08:00
“attilapiros” e103c4a5e7 [SPARK-24920][CORE] Allow sharing Netty's memory pool allocators
## What changes were proposed in this pull request?

Introducing shared polled ByteBuf allocators.
This feature can be enabled via the "spark.network.sharedByteBufAllocators.enabled" configuration.

When it is on then only two pooled ByteBuf allocators are created:
- one for transport servers where caching is allowed and
- one for transport clients where caching is disabled

This way the cache allowance remains as before.
Both shareable pools are created with numCores parameter set to 0 (which defaults to the available processors) as conf.serverThreads() and conf.clientThreads() are module dependant and the lazy creation of this allocators would lead to unpredicted behaviour.

When "spark.network.sharedByteBufAllocators.enabled" is false then a new allocator is created for every transport client and server separately as was before this PR.

## How was this patch tested?

Existing unit tests.

Closes #23278 from attilapiros/SPARK-24920.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-08 13:11:11 -06:00
Marcelo Vanzin 2783e4c45f [SPARK-24522][UI] Create filter to apply HTTP security checks consistently.
Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.

This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.

The new filter is added at the end of the filter chain, because authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.

As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.

Tested with added unit tests and in a history server with SPNEGO auth
configured.

Closes #23302 from vanzin/SPARK-24522.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-01-08 11:25:33 -06:00
“attilapiros” c101182b10 [SPARK-26002][SQL] Fix day of year calculation for Julian calendar days
## What changes were proposed in this pull request?

Fixing leap year calculations for date operators (year/month/dayOfYear) where the Julian calendars are used (before 1582-10-04). In a Julian calendar every years which are multiples of 4 are leap years (there is no extra exception for years multiples of 100).

## How was this patch tested?

With a unit test ("SPARK-26002: correct day of year calculations for Julian calendar years") which focuses to these corner cases.

Manually:

```
scala> sql("select year('1500-01-01')").show()

+------------------------------+
|year(CAST(1500-01-01 AS DATE))|
+------------------------------+
|                          1500|
+------------------------------+

scala> sql("select dayOfYear('1100-01-01')").show()

+-----------------------------------+
|dayofyear(CAST(1100-01-01 AS DATE))|
+-----------------------------------+
|                                  1|
+-----------------------------------+
```

Closes #23000 from attilapiros/julianOffByDays.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-09 01:24:47 +08:00
liuxian b7113822d5 [MINOR][WEBUI] Modify the name of the column named "shuffle spill" in the StagePage
## What changes were proposed in this pull request?

![default](https://user-images.githubusercontent.com/24688163/50752687-16463f00-128a-11e9-8ee3-4d156f7631f6.png)
For this DAG, it has no shuffle operation, only sorting, and sorting leads to spill.

![default](https://user-images.githubusercontent.com/24688163/50752974-0f6bfc00-128b-11e9-9362-a0f440e02359.png)
So I think the name of the column named "shuffle spill" is not all right  in the StagePage

## How was this patch tested?
Manual testing

Closes #23483 from 10110346/shufflespillwebui.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-08 10:45:23 -06:00
Wenchen Fan 72a572ffd6 [SPARK-26323][SQL] Scala UDF should still check input types even if some inputs are of type Any
## What changes were proposed in this pull request?

For Scala UDF, when checking input nullability, we will skip inputs with type `Any`, and only check the inputs that provide nullability info.

We should do the same for checking input types.

## How was this patch tested?

new tests

Closes #23275 from cloud-fan/udf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-08 22:44:33 +08:00
Yuming Wang 29a7d2da44 [SPARK-24196][SQL] Implement Spark's own GetSchemasOperation
## What changes were proposed in this pull request?

This PR fix SQL Client tools can't show DBs by implementing Spark's own `GetSchemasOperation`.

## How was this patch tested?
unit tests and manual tests
![image](https://user-images.githubusercontent.com/5399861/47782885-3dd5d400-dd3c-11e8-8586-59a8c15c7020.png)
![image](https://user-images.githubusercontent.com/5399861/47782899-4928ff80-dd3c-11e8-9d2d-ba9580ba4301.png)

Closes #22903 from wangyum/SPARK-24196.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-07 18:59:43 -08:00
Dongjoon Hyun 6f35ede31c
[SPARK-26554][BUILD][FOLLOWUP] Use GitHub instead of GitBox to check HEADER
## What changes were proposed in this pull request?

This PR uses GitHub repository instead of GitBox because GitHub repo returns HTTP header status correctly.

## How was this patch tested?

Manual.

```
$ ./do-release-docker.sh -d /tmp/test -n
Branch [branch-2.4]:
Current branch version is 2.4.1-SNAPSHOT.
Release [2.4.1]:
RC # [1]:
This is a dry run. Please confirm the ref that will be built for testing.
Ref [v2.4.1-rc1]:
```

Closes #23482 from dongjoon-hyun/SPARK-26554-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-07 17:54:05 -08:00
Adrian Tanase 5fb5a0292d [MINOR][K8S] add missing docs for podTemplateContainerName properties
## What changes were proposed in this pull request?

Adding docs for an enhancement that came in late in this PR: #22146
Currently the docs state that we're going to use the first container in a pod template, which was the implementation for some time, until it was improved with 2 new properties.

## How was this patch tested?

I tested that the properties work by combining pod templates with client-mode and a simple pod template.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #23155 from aditanase/k8s-readme.

Authored-by: Adrian Tanase <atanase@adobe.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-07 19:03:38 -06:00
Hyukjin Kwon 5102ccc4ab [SPARK-26339][SQL][FOLLOW-UP] Issue warning instead of throwing an exception for underscore files
## What changes were proposed in this pull request?

The PR https://github.com/apache/spark/pull/23446 happened to introduce a behaviour change - empty dataframes can't be read anymore from underscore files. It looks controversial to allow or disallow this case so this PR targets to fix to issue warning instead of throwing an exception to be more conservative.

**Before**

```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
  file:/.../_tmp1;
  at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
  at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:651)
  at org.apache.spark.sql.DataFrameReader.parquet(DataFrameReader.scala:635)
  ... 49 elided

scala> spark.read.text("_tmp*").show()
org.apache.spark.sql.AnalysisException: All paths were ignored:
file:/.../_tmp
  file:/.../_tmp1;
  at org.apache.spark.sql.execution.datasources.DataSource.checkAndGlobPathIfNecessary(DataSource.scala:570)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:360)
  at org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:231)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:219)
  at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:723)
  at org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:695)
  ... 49 elided
```

**After**

```scala
scala> spark.read.schema("a int").parquet("_tmp*").show()
19/01/07 15:14:43 WARN DataSource: All paths were ignored:
  file:/.../_tmp
  file:/.../_tmp1
+---+
|  a|
+---+
+---+

scala> spark.read.text("_tmp*").show()
19/01/07 15:14:51 WARN DataSource: All paths were ignored:
  file:/.../_tmp
  file:/.../_tmp1
+-----+
|value|
+-----+
+-----+
```

## How was this patch tested?

Manually tested as above.

Closes #23481 from HyukjinKwon/SPARK-26339.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-07 15:48:54 -08:00
Marco Gaido 1a641525e6 [SPARK-26491][CORE][TEST] Use ConfigEntry for hardcoded configs for test categories
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.test` and `spark.testing` configs to use `ConfigEntry` and put them in the config package.

## How was this patch tested?

existing UTs

Closes #23413 from mgaido91/SPARK-26491.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-07 15:35:33 -08:00
maryannxue 98be8953c7 [SPARK-26065][SQL] Change query hint from a LogicalPlan to a field
## What changes were proposed in this pull request?

The existing query hint implementation relies on a logical plan node `ResolvedHint` to store query hints in logical plans, and on `Statistics` in physical plans. Since `ResolvedHint` is not really a logical operator and can break the pattern matching for existing and future optimization rules, it is a issue to the Optimizer as the old `AnalysisBarrier` was to the Analyzer.

Given the fact that all our query hints are either 1) a join hint, i.e., broadcast hint; or 2) a re-partition hint, which is indeed an operator, we only need to add a hint field on the Join plan and that will be a good enough solution for the current hint usage.

This PR is to let `Join` node have a hint for its left sub-tree and another hint for its right sub-tree and each hint is a merged result of all the effective hints specified in the corresponding sub-tree. The "effectiveness" of a hint, i.e., whether that hint should be propagated to the `Join` node, is currently consistent with the hint propagation rules originally implemented in the `Statistics` approach. Note that the `ResolvedHint` node still has to live through the analysis stage because of the `Dataset` interface, but it will be got rid of and moved to the `Join` node in the "pre-optimization" stage.

This PR also introduces a change in how hints work with join reordering. Before this PR, hints would stop join reordering. For example, in "a.join(b).join(c).hint("broadcast").join(d)", the broadcast hint would stop d from participating in the cost-based join reordering while still allowing reordering from under the hint node. After this PR, though, the broadcast hint will not interfere with join reordering at all, and after reordering if a relation associated with a hint stays unchanged or equivalent to the original relation, the hint will be retained, otherwise will be discarded. For example, the original plan is like "a.join(b).hint("broadcast").join(c).hint("broadcast").join(d)", thus the join order is "a JOIN b JOIN c JOIN d". So if after reordering the join order becomes "a JOIN b JOIN (c JOIN d)", the plan will be like "a.join(b).hint("broadcast").join(c.join(d))"; but if after reordering the join order becomes "a JOIN c JOIN b JOIN d", the plan will be like "a.join(c).join(b).hint("broadcast").join(d)".

## How was this patch tested?

Added new tests.

Closes #23036 from maryannxue/query-hint.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-07 13:59:40 -08:00
Marcelo Vanzin 669e8a1559 [SPARK-25689][YARN] Make driver, not AM, manage delegation tokens.
This change modifies the behavior of the delegation token code when running
on YARN, so that the driver controls the renewal, in both client and cluster
mode. For that, a few different things were changed:

* The AM code only runs code that needs DTs when DTs are available.

In a way, this restores the AM behavior to what it was pre-SPARK-23361, but
keeping the fix added in that bug. Basically, all the AM code is run in a
"UGI.doAs()" block; but code that needs to talk to HDFS (basically the
distributed cache handling code) was delayed to the point where the driver
is up and running, and thus when valid delegation tokens are available.

* SparkSubmit / ApplicationMaster now handle user login, not the token manager.

The previous AM code was relying on the token manager to keep the user
logged in when keytabs are used. This required some odd APIs in the token
manager and the AM so that the right UGI was exposed and used in the right
places.

After this change, the logged in user is handled separately from the token
manager, so the API was cleaned up, and, as explained above, the whole AM
runs under the logged in user, which also helps with simplifying some more code.

* Distributed cache configs are sent separately to the AM.

Because of the delayed initialization of the cached resources in the AM, it
became easier to write the cache config to a separate properties file instead
of bundling it with the rest of the Spark config. This also avoids having
to modify the SparkConf to hide things from the UI.

* Finally, the AM doesn't manage the token manager anymore.

The above changes allow the token manager to be completely handled by the
driver's scheduler backend code also in YARN mode (whether client or cluster),
making it similar to other RMs. To maintain the fix added in SPARK-23361 also
in client mode, the AM now sends an extra message to the driver on initialization
to fetch delegation tokens; and although it might not really be needed, the
driver also keeps the running AM updated when new tokens are created.

Tested in a kerberized cluster with the same tests used to validate SPARK-23361,
in both client and cluster mode. Also tested with a non-kerberized cluster.

Closes #23338 from vanzin/SPARK-25689.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-01-07 14:40:08 -06:00
Shahid 71183b2833 [SPARK-24489][ML] Check for invalid input type of weight data in ml.PowerIterationClustering
## What changes were proposed in this pull request?
The test case will result the following failure. currently in ml.PIC, there is no check for the data type of weight column.
 ```
 test("invalid input types for weight") {
    val invalidWeightData = spark.createDataFrame(Seq(
      (0L, 1L, "a"),
      (2L, 3L, "b")
    )).toDF("src", "dst", "weight")

    val pic = new PowerIterationClustering()
      .setWeightCol("weight")

    val result = pic.assignClusters(invalidWeightData)
  }
```
```
Job aborted due to stage failure: Task 0 in stage 8077.0 failed 1 times, most recent failure: Lost task 0.0 in stage 8077.0 (TID 882, localhost, executor driver): scala.MatchError: [0,1,null] (of class org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema)
	at org.apache.spark.ml.clustering.PowerIterationClustering$$anonfun$3.apply(PowerIterationClustering.scala:178)
	at org.apache.spark.ml.clustering.PowerIterationClustering$$anonfun$3.apply(PowerIterationClustering.scala:178)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at org.apache.spark.graphx.EdgeRDD$$anonfun$1.apply(EdgeRDD.scala:107)
	at org.apache.spark.graphx.EdgeRDD$$anonfun$1.apply(EdgeRDD.scala:105)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:847)
```
In this PR, added check types for weight column.
## How was this patch tested?
UT added

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #21509 from shahidki31/testCasePic.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2019-01-07 09:15:50 -08:00
ayudovin 868e02533d [SPARK-26383][CORE] NPE when use DataFrameReader.jdbc with wrong URL
### What changes were proposed in this pull request?
When passing wrong url to jdbc then It would throw IllegalArgumentException instead of NPE.
### How was this patch tested?
Adding test case to Existing tests in JDBCSuite

Closes #23464 from ayudovin/fixing-npe.

Authored-by: ayudovin <a.yudovin6695@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-07 08:58:33 -06:00
Liang-Chi Hsieh a927c764c1 [SPARK-26559][ML][PYSPARK] ML image can't work with numpy versions prior to 1.9
## What changes were proposed in this pull request?

Due to [API change](https://github.com/numpy/numpy/pull/4257/files#diff-c39521d89f7e61d6c0c445d93b62f7dc) at 1.9, PySpark image doesn't work with numpy version prior to 1.9.

When running image test with numpy version prior to 1.9, we can see error:
```
test_read_images (pyspark.ml.tests.test_image.ImageReaderTest) ... ERROR
test_read_images_multiple_times (pyspark.ml.tests.test_image.ImageReaderTest2) ... ok

======================================================================
ERROR: test_read_images (pyspark.ml.tests.test_image.ImageReaderTest)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/tests/test_image.py", line 36, in test_read_images
    self.assertEqual(ImageSchema.toImage(array, origin=first_row[0]), first_row)
  File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/image.py", line 193, in toImage
    data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes())
AttributeError: 'numpy.ndarray' object has no attribute 'tobytes'

----------------------------------------------------------------------
Ran 2 tests in 29.040s

FAILED (errors=1)
```

## How was this patch tested?

Manually test with numpy version prior and after 1.9.

Closes #23484 from viirya/fix-pyspark-image.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-07 18:36:52 +08:00
Dongjoon Hyun 468d25ec74
[MINOR][BUILD] Fix script name in release-tag.sh usage message
## What changes were proposed in this pull request?

This PR fixes the old script name in `release-tag.sh`.

    $ ./release-tag.sh --help | head -n1
    usage: tag-release.sh

## How was this patch tested?

Manual.

    $ ./release-tag.sh --help | head -n1
    usage: release-tag.sh

Closes #23477 from dongjoon-hyun/SPARK-RELEASE-TAG.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-06 22:45:18 -08:00
Dongjoon Hyun 61133cb8a6
[SPARK-26536][BUILD][FOLLOWUP][TEST-MAVEN] Make StreamingReadSupport public for maven testing
## What changes were proposed in this pull request?

`StreamingReadSupport` is designed to be a `package` interface. Mockito seems to complain during `Maven` testing. This doesn't fail in `sbt` and IntelliJ. For mock-testing purpose, this PR makes it `public` interface and adds explicit comments like `public interface ReadSupport`

```scala
EpochCoordinatorSuite:
*** RUN ABORTED ***
  java.lang.IllegalAccessError: tried to
access class org.apache.spark.sql.sources.v2.reader.streaming.StreamingReadSupport
from class org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338
  at org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport$MockitoMock$58628338.<clinit>(Unknown Source)
  at sun.reflect.GeneratedSerializationConstructorAccessor632.newInstance(Unknown Source)
  at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
  at org.objenesis.instantiator.sun.SunReflectionFactoryInstantiator.newInstance(SunReflectionFactoryInstantiator.java:48)
  at org.objenesis.ObjenesisBase.newInstance(ObjenesisBase.java:73)
  at org.mockito.internal.creation.instance.ObjenesisInstantiator.newInstance(ObjenesisInstantiator.java:19)
  at org.mockito.internal.creation.bytebuddy.SubclassByteBuddyMockMaker.createMock(SubclassByteBuddyMockMaker.java:47)
  at org.mockito.internal.creation.bytebuddy.ByteBuddyMockMaker.createMock(ByteBuddyMockMaker.java:25)
  at org.mockito.internal.util.MockUtil.createMock(MockUtil.java:35)
  at org.mockito.internal.MockitoCore.mock(MockitoCore.java:69)
```

## How was this patch tested?

Pass the Jenkins with Maven build

Closes #23463 from dongjoon-hyun/SPARK-26536-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-06 21:00:10 -08:00
Dongjoon Hyun fe039faddf
[SPARK-26554][BUILD] Update release-util.sh to avoid GitBox fake 200 headers
## What changes were proposed in this pull request?

Unlike the previous Apache Git repository, new GitBox repository returns a fake HTTP 200 header instead of `404 Not Found` header. This makes release scripts out of order. This PR aims to fix it to handle the html body message instead of the fake HTTP headers. This is a release blocker.

```bash
$ curl -s --head --fail "https://gitbox.apache.org/repos/asf?p=spark.git;a=commit;h=v3.0.0"
HTTP/1.1 200 OK
Date: Sun, 06 Jan 2019 22:42:39 GMT
Server: Apache/2.4.18 (Ubuntu)
Vary: Accept-Encoding
Access-Control-Allow-Origin: *
Access-Control-Allow-Methods: POST, GET, OPTIONS
Access-Control-Allow-Headers: X-PINGOTHER
Access-Control-Max-Age: 1728000
Content-Type: text/html; charset=utf-8
```

**BEFORE**
```bash
$ ./do-release-docker.sh -d /tmp/test -n
Branch [branch-2.4]:
Current branch version is 2.4.1-SNAPSHOT.
Release [2.4.1]:
RC # [1]:
v2.4.1-rc1 already exists. Continue anyway [y/n]?
```

**AFTER**
```bash
$ ./do-release-docker.sh -d /tmp/test -n
Branch [branch-2.4]:
Current branch version is 2.4.1-SNAPSHOT.
Release [2.4.1]:
RC # [1]:
This is a dry run. Please confirm the ref that will be built for testing.
Ref [v2.4.1-rc1]:
```

## How was this patch tested?

Manual.

Closes #23476 from dongjoon-hyun/SPARK-26554.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-06 19:59:31 -08:00
Maxim Gekk b305d71625
[SPARK-26547][SQL] Remove duplicate toHiveString from HiveUtils
## What changes were proposed in this pull request?

The `toHiveString()` and `toHiveStructString` methods were removed from `HiveUtils` because they have been already implemented in `HiveResult`. One related test was moved to `HiveResultSuite`.

## How was this patch tested?

By tests from `hive-thriftserver`.

Closes #23466 from MaxGekk/dedup-hive-result-string.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-06 17:36:06 -08:00
Hirobe Keiichi 9d8e9b394b [SPARK-26339][SQL] Throws better exception when reading files that start with underscore
## What changes were proposed in this pull request?
My pull request #23288 was resolved and merged to master, but it turned out  later that my change breaks another regression test. Because we cannot reopen pull request, I create a new pull request here.
Commit 92934b4 is only change after pull request #23288.
`CheckFileExist` was avoided at 239cfa4 after discussing #23288 (comment).
But, that change turned out to be wrong because we should not check if argument checkFileExist is false.

Test 27e42c1de5/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala (L2555)
failed when we avoided checkFileExist, but now successed after commit 92934b4 .

## How was this patch tested?
Both of below tests were passed.
```
testOnly org.apache.spark.sql.execution.datasources.csv.CSVSuite
testOnly org.apache.spark.sql.SQLQuerySuite
```

Closes #23446 from KeiichiHirobe/SPARK-26339.

Authored-by: Hirobe Keiichi <keiichi_hirobe@forcia.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-06 08:52:09 -06:00
SongYadong 737f08949a [SPARK-26527][CORE] Let acquireUnrollMemory fail fast if required space exceeds memory limit
## What changes were proposed in this pull request?

When acquiring unroll memory from `StaticMemoryManager`, let it fail fast if required space exceeds memory limit, just like acquiring storage memory.
I think this may reduce some computation and memory evicting costs especially when required space(`numBytes`) is very big.

## How was this patch tested?

Existing unit tests.

Closes #23426 from SongYadong/acquireUnrollMemory_fail_fast.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-06 08:46:20 -06:00
Dave DeCaprio a17851cb95 [SPARK-26548][SQL] Don't hold CacheManager write lock while computing executedPlan
## What changes were proposed in this pull request?

Address SPARK-26548, in Spark 2.4.0, the CacheManager holds a write lock while computing the executedPlan for a cached logicalPlan.  In some cases with very large query plans this can be an expensive operation, taking minutes to run.  The entire cache is blocked during this time.  This PR changes that so the writeLock is only obtained after the executedPlan is generated, this reduces the time the lock is held to just the necessary time when the shared data structure is being updated.

gatorsmile and cloud-fan - You can committed patches in this area before.  This is a small incremental change.

## How was this patch tested?

Has been tested on a live system where the blocking was causing major issues and it is working well.
 CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.

Closes #23469 from DaveDeCaprio/optimizer-unblocked.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-05 19:20:35 -08:00
Kris Mok 4ab5b5b918 [SPARK-26545] Fix typo in EqualNullSafe's truth table comment
## What changes were proposed in this pull request?

The truth table comment in EqualNullSafe incorrectly marked FALSE results as UNKNOWN.

## How was this patch tested?

N/A

Closes #23461 from rednaxelafx/fix-typo.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-05 14:37:04 -08:00
Maxim Gekk 0037bbb717 [MINOR][DOC] Fix typos in the SQL migration guide
## What changes were proposed in this pull request?

Fixed a few typos in the migration guide.

Closes #23465 from MaxGekk/fix-typos-migration-guide.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-05 22:53:28 +08:00
Maxim Gekk 980e6bcd1c [SPARK-26246][SQL][FOLLOWUP] Inferring TimestampType from JSON
## What changes were proposed in this pull request?

Added new JSON option `inferTimestamp` (`true` by default) to control inferring of `TimestampType` from string values.

## How was this patch tested?

Add new UT to `JsonInferSchemaSuite`.

Closes #23455 from MaxGekk/json-infer-time-followup.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-05 21:50:27 +08:00
Marco Gaido 1af1190bee
[SPARK-26078][SQL][FOLLOWUP] Remove useless import
## What changes were proposed in this pull request?

While backporting the patch to 2.4/2.3, I realized that the patch introduces unneeded imports (probably leftovers from intermediate changes). This PR removes the useless import.

## How was this patch tested?

NA

Closes #23451 from mgaido91/SPARK-26078_FOLLOWUP.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-05 01:14:58 -08:00
Dongjoon Hyun 5969b8a2ed
[SPARK-26541][BUILD] Add -Pdocker-integration-tests to dev/scalastyle
## What changes were proposed in this pull request?

This PR makes `scalastyle` to check `docker-integration-tests` module additionally and fixes one error.

## How was this patch tested?

Pass the Jenkins with the updated Scalastyle.
```
========================================================================
Running Scala style checks
========================================================================
Scalastyle checks passed.
```

Closes #23459 from dongjoon-hyun/SPARK-26541.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-05 00:55:17 -08:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
shane knapp bccb8602d7
[SPARK-26537][BUILD] change git-wip-us to gitbox
## What changes were proposed in this pull request?

due to apache recently moving from git-wip-us.apache.org to gitbox.apache.org, we need to update the packaging scripts to point to the new repo location.

this will also need to be backported to 2.4, 2.3, 2.1, 2.0 and 1.6.

## How was this patch tested?

the build system will test this.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #23454 from shaneknapp/update-apache-repo.

Authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 18:27:26 -08:00
Sean Owen 89cebf4932 [SPARK-24421][CORE][FOLLOWUP] Use normal direct ByteBuffer allocation if Cleaner can't be set
## What changes were proposed in this pull request?

In Java 9+ we can't use sun.misc.Cleaner by default anymore, and this was largely handled in https://github.com/apache/spark/pull/22993 However I think the change there left a significant problem.

If a DirectByteBuffer is allocated using the reflective hack in Platform, now, we by default can't set a Cleaner. But I believe this means the memory isn't freed promptly or possibly at all. If a Cleaner can't be set, I think we need to use normal APIs to allocate the direct ByteBuffer.

According to comments in the code, the downside is simply that the normal APIs will check and impose limits on how much off-heap memory can be allocated. Per the original review on https://github.com/apache/spark/pull/22993 this much seems fine, as either way in this case the user would have to add a JVM setting (increase max, or allow the reflective access).

## How was this patch tested?

Existing tests. This resolved an OutOfMemoryError in Java 11 from TimSort tests without increasing test heap size. (See https://github.com/apache/spark/pull/23419#issuecomment-450772125 ) This suggests there is a problem and that this resolves it.

Closes #23424 from srowen/SPARK-24421.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-04 15:37:09 -06:00
Sean Owen 36440e6447 [SPARK-26306][TEST][BUILD] More memory to de-flake SorterSuite
## What changes were proposed in this pull request?

Increase test memory to avoid OOM in TimSort-related tests.

## How was this patch tested?

Existing tests.

Closes #23425 from srowen/SPARK-26306.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-04 15:35:23 -06:00
Takuya UESHIN 4419e1daca [SPARK-26445][CORE] Use ConfigEntry for hardcoded configs for driver/executor categories.
## What changes were proposed in this pull request?

The PR makes hardcoded spark.driver, spark.executor, and spark.cores.max configs to use `ConfigEntry`.

Note that some config keys are from `SparkLauncher` instead of defining in the config package object because the string is already defined in it and it does not depend on core module.

## How was this patch tested?

Existing tests.

Closes #23415 from ueshin/issues/SPARK-26445/hardcoded_driver_executor_configs.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-04 22:12:35 +08:00
Dongjoon Hyun 27e42c1de5
[MINOR][NETWORK][TEST] Fix TransportFrameDecoderSuite to use ByteBuf instead of ByteBuffer
## What changes were proposed in this pull request?

`fireChannelRead` expects `io.netty.buffer.ByteBuf`.I checked that this is the only place which misuse `java.nio.ByteBuffer` in `network` module.

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23442 from dongjoon-hyun/SPARK-NETWORK-COMMON.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-03 20:01:19 -08:00
Liu,Linhong f65dc9593e [SPARK-26526][SQL][TEST] Fix invalid test case about non-deterministic expression
## What changes were proposed in this pull request?

Test case in SPARK-10316 is used to make sure non-deterministic `Filter` won't be pushed through `Project`
But in current code base this test case can't cover this purpose.
Change LogicalRDD to HadoopFsRelation can fix this issue.

## How was this patch tested?

Modified test pass.

Closes #23440 from LinhongLiu/fix-test.

Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-04 10:51:33 +08:00
Jungtaek Lim (HeartSaVioR) 05372d188a [SPARK-26489][CORE] Use ConfigEntry for hardcoded configs for python/r categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use ConfigEntry.

* spark.pyspark
* spark.python
* spark.r

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties, python source code)

## How was this patch tested?

Existing tests.

Closes #23428 from HeartSaVioR/SPARK-26489.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-03 14:30:27 -08:00
Gengliang Wang e2dbafdbc5 [SPARK-26447][SQL] Allow OrcColumnarBatchReader to return less partition columns
## What changes were proposed in this pull request?

Currently OrcColumnarBatchReader returns all the partition column values in the batch read.
In data source V2, we can improve it by returning the required partition column values only.

This PR is part of https://github.com/apache/spark/pull/23383 . As cloud-fan suggested, create a new PR to make review easier.

Also, this PR doesn't improve `OrcFileFormat`, since in the method `buildReaderWithPartitionValues`, the `requiredSchema` filter out all the partition columns, so we can't know which partition column is required.

## How was this patch tested?

Unit test

Closes #23387 from gengliangwang/refactorOrcColumnarBatch.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-04 00:37:03 +08:00
Liang-Chi Hsieh 40711eef16 [SPARK-26517][SQL][TEST] Avoid duplicate test in ParquetSchemaPruningSuite
## What changes were proposed in this pull request?

`testExactCaseQueryPruning` and `testMixedCaseQueryPruning` don't need to set up `PARQUET_VECTORIZED_READER_ENABLED` config. Because `withMixedCaseData` will run against both Spark vectorized reader and Parquet-mr reader.

## How was this patch tested?

Existing test.

Closes #23427 from viirya/fix-parquet-schema-pruning-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-03 10:30:47 -06:00
Liupengcheng 88b074f3f0 [SPARK-26501][CORE][TEST] Fix unexpected overriden of exitFn in SparkSubmitSuite
## What changes were proposed in this pull request?

The overriden of SparkSubmit's exitFn at some previous tests in SparkSubmitSuite may cause the following tests pass even they failed when they were run separately. This PR is to fix this problem.

## How was this patch tested?

unittest

Closes #23404 from liupc/Fix-SparkSubmitSuite-exitFn.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-03 10:26:14 -06:00
Maxim Gekk 2a30deb85a [SPARK-26502][SQL] Move hiveResultString() from QueryExecution to HiveResult
## What changes were proposed in this pull request?

In the PR, I propose to move `hiveResultString()` out of `QueryExecution` and put it to a separate object.

Closes #23409 from MaxGekk/hive-result-string.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-03 11:27:40 +01:00
Hyukjin Kwon 56967b7e28 [SPARK-26403][SQL] Support pivoting using array column for pivot(column) API
## What changes were proposed in this pull request?

This PR fixes `pivot(Column)` can accepts `collection.mutable.WrappedArray`.

Note that we return `collection.mutable.WrappedArray` from `ArrayType`, and `Literal.apply` doesn't support this.

We can unwrap the array and use it for type dispatch.

```scala
val df = Seq(
  (2, Seq.empty[String]),
  (2, Seq("a", "x")),
  (3, Seq.empty[String]),
  (3, Seq("a", "x"))).toDF("x", "s")
df.groupBy("x").pivot("s").count().show()
```

Before:

```
Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray()
java.lang.RuntimeException: Unsupported literal type class scala.collection.mutable.WrappedArray$ofRef WrappedArray()
	at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:80)
	at org.apache.spark.sql.RelationalGroupedDataset.$anonfun$pivot$2(RelationalGroupedDataset.scala:427)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
	at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
	at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
	at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:39)
	at scala.collection.TraversableLike.map(TraversableLike.scala:237)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
	at scala.collection.AbstractTraversable.map(Traversable.scala:108)
	at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:425)
	at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:406)
	at org.apache.spark.sql.RelationalGroupedDataset.pivot(RelationalGroupedDataset.scala:317)
	at org.apache.spark.sql.DataFramePivotSuite.$anonfun$new$1(DataFramePivotSuite.scala:341)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
```

After:

```
+---+---+------+
|  x| []|[a, x]|
+---+---+------+
|  3|  1|     1|
|  2|  1|     1|
+---+---+------+
```

## How was this patch tested?

Manually tested and unittests were added.

Closes #23349 from HyukjinKwon/SPARK-26403.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-03 11:01:54 +08:00
Maxim Gekk 8be4d24a27 [SPARK-26023][SQL][FOLLOWUP] Dumping truncated plans and generated code to a file
## What changes were proposed in this pull request?

`DataSourceScanExec` overrides "wrong" `treeString` method without `append`. In the PR, I propose to make `treeString`s **final** to prevent such mistakes in the future. And removed the `treeString` and `verboseString` since they both use `simpleString` with reduction.

## How was this patch tested?

It was tested by `DataSourceScanExecRedactionSuite`

Closes #23431 from MaxGekk/datasource-scan-exec-followup.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-02 16:57:10 -08:00
seancxmao d40654861b [SPARK-26277][SQL][TEST] WholeStageCodegen metrics should be tested with whole-stage codegen enabled
## What changes were proposed in this pull request?
In `org.apache.spark.sql.execution.metric.SQLMetricsSuite`, there's a test case named "WholeStageCodegen metrics". However, it is executed with whole-stage codegen disabled. This PR fixes this by enable whole-stage codegen for this test case.

## How was this patch tested?
Tested locally using exiting test cases.

Closes #23224 from seancxmao/codegen-metrics.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-02 15:45:14 -06:00
Sean Owen 4bdfda92a1 [SPARK-26507][CORE] Fix core tests for Java 11
## What changes were proposed in this pull request?

This should make tests in core modules pass for Java 11.

## How was this patch tested?

Existing tests, with modifications.

Closes #23419 from srowen/Java11.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-02 11:23:53 -06:00
Kazuaki Ishizaki 79b05481a2 [SPARK-26508][CORE][SQL] Address warning messages in Java reported at lgtm.com
## What changes were proposed in this pull request?

This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com).

[lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project.

This PR addresses the following warnings:

- Result of multiplication cast to wider type
- Implicit narrowing conversion in compound assignment
- Boxed variable is never null
- Useless null check

NOTE: `Potential input resource leak` looks false positive for now.

## How was this patch tested?

Existing UTs

Closes #23420 from kiszk/SPARK-26508.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-01 22:37:28 -06:00
Hyukjin Kwon d371180c01 [MINOR][R] Deduplicate RStudio setup documentation
## What changes were proposed in this pull request?

This PR targets to deduplicate RStudio setup for SparkR.

## How was this patch tested?

N/A

Closes #23421 from HyukjinKwon/minor-doc.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-02 08:04:36 +08:00
Hyukjin Kwon 39a0493387 [SPARK-26227][R] from_[csv|json] should accept schema_of_[csv|json] in R API
## What changes were proposed in this pull request?

**1. Document `from_csv(..., schema_of_csv(...))` support:**

```R
csv <- "Amsterdam,2018"
df <- sql(paste0("SELECT '", csv, "' as csv"))
head(select(df, from_csv(df$csv, schema_of_csv(csv))))
```

```
    from_csv(csv)
1 Amsterdam, 2018
```

**2. Allow `from_json(..., schema_of_json(...))`**

Before:

```R
df2 <- sql("SELECT named_struct('name', 'Bob') as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json))))
```

```
Error in (function (classes, fdef, mtable)  :
  unable to find an inherited method for function ‘from_json’ for signature ‘"Column", "Column"’
```

After:

```R
df2 <- sql("SELECT named_struct('name', 'Bob') as people")
df2 <- mutate(df2, people_json = to_json(df2$people))
head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json))))
```

```
  from_json(people_json)
1                    Bob
```

**3. (While I'm here) Allow `structType` as schema for `from_csv` support to match with `from_json`.**

Before:

```R
csv <- "Amsterdam,2018"
df <- sql(paste0("SELECT '", csv, "' as csv"))
head(select(df, from_csv(df$csv, structType("city STRING, year INT"))))
```

```
Error in (function (classes, fdef, mtable)  :
  unable to find an inherited method for function ‘from_csv’ for signature ‘"Column", "structType"’
```

After:

```R
csv <- "Amsterdam,2018"
df <- sql(paste0("SELECT '", csv, "' as csv"))
head(select(df, from_csv(df$csv, structType("city STRING, year INT"))))
```

```
    from_csv(csv)
1 Amsterdam, 2018
```

## How was this patch tested?

Manually tested and unittests were added.

Closes #23184 from HyukjinKwon/SPARK-26227-1.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-02 08:01:34 +08:00
Maxim Gekk 5da55873fa [SPARK-26374][TEST][SQL] Enable TimestampFormatter in HadoopFsRelationTest
## What changes were proposed in this pull request?

Default timestamp pattern defined in `JSONOptions` doesn't allow saving/loading timestamps with time zones of seconds precision. Because of that, the round trip test failed for timestamps before 1582. In the PR, I propose to extend zone offset section from `XXX` to `XXXXX` which should allow to save/load zone offsets like `-07:52:48`.

## How was this patch tested?

It was tested by `JsonHadoopFsRelationSuite` and `TimestampFormatterSuite`.

Closes #23417 from MaxGekk/hadoopfsrelationtest-new-formatter.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-02 07:59:32 +08:00