Commit graph

23931 commits

Author SHA1 Message Date
Wenchen Fan d3813d8b21 [SPARK-27064][SS] create StreamingWrite at the beginning of streaming execution
## What changes were proposed in this pull request?

According to the [design](https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing), the life cycle of `StreamingWrite` should be the same as the read side `MicroBatch/ContinuousStream`, i.e. each run of the stream query, instead of each epoch.

This PR fixes it.

## How was this patch tested?

existing tests

Closes #23981 from cloud-fan/dsv2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 19:47:54 +08:00
Liang-Chi Hsieh f55c760df6 [SPARK-27034][SQL][FOLLOWUP] Rename ParquetSchemaPruning to SchemaPruning
## What changes were proposed in this pull request?

This is a followup to #23943. This proposes to rename ParquetSchemaPruning to SchemaPruning as ParquetSchemaPruning supports both Parquet and ORC v1 now.

## How was this patch tested?

Existing tests.

Closes #24077 from viirya/nested-schema-pruning-orc-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-13 20:12:01 +09:00
DB Tsai 2b9ad2516e [MINOR][BUILD] Add Scala 2.12 profile back for branch-2.4 build
Closes #24074 from dbtsai/scala-2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 20:08:52 -07:00
Jungtaek Lim (HeartSaVioR) 1b06cda532 [MINOR][SQL] Refactor RowEncoder to use existing (De)serializerBuildHelper methods
## What changes were proposed in this pull request?

This patch proposes to reuse existing methods in (De)serializerBuildHelper in RowEncoder to achieve deduplication as well as consistent creation of serialization/deserialization of same type.

## How was this patch tested?

Existing UT.

Closes #24014 from HeartSaVioR/SPARK-27092.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-13 10:54:47 +08:00
Takeshi Yamamuro 1e9469bb7a [SPARK-26976][SQL] Forbid reserved keywords as identifiers when ANSI mode is on
## What changes were proposed in this pull request?
This pr added code to forbid reserved keywords as identifiers when ANSI mode is on.
This is a follow-up of SPARK-26215(#23259).

## How was this patch tested?
Added tests in `TableIdentifierParserSuite`.

Closes #23880 from maropu/SPARK-26976.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-13 11:20:27 +09:00
hehuiyuan 7beb464564 [MINOR][DOC] Fix the description of Pod Metadata's annotations
## What changes were proposed in this pull request?

![annotation](https://user-images.githubusercontent.com/18002496/54189638-2d551780-44ed-11e9-9efc-3691bec42130.jpg)

Closes #24064 from hehuiyuan/hehuiyuan-patch-4.

Authored-by: hehuiyuan <hehuiyuan@ZBMAC-C02WD3K5H.local>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 19:29:32 -05:00
Ajith e60d8fce0b [SPARK-27045][SQL] SQL tab in UI shows actual SQL instead of callsite in case of SparkSQLDriver
## What changes were proposed in this pull request?

When we run sql in spark via SparkSQLDriver (thrift server, spark-sql), SQL string is siet via ``setJobDescription``. the SparkUI SQL tab must show SQL instead of stacktrace in case ``setJobDescription`` is set which is more useful to end user. Instead it currently shows in description column the callsite shortform which is less useful

![image](https://user-images.githubusercontent.com/22072336/53734682-aaa7d900-3eaa-11e9-957b-0e5006db417e.png)

## How was this patch tested?

Manually:
![image](https://user-images.githubusercontent.com/22072336/53734657-9f54ad80-3eaa-11e9-8dc5-2b38f6970f4e.png)

Closes #23958 from ajithme/sqlui.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 16:14:29 -07:00
Yuming Wang dccf6615c3 [SPARK-27130][BUILD] Automatically select profile when executing sbt-checkstyle
## What changes were proposed in this pull request?

This PR makes it automatically select profile when executing `sbt-checkstyle`. The reason for this is that `hadoop-2.7` and `hadoop-3.1` may have different `hive-thriftserver` module in the future.

## How was this patch tested?

manual tests:
```
Update AbstractService.java file.
export HADOOP_PROFILE=hadoop2.7
./dev/run-tests
```
The result:
![image](https://user-images.githubusercontent.com/5399861/54197992-5337e780-4500-11e9-930c-722982cdcd45.png)

Closes #24065 from wangyum/SPARK-27130.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-13 08:03:46 +09:00
Jungtaek Lim (HeartSaVioR) f57af2286f [MINOR][CORE] Use https for bintray spark-packages repository
## What changes were proposed in this pull request?

This patch changes the schema of url from http to https for bintray spark-packages repository. Looks like we already changed the schema of repository url for pom.xml but missed inside the code.

## How was this patch tested?

Manually ran the `--package` via `./bin/spark-shell --verbose  --packages "RedisLabs:spark-redis:0.3.2"`

```
...
Ivy Default Cache set to: /Users/jlim/.ivy2/cache
The jars for the packages stored in: /Users/jlim/.ivy2/jars
:: loading settings :: url = jar:file:/Users/jlim/WorkArea/ScalaProjects/spark/dist/jars/ivy-2.4.0.jar!/org/apache/ivy/core/settings/ivysettings.xml
RedisLabs#spark-redis added as a dependency
:: resolving dependencies :: org.apache.spark#spark-submit-parent-2fee2e18-7832-4a4d-9e97-7b3d0fef766d;1.0
	confs: [default]
	found RedisLabs#spark-redis;0.3.2 in spark-packages
	found redis.clients#jedis;2.7.2 in central
	found org.apache.commons#commons-pool2;2.3 in central
downloading https://dl.bintray.com/spark-packages/maven/RedisLabs/spark-redis/0.3.2/spark-redis-0.3.2.jar ...
	[SUCCESSFUL ] RedisLabs#spark-redis;0.3.2!spark-redis.jar (824ms)
downloading https://repo1.maven.org/maven2/redis/clients/jedis/2.7.2/jedis-2.7.2.jar ...
	[SUCCESSFUL ] redis.clients#jedis;2.7.2!jedis.jar (576ms)
downloading https://repo1.maven.org/maven2/org/apache/commons/commons-pool2/2.3/commons-pool2-2.3.jar ...
	[SUCCESSFUL ] org.apache.commons#commons-pool2;2.3!commons-pool2.jar (150ms)
:: resolution report :: resolve 4586ms :: artifacts dl 1555ms
	:: modules in use:
	RedisLabs#spark-redis;0.3.2 from spark-packages in [default]
	org.apache.commons#commons-pool2;2.3 from central in [default]
	redis.clients#jedis;2.7.2 from central in [default]
	---------------------------------------------------------------------
	|                  |            modules            ||   artifacts   |
	|       conf       | number| search|dwnlded|evicted|| number|dwnlded|
	---------------------------------------------------------------------
	|      default     |   3   |   3   |   3   |   0   ||   3   |   3   |
	---------------------------------------------------------------------
```

Closes #24061 from HeartSaVioR/MINOR-use-https-to-bintray-repository.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 18:01:16 -05:00
Liang-Chi Hsieh b0c2b3bfd9 [SPARK-27034][SQL] Nested schema pruning for ORC
## What changes were proposed in this pull request?

We only supported nested schema pruning for Parquet previously. This proposes to support nested schema pruning for ORC too.

Note: This only covers ORC v1. For ORC v2, the necessary change is at the schema pruning rule. We should deal with ORC v2 as a TODO item, in order to reduce review burden.

## How was this patch tested?

Added tests.

Closes #23943 from viirya/nested-schema-pruning-orc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-12 15:39:16 -07:00
Dongjoon Hyun 78314af580 [SPARK-27123][SQL] Improve CollapseProject to handle projects cross limit/repartition/sample
## What changes were proposed in this pull request?

`CollapseProject` optimizer rule simplifies some plans by merging the adjacent projects and performing alias substitutions.
```scala
scala> sql("SELECT b c FROM (SELECT a b FROM t)").explain
== Physical Plan ==
*(1) Project [a#5 AS c#1]
+- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

We can do that more complex cases like the following. This PR aims to handle adjacent projects across limit/repartition/sample. Here, repartition means `Repartition`, not `RepartitionByExpression`.

**BEFORE**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
*(2) Project [b#0 AS c#1]
+- Exchange RoundRobinPartitioning(1)
   +- *(1) Project [a#5 AS b#0]
      +- Scan hive default.t [a#5], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#5]
```

**AFTER**
```scala
scala> sql("SELECT b c FROM (SELECT /*+ REPARTITION(1) */ a b FROM t)").explain
== Physical Plan ==
Exchange RoundRobinPartitioning(1)
+- *(1) Project [a#11 AS c#7]
   +- Scan hive default.t [a#11], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [a#11]
```

## How was this patch tested?

Pass the Jenkins with the newly added and updated test cases.

Closes #24049 from dongjoon-hyun/SPARK-27123.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-12 21:45:40 +00:00
Liupengcheng d5cfe08fdc [SPARK-26927][CORE] Ensure executor is active when processing events in dynamic allocation manager.
## What changes were proposed in this pull request?

There is a race condition in the `ExecutorAllocationManager` that the `SparkListenerExecutorRemoved` event is posted before the `SparkListenerTaskStart` event, which will cause the incorrect result of `executorIds`. Then, when some executor idles, the real executors will be removed even actual executor number is equal to `minNumExecutors` due to the incorrect computation of `newExecutorTotal`(may greater than the `minNumExecutors`), thus may finally causing zero available executors but a wrong positive number of executorIds was kept in memory.

What's more, even the `SparkListenerTaskEnd` event can not make the fake `executorIds` released, because later idle event for the fake executors can not cause the real removal of these executors, as they are already removed and they are not exist in the `executorDataMap`  of `CoaseGrainedSchedulerBackend`, so that the `onExecutorRemoved` method will never be called again.

For details see https://issues.apache.org/jira/browse/SPARK-26927

This PR is to fix this problem.

## How was this patch tested?

existUT and added UT

Closes #23842 from liupc/Fix-race-condition-that-casues-dyanmic-allocation-not-working.

Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-12 13:53:42 -07:00
ankurgupta 688b0c01fa [SPARK-26089][CORE] Handle corruption in large shuffle blocks
## What changes were proposed in this pull request?

SPARK-4105 added corruption detection in shuffle blocks but that was limited to blocks which are
smaller than maxBytesInFlight/3. This commit adds upon that by adding corruption check for large
blocks. There are two changes/improvements that are made in this commit:

1. Large blocks are checked upto maxBytesInFlight/3 size in a similar way as smaller blocks, so if a
large block is corrupt in the starting, that block will be re-fetched and if that also fails,
FetchFailureException will be thrown.
2. If large blocks are corrupt after size maxBytesInFlight/3, then any IOException thrown while
reading the stream will be converted to FetchFailureException.  This is slightly more aggressive
than was originally intended but since the consumer of the stream may have already read some records and processed them, we can't just re-fetch the block, we need to fail the whole task. Additionally, we also thought about maybe adding a new type of TaskEndReason, which would re-try the task couple of times before failing the previous stage, but given the complexity involved in that solution we decided to not proceed in that direction.

Thanks to squito for direction and support.

## How was this patch tested?

Changed the junit test for big blocks to check for corruption.

Closes #23453 from ankuriitg/ankurgupta/SPARK-26089.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-12 14:27:44 -05:00
zuotingbing 3f9247de1e [SPARK-27010][SQL] Find out the actual port number when hive.server2.thrift.port=0
## What changes were proposed in this pull request?
Currently, if we set hive.server2.thrift.port=0, it hard to find out the actual port number which one we should use when using beeline to connect.

before:
![2019-02-28_170942](https://user-images.githubusercontent.com/24823338/53557240-779ad800-3b80-11e9-9567-175f28aa61da.png)

after:
![2019-02-28_170904](https://user-images.githubusercontent.com/24823338/53557255-7f5a7c80-3b80-11e9-8ba6-9764c03e5407.png)

use beeline to connect success:
![2019-02-28_170844](https://user-images.githubusercontent.com/24823338/53557267-85e8f400-3b80-11e9-90a5-f7f53a51cc32.png)

## How was this patch tested?
 manual tests

Closes #23917 from zuotingbing/SPARK-27010.

Authored-by: zuotingbing <zuo.tingbing9@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:38:41 -05:00
shivusondur 4b6d39d85d [SPARK-27090][CORE] Removing old LEGACY_DRIVER_IDENTIFIER ("<driver>")
## What changes were proposed in this pull request?
LEGACY_DRIVER_IDENTIFIER and its reference are removed.
corresponding references test are updated.

## How was this patch tested?
tested  UT test cases

Closes #24026 from shivusondur/newjira2.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:29:39 -05:00
TigerYang414 60a899b8c3 [SPARK-27041][PYSPARK] Use imap() for python 2.x to resolve oom issue
## What changes were proposed in this pull request?

With large partition, pyspark may exceeds executor memory limit and trigger out of memory for python 2.7.
This is because map() is used. Unlike in python3.x, python 2.7 map() will generate a list and need to read all data into memory.

The proposed fix will use imap in python 2.7 and it has been verified.

## How was this patch tested?
Manual test.
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #23954 from TigerYang414/patch-1.

Lead-authored-by: TigerYang414 <39265202+TigerYang414@users.noreply.github.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 10:23:26 -05:00
Shahid 1853db3186 [SPARK-27125][SQL][TEST] Add test suite for sql execution page
## What changes were proposed in this pull request?
Added test suite for AllExecutionsPage class. Checked the scenarios for SPARK-27019 and SPARK-27075.

## How was this patch tested?
Added UT, manually tested

Closes #24052 from shahidki31/SPARK-27125.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 10:15:28 -05:00
Dilip Biswal b15423361b [SPARK-27016][SQL][BUILD][FOLLOW-UP] Treat all antlr warnings as errors while generating the parser.
## What changes were proposed in this pull request?
Use the sbt maven plugin option`antlr4TreatWarningsAsErrors` to make sure the warnings are treated as errors while generating the parser. In the absence of it, we may inadvertently introduce problems while making grammar changes. Please refer to PR-23897 to know more about the context. We made a change in [pr-23925](https://github.com/apache/spark/pull/23925) which handled only the maven build.

In this PR, we handle the sbt build. I had submitted [PR-23](https://github.com/ihji/sbt-antlr4/pull/23) to enhance the sbt-antlr plugin to make is possible to pass the error on warning option.

## How was this patch tested?
Force an warning in the grammar file to check if the build fails. Then remove the warning to verify the build succeeds.

Closes #24060 from dilipbiswal/sbt_build_antlr.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-11 21:27:51 -07:00
Ajith b8dd84b9e4 [SPARK-27011][SQL] reset command fails with cache
## What changes were proposed in this pull request?

When cache is enabled ( i.e once cache table command is executed), any following sql will trigger
 CacheManager#lookupCachedData which will create a copy of the tree node, which inturn calls TreeNode#makeCopy. Here the problem is it will try to create a copy instance. But as ResetCommand is a case object this will fail

## How was this patch tested?

Added UT to reproduce the issue

Closes #23918 from ajithme/reset.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-12 11:02:09 +08:00
Maxim Gekk 60be6d2ea3 [SPARK-27109][SQL] Refactoring of TimestampFormatter and DateFormatter
## What changes were proposed in this pull request?

In PR, I propose to refactor the `parse()` method of `Iso8601DateFormatter`/`Iso8601DateFormatter` and `toInstantWithZoneId` of `toInstantWithZoneId` to achieve the following:
- Avoid unnecessary conversion of parsed input to `java.time.Instant` before converting it to micros and days. Necessary information exists in `ZonedDateTime` already, and micros/days can be extracted from the former one.
- Avoid additional extraction of LocalTime from parsed object, more precisely, double query of `TemporalQueries.localTime` from `temporalAccessor`.
- Avoid additional extraction of zone id from parsed object, in particular, double query of `TemporalQueries.offset()`.
- Using `ZoneOffset.UTC` instead of `ZoneId.of` in `DateFormatter`. This allows to avoid looking for zone offset by zone id.

## How was this patch tested?

By existing test suite `DateTimeUtilsSuite`, `TimestampFormatterSuite` and `DateFormatterSuite`.

Closes #24030 from MaxGekk/query-localtime.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 19:02:30 -05:00
hongdongdong 1029bf9c35 Use variable instead of function to keep the format uniform
## What changes were proposed in this pull request?

The change just use  variable(_taskScheduler) instead of function(taskScheduler) to keep the format uniform in different situation.

## How was this patch tested?

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

Closes #24048 from hddong/Use-variable-instead-of-function.

Authored-by: hongdongdong <hongdongdong@cmss.chinamobile.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 19:00:26 -05:00
Hyukjin Kwon 3725b1324f [SPARK-26923][SQL][R] Refactor ArrowRRunner and RRunner to share one BaseRRunner
## What changes were proposed in this pull request?

This PR proposes to have one base R runner.

In the high level,

Previously, it had `ArrowRRunner` and it inherited `RRunner`:

```
└── RRunner
    └── ArrowRRunner
```

After this PR, now it has a `BaseRRunner`, and `ArrowRRunner` and `RRunner` inherit `BaseRRunner`:

```
└── BaseRRunner
    ├── ArrowRRunner
    └── RRunner
```

This way is consistent with Python's.

In more details, see below:

```scala
class BaseRRunner[IN, OUT] {

  def compute: Iterator[OUT] = {
    ...
    newWriterThread(...).start()
    ...
    newReaderIterator(...)
    ...
  }

  // Make a thread that writes data from JVM to R process
  abstract protected def newWriterThread(..., iter: Iterator[IN], ...): WriterThread

  // Make an iterator that reads data from the R process to JVM
  abstract protected def newReaderIterator(...): ReaderIterator

  abstract class WriterThread(..., iter: Iterator[IN], ...) extends Thread {
    override def run(): Unit {
      ...
      writeIteratorToStream(...)
      ...
    }

    // Actually writing logic to the socket stream.
    abstract protected def writeIteratorToStream(dataOut: DataOutputStream): Unit
  }

  abstract class ReaderIterator extends Iterator[OUT] {
    override def hasNext(): Boolean = {
      ...
      read(...)
      ...
    }

    override def next(): OUT = {
      ...
      hasNext()
      ...
    }

    // Actually reading logic from the socket stream.
    abstract protected def read(...): OUT
  }
}
```

```scala
case [Arrow]RRunner extends BaseRRunner {
  override def newWriterThread(...) {
    new WriterThread(...) {
      override def writeIteratorToStream(...) {
        ...
      }
    }
  }

  override def newReaderIterator(...) {
    new ReaderIterator(...) {
      override def read(...) {
        ...
      }
    }
  }
}
```

## How was this patch tested?

Manually tested and existing tests should cover.

Closes #23977 from HyukjinKwon/SPARK-26923.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-12 08:45:29 +09:00
hehuiyuan fd1852b344 [MINOR][DOC] Fix spark.kubernetes.executor.label.[LabelName] parameter meaning
## What changes were proposed in this pull request?

It would be better to change the explanation to this spark.kubernetes.executor.label.[LabelName].
Before:
    Note that Spark also adds its own labels to the **driver pod** for bookkeeping purposes.

After modification:
   Note that Spark also adds its own labels to the **executor pod** for bookkeeping purposes.

Closes #24054 from hehuiyuan/hehuiyuan-patch-3.

Authored-by: hehuiyuan <hehuiyuan@ZBMAC-C02WD3K5H.local>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-11 16:26:47 -07:00
sychen 064604aaa7 [SPARK-27073][CORE] Fix a race condition when handling of IdleStateEvent
## What changes were proposed in this pull request?

When TransportChannelHandler processes IdleStateEvent, it first calculates whether the last request time has timed out.
At this time, TransportClient.sendRpc initiates a request.
TransportChannelHandler gets responseHandler.numOutstandingRequests() > 0, causing the normal connection to be closed.

## How was this patch tested?

Closes #23989 from cxzl25/fix_IdleStateEvent_timeout.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-11 15:16:16 -07:00
Marcelo Vanzin f1e223bfa3 [SPARK-27004][CORE] Remove stale HTTP auth code.
This code is from the era when Spark used an HTTP server to distribute
dependencies, which is long gone. Nowadays it only causes problems when
someone is using dependencies from an HTTP server with Spark auth on.

Closes #24033 from vanzin/SPARK-27004.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-11 12:27:25 -07:00
chandulal.kavar d4542a8ba8 [SPARK-27061][K8S] Expose Driver UI port on driver service to access …
## What changes were proposed in this pull request?

Expose Spark UI port on driver service to access logs from service.

## How was this patch tested?

The patch was tested using unit tests being contributed as a part of the PR

Closes #23990 from chandulal/SPARK-27061.

Authored-by: chandulal.kavar <cckavar@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-11 10:41:31 -07:00
Wenchen Fan 31878c9daa [SPARK-27119][SQL] Do not infer schema when reading Hive serde table with native data source
## What changes were proposed in this pull request?

In Spark 2.1, we hit a correctness bug. When reading a Hive serde parquet table with the native parquet data source, and the actual file schema doesn't match the table schema in Hive metastore(only upper/lower case difference), the query returns 0 results.

The reason is that, the parquet reader is case sensitive. If we push down filters with column names that don't match the file physical schema case-sensitively, no data will be returned.

To fix this bug, there were 2 solutions proposed at that time:
1. Add a config to optionally disable parquet filter pushdown, and make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/16797

2. Infer the actual schema from data files, when reading Hive serde table with native data source. A config is provided to disable it.
https://github.com/apache/spark/pull/17229

Solution 2 was accepted and merged to Spark 2.1.1

In Spark 2.4, we refactored the parquet data source a little:
1. do parquet filter pushdown with the actual file schema.
https://github.com/apache/spark/pull/21696

2. make parquet filter pushdown case insensitive.
https://github.com/apache/spark/pull/22197

3. make parquet column pruning case insensitive.
https://github.com/apache/spark/pull/22148

With these patches, the correctness bug in Spark 2.1 no longer exists, and the schema inference becomes unnecessary.

To be safe, this PR just changes the default value to NEVER_INFER, so that users can set it back to INFER_AND_SAVE. If we don't receive any bug reports for it, we can remove the related code in the next release.

## How was this patch tested?

existing tests

Closes #24041 from cloud-fan/infer.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-11 09:44:29 -07:00
Jagadesh Kiran d9978fb4e4 [SPARK-26860][PYSPARK][SPARKR] Fix for RangeBetween and RowsBetween docs to be in sync with spark documentation
The docs describing RangeBetween & RowsBetween for pySpark & SparkR are not in sync with Spark description.

a. Edited PySpark and SparkR docs  and made description same for both RangeBetween and RowsBetween
b. created executable examples in both pySpark and SparkR documentation
c. Locally tested the patch for scala Style checks and UT for checking no testcase failures

Closes #23946 from jagadesh-kiran/master.

Authored-by: Jagadesh Kiran <jagadesh.n@in.verizon.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:53:09 -05:00
Ajith b98922abf2 [SPARK-27116] Environment tab must sort Hadoop Configuration by default
## What changes were proposed in this pull request?

Environment tab in SparkUI do not have Hadoop Configuration sorted. All other tables in the same page like Spark Configrations, System Configuration etc are sorted by keys by default

## How was this patch tested?

Manually tested on SparkUI

Closes #24038 from ajithme/sqluisort.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:43:49 -05:00
Wenchen Fan 8114b63d56 [SPARK-27117][SQL] current_date/current_timestamp should not refer to columns with ansi parser mode
## What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/19559 .

It revisits https://issues.apache.org/jira/browse/SPARK-27117 , which should be an invalid use case according to the SQL standard.

`current_date/current_timestamp` are reserved keywords, if users want to access columns named `current_date/current_timestamp`, they should quote the name like ```select `current_date` from tbl```

If ansi mode is not enabled(which is the default), this PR won't introduce any changes.

## How was this patch tested?

a new test case

Closes #24039 from cloud-fan/current_datetime.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 20:40:03 +08:00
Dilip Biswal 1b9fd67904 [SPARK-27096][SQL] Reconcile the join types between data frame and sql interface
## What changes were proposed in this pull request?
Currently in the grammar file, we have the joinType rule defined as following :
```
joinType
    : INNER?
   ....
   ....
    | LEFT SEMI
    | LEFT? ANTI
    ;
```
The keyword LEFT is optional for ANTI join even though its not optional for SEMI join. When
using data frame interface join type "anti" is not allowed. The allowed types are "left_anti" or
"leftanti" for anti joins. ~~In this PR, i am making the LEFT keyword mandatory for ANTI joins so
it aligns better with the LEFT SEMI join in the grammar file and also the join types allowed from dataframe api.~~

This PR makes LEFT optional for SEMI join in .g4 and add "semi" and "anti" join types from dataframe.

~~I have not opened any JIRA for this as probably we may need some discussion to see if
we are going to address this or not.~~

## How was this patch tested?
Modified the join type tests.

Closes #23982 from dilipbiswal/join_fix.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 14:02:21 +08:00
Takeshi Yamamuro 7a9537c338 [SPARK-21351][SQL] Remove the UpdateAttributeNullability rule from the optimizer
## What changes were proposed in this pull request?
This pr removed `UpdateAttributeNullability` from the optimizer because the same logic happens in the analyzer. See SPARK-26459(#23390) for more detailed discussion.

## How was this patch tested?
N/A

Closes #23508 from maropu/SPARK-21351.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-11 13:34:14 +08:00
Jungtaek Lim (HeartSaVioR) f0bde69ebc [MINOR][SQL] Throw better exception for Encoder with tuple more than 22 elements
## What changes were proposed in this pull request?

This patch proposes to throw better exception with better error message when encoding to tuple which elements are more than 22.

**BEFORE**
```scala
scala> import org.apache.spark.sql.catalyst.encoders._
scala> val encoders = (0 to 22).map(_ => org.apache.spark.sql.Encoders.scalaInt.asInstanceOf[ExpressionEncoder[_]])
scala> ExpressionEncoder.tuple(encoders)
java.lang.ClassNotFoundException: scala.Tuple23
```

**AFTER**
```scala
scala> ExpressionEncoder.tuple(encoders)
java.lang.UnsupportedOperationException: Due to Scala's limited support of tuple, tuple with more than 22 elements are not supported.
```

## How was this patch tested?

Added UT.

Closes #24046 from HeartSaVioR/MINOR-throw-better-exception-for-tuple-more-than-22.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-11 13:44:45 +09:00
Yuming Wang 8ab13065f6 [SPARK-23807][FOLLOW-UP][BUILD][TEST-HADOOP3.1] Add test-hadoop3.1 phrase
## What changes were proposed in this pull request?

Add `test-hadoop3.1` phrase to test Spark against Spark’s Hadoop 3.1 profile.

## How was this patch tested?
Tested on jenkins. This is output:
```
[info] Using build tool sbt with Hadoop profile hadoop3.1 under environment amplab_jenkins
...
[info] Building Spark (w/Hive 1.2.1) using SBT with these arguments:  -Phadoop-3.1 -Pkubernetes -Phive-thriftserver -Pkinesis-asl -Pyarn -Pspark-ganglia-lgpl -Phive -Pmesos test:package streaming-kinesis-asl-assembly/assembly
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/103282/console

Closes #24045 from wangyum/SPARK-23807.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-11 11:15:58 +09:00
Hyukjin Kwon 93ea353cae [SPARK-26920][R] Deduplicate type checking across Arrow optimization in SparkR
## What changes were proposed in this pull request?

This PR proposes two things.

1.. Deduplicates the type checking logic. While I am here, I checked each type. Currently, binary type, float type, nested struct type and array type are not supported.

**For map and nested struct types:**

 it's expected to be unsupported per Spark's arrow optimization.

```
Exception in thread "serve-Arrow" java.lang.UnsupportedOperationException: Unsupported data type: map<string,double>
...
```
```
Exception in thread "serve-Arrow" java.lang.UnsupportedOperationException: Unsupported data type:  struct<type:tinyint,size:int,indices:array<int>,values:array<double>>
...
```

Please track the trace below to double check.

```
	at org.apache.spark.sql.execution.arrow.ArrowUtils$.toArrowType(ArrowUtils.scala:56)
	at org.apache.spark.sql.execution.arrow.ArrowUtils$.toArrowField(ArrowUtils.scala:92)
	at org.apache.spark.sql.execution.arrow.ArrowUtils$.$anonfun$toArrowSchema$1(ArrowUtils.scala:116)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99)
	at scala.collection.TraversableLike.map(TraversableLike.scala:237)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:230)
	at org.apache.spark.sql.types.StructType.map(StructType.scala:99)
	at org.apache.spark.sql.execution.arrow.ArrowUtils$.toArrowSchema(ArrowUtils.scala:115)
	at org.apache.spark.sql.execution.arrow.ArrowBatchStreamWriter.<init>(ArrowConverters.scala:50)
	at org.apache.spark.sql.Dataset.$anonfun$collectAsArrowToR$2(Dataset.scala:3215)
	at org.apache.spark.sql.Dataset.$anonfun$collectAsArrowToR$2$adapted(Dataset.scala:3212)
```

**For float and binary types:**

They cause corrupt values in some cases. It needs to be investigated separately.

**For array type:**

```
Error in Table__to_dataframe(x, use_threads = use_threads) :
  cannot handle Array of type list
```

Seems to be Arrow's R library limitation. It needs to be investigated separately as well.

2.. While I am touching the type specification codes across Arrow optimization, I move the Arrow optimization related tests into a separate filed called `test_arrow.R`.

## How was this patch tested?

Tests were added and also manually tested.

Closes #23969 from HyukjinKwon/SPARK-26920.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-11 10:23:24 +09:00
Gabor Somogyi 3729efb4d0 [SPARK-26856][PYSPARK] Python support for from_avro and to_avro APIs
## What changes were proposed in this pull request?

Avro is built-in but external data source module since Spark 2.4 but  `from_avro` and `to_avro` APIs not yet supported in pyspark.

In this PR I've made them available from pyspark.

## How was this patch tested?

Please see the python API examples what I've added.

cd docs/
SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll build
Manual webpage check.

Closes #23797 from gaborgsomogyi/SPARK-26856.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-11 10:15:07 +09:00
Gabor Somogyi 29d9021245 [SPARK-24621][WEBUI] Show secure URLs on web pages
## What changes were proposed in this pull request?

Web UI URLs are pointing to `http://` targets even if SSL is enabled. In this PR I've changed the code to point to `https://` URLs.

## How was this patch tested?

Existing unit tests + manually by starting standalone master/worker/spark-shell. Please see jira.

Closes #23991 from gaborgsomogyi/SPARK-24621.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-10 19:28:35 -05:00
Yuming Wang eed3091a60 [SPARK-27120][BUILD][TEST] Upgrade scalatest version to 3.0.5
## What changes were proposed in this pull request?

**ScalaTest 3.0.5 Release Notes**

**Bug Fixes**

- Fixed the implicit view not available problem when used with compile macro.
- Fixed a stack depth problem in RefSpecLike and fixture.SpecLike under Scala 2.13.
- Changed Framework and ScalaTestFramework to set spanScaleFactor for Runner object instances for different Runners using different class loaders. This fixed a problem whereby an incorrect Runner.spanScaleFactor could be used when the tests for multiple sbt project's were run concurrently.
- Fixed a bug in endsWith regex matcher.

**Improvements**
- Removed duplicated parsing code for -C in ArgsParser.
- Improved performance in WebBrowser.
- Documentation typo rectification.
- Improve validity of Junit XML reports.
- Improved performance by replacing all .size == 0 and .length == 0 to .isEmpty.

**Enhancements**
- Added 'C' option to -P, which will tell -P to use cached thread pool.
- External Dependencies Update
- Bumped up scala-js version to 0.6.22.
- Changed to depend on mockito-core, not mockito-all.
- Bumped up jmock version to 2.8.3.
- Bumped up junit version to 4.12.
- Removed dependency to scala-parser-combinators.

More details:
http://www.scalatest.org/release_notes/3.0.5

## How was this patch tested?

manual tests on local machine:
```
nohup build/sbt clean -Djline.terminal=jline.UnsupportedTerminal -Phadoop-2.7  -Pkubernetes -Phive-thriftserver -Pyarn -Pspark-ganglia-lgpl -Phive -Pkinesis-asl -Pmesos test > run.scalatest.log &
```

Closes #24042 from wangyum/SPARK-27120.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-10 15:22:52 -07:00
Takeshi Yamamuro f0927d8ac4 [SPARK-27110][SQL] Moves some functions from AnalyzeColumnCommand to command/CommandUtils
## What changes were proposed in this pull request?
To reuse some common logics for improving `Analyze` commands (See the description of `SPARK-25196` for details), this pr moved some functions from `AnalyzeColumnCommand` to `command/CommandUtils`.  A follow-up pr will add code to extend `Analyze` commands for cached tables.

## How was this patch tested?
Existing tests.

Closes #22204 from maropu/SPARK-25196.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-10 15:17:46 -07:00
Hyukjin Kwon 28d003097b [SPARK-27102][R][PYTHON][CORE] Remove the references to Python's Scala codes in R's Scala codes
## What changes were proposed in this pull request?

Currently, R's Scala codes happened to refer Python's Scala codes for code deduplications. It's a bit odd. For instance, when we face an exception from R, it shows python related code path, which makes confusing to debug. It should rather have one code base and R's and Python's should share.

This PR proposes:

1. Make a `SocketAuthServer` and move `PythonServer` so that `PythonRDD` and `RRDD` can share it.
2. Move `readRDDFromFile` and `readRDDFromInputStream` into `JavaRDD`.
3. Reuse `RAuthHelper` and remove `RSocketAuthHelper` in `RRDD`.
4. Rename `getEncryptionEnabled` to `isEncryptionEnabled` while I am here.

So, now, the places below:

- `sql/core/src/main/scala/org/apache/spark/sql/api/r`
- `core/src/main/scala/org/apache/spark/api/r`
- `mllib/src/main/scala/org/apache/spark/ml/r`

don't refer Python's Scala codes.

## How was this patch tested?

Existing tests should cover this.

Closes #24023 from HyukjinKwon/SPARK-27102.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-10 15:08:23 +09:00
Yuming Wang 470313e660 [SPARK-27118][SQL] Upgrade Hive Metastore Client to the latest versions for Hive 1.0.x/1.1.x
## What changes were proposed in this pull request?

Hive 1.1.1 and Hive 1.0.1 released. We should upgrade Hive Metastore Client version.

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329444&styleName=Text&projectId=12310843
https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329557&styleName=Text&projectId=12310843

## How was this patch tested?

N/A

Closes #24040 from wangyum/SPARK-27118.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:50:10 -08:00
Yuming Wang f732647ae4 [SPARK-27054][BUILD][SQL] Remove the Calcite dependency
## What changes were proposed in this pull request?

Calcite is only used for [runSqlHive](02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L699-L705)) when `hive.cbo.enable=true`([SemanticAnalyzer](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java#L278-L280)).
So we can disable `hive.cbo.enable` and remove Calcite dependency.

## How was this patch tested?

Exist tests

Closes #23970 from wangyum/SPARK-27054.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:34:24 -08:00
Shixiong Zhu 6e1c0827ec
[SPARK-27111][SS] Fix a race that a continuous query may fail with InterruptedException
## What changes were proposed in this pull request?

Before a Kafka consumer gets assigned with partitions, its offset will contain 0 partitions. However, runContinuous will still run and launch a Spark job having 0 partitions. In this case, there is a race that epoch may interrupt the query execution thread after `lastExecution.toRdd`, and either `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` or the next `runContinuous` will get interrupted unintentionally.

To handle this case, this PR has the following changes:

- Clean up the resources in `queryExecutionThread.runUninterruptibly`. This may increase the waiting time of `stop` but should be minor because the operations here are very fast (just sending an RPC message in the same process and stopping a very simple thread).
- Clear the interrupted status at the end so that it won't impact the `runContinuous` call. We may clear the interrupted status set by `stop`, but it doesn't affect the query termination because `runActivatedStream` will check `state` and exit accordingly.

I also updated the clean up codes to make sure exceptions thrown from `epochEndpoint.askSync[Unit](StopContinuousExecutionWrites)` won't stop the clean up.

## How was this patch tested?

Jenkins

Closes #24034 from zsxwing/SPARK-27111.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-03-09 14:26:58 -08:00
Marco Gaido 25bcf59b3b [SPARK-25838][ML] Remove formatVersion from Saveable
## What changes were proposed in this pull request?

`Saveable` interface introduces `formatVersion` which is protected and it is used nowhere. So the PR proposes to remove it.

## How was this patch tested?

existing tests

Closes #22830 from mgaido91/SPARK-25838.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-09 09:44:20 -06:00
CodeGod a29df5fa02 [SPARK-27080][SQL] bug fix: mergeWithMetastoreSchema with uniform lower case comparison
## What changes were proposed in this pull request?
When reading parquet file with merging metastore schema and file schema, we should compare field names using uniform case. In current implementation, lowercase is used but one omission. And this patch fix it.

## How was this patch tested?
Unit test

Closes #24001 from codeborui/mergeSchemaBugFix.

Authored-by: CodeGod <>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-09 21:28:10 +08:00
Kris Mok 57ae251f75 [SPARK-27097] Avoid embedding platform-dependent offsets literally in whole-stage generated code
## What changes were proposed in this pull request?

Spark SQL performs whole-stage code generation to speed up query execution. There are two steps to it:
- Java source code is generated from the physical query plan on the driver. A single version of the source code is generated from a query plan, and sent to all executors.
  - It's compiled to bytecode on the driver to catch compilation errors before sending to executors, but currently only the generated source code gets sent to the executors. The bytecode compilation is for fail-fast only.
- Executors receive the generated source code and compile to bytecode, then the query runs like a hand-written Java program.

In this model, there's an implicit assumption about the driver and executors being run on similar platforms. Some code paths accidentally embedded platform-dependent object layout information into the generated code, such as:
```java
Platform.putLong(buffer, /* offset */ 24, /* value */ 1);
```
This code expects a field to be at offset +24 of the `buffer` object, and sets a value to that field.
But whole-stage code generation generally uses platform-dependent information from the driver. If the object layout is significantly different on the driver and executors, the generated code can be reading/writing to wrong offsets on the executors, causing all kinds of data corruption.

One code pattern that leads to such problem is the use of `Platform.XXX` constants in generated code, e.g. `Platform.BYTE_ARRAY_OFFSET`.

Bad:
```scala
val baseOffset = Platform.BYTE_ARRAY_OFFSET
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will embed the value of `Platform.BYTE_ARRAY_OFFSET` on the driver into the generated code.

Good:
```scala
val baseOffset = "Platform.BYTE_ARRAY_OFFSET"
// codegen template:
s"Platform.putLong($buffer, $baseOffset, $value);"
```
This will generate the offset symbolically -- `Platform.putLong(buffer, Platform.BYTE_ARRAY_OFFSET, value)`, which will be able to pick up the correct value on the executors.

Caveat: these offset constants are declared as runtime-initialized `static final` in Java, so they're not compile-time constants from the Java language's perspective. It does lead to a slightly increased size of the generated code, but this is necessary for correctness.

NOTE: there can be other patterns that generate platform-dependent code on the driver which is invalid on the executors. e.g. if the endianness is different between the driver and the executors, and if some generated code makes strong assumption about endianness, it would also be problematic.

## How was this patch tested?

Added a new test suite `WholeStageCodegenSparkSubmitSuite`. This test suite needs to set the driver's extraJavaOptions to force the driver and executor use different Java object layouts, so it's run as an actual SparkSubmit job.

Authored-by: Kris Mok <kris.mokdatabricks.com>

Closes #24031 from gatorsmile/cherrypickSPARK-27097.

Lead-authored-by: Kris Mok <kris.mok@databricks.com>
Co-authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-03-09 01:20:32 +00:00
liuxian 326fc746d3 [SPARK-27056][MESOS] Remove start-shuffle-service.sh
## What changes were proposed in this pull request?

 `start-shuffle-service.sh`  was only used by Mesos  before `start-mesos-shuffle-service.sh`.
Obviously, `start-mesos-shuffle-service.sh` solves some problems, it is better than  `start-shuffle-service.sh`.
So now we should delete `start-shuffle-service.sh` in case users use it.

## How was this patch tested?
N/A

Closes #23975 from 10110346/rmshuffleservice_sh.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-08 18:51:38 -06:00
sandeep-katta 14f2286e56 [SPARK-27101][PYTHON] Drop the created database after the test in test_session
## What changes were proposed in this pull request?

Cleaning the testcase, drop the database after use

## How was this patch tested?

existing UT

Closes #24021 from sandeep-katta/cleanPythonTest.

Authored-by: sandeep-katta <sandeep.katta2007@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-09 09:12:33 +09:00
Sunitha Kambhampati bd2710bd79 [MINOR][SQL] Fix the typo in the spark.sql.extensions conf doc
## What changes were proposed in this pull request?
Fix the  typo (missing the s)  in the class name (SparkSessionExtensions)  in the doc for Spark conf spark.sql.extensions.

## How was this patch tested?
Verified by checking that the configuration doc shows up correctly in spark-shell using the SET -v

Closes #24020 from skambha/fixnametypo.

Authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-09 08:51:19 +09:00
SongYadong 14b1312727 [SPARK-27103][SQL][MINOR] List SparkSql reserved keywords in alphabet order
## What changes were proposed in this pull request?

This PR tries to correct spark-sql reserved keywords' position in list if they are not in alphabetical order.
In test suite some repeated words are removed. Also some comments are added for remind.

## How was this patch tested?

Existing unit tests.

Closes #23985 from SongYadong/sql_reserved_alphabet.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-08 10:51:39 -08:00