## What changes were proposed in this pull request?
in https://github.com/apache/spark/pull/18064, to work around the nested sql execution id issue, we introduced several internal methods in `Dataset`, like `collectInternal`, `countInternal`, `showInternal`, etc., to avoid nested execution id.
However, this approach has poor expansibility. When we hit other nested execution id cases, we may need to add more internal methods in `Dataset`.
Our goal is to ignore the nested execution id in some cases, and we can have a better approach to achieve this goal, by introducing `SQLExecution.ignoreNestedExecutionId`. Whenever we find a place which needs to ignore the nested execution, we can just wrap the action with `SQLExecution.ignoreNestedExecutionId`, and this is more expansible than the previous approach.
The idea comes from https://github.com/apache/spark/pull/17540/files#diff-ab49028253e599e6e74cc4f4dcb2e3a8R57 by rdblue
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18419 from cloud-fan/follow.
## What changes were proposed in this pull request?
Time windowing in Spark currently performs an Expand + Filter, because there is no way to guarantee the amount of windows a timestamp will fall in, in the general case. However, for tumbling windows, a record is guaranteed to fall into a single bucket. In this case, doubling the number of records with Expand is wasteful, and can be improved by using a simple Projection instead.
Benchmarks show that we get an order of magnitude performance improvement after this patch.
## How was this patch tested?
Existing unit tests. Benchmarked using the following code:
```scala
import org.apache.spark.sql.functions._
spark.time {
spark.range(numRecords)
.select(from_unixtime((current_timestamp().cast("long") * 1000 + 'id / 1000) / 1000) as 'time)
.select(window('time, "10 seconds"))
.count()
}
```
Setup:
- 1 c3.2xlarge worker (8 cores)
![image](https://user-images.githubusercontent.com/5243515/27348748-ed991b84-55a9-11e7-8f8b-6e7abc524417.png)
1 B rows ran in 287 seconds after this optimization. I didn't wait for it to finish without the optimization. Shows about 5x improvement for large number of records.
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#18364 from brkyvz/opt-tumble.
## What changes were proposed in this pull request?
Storage URI of a partitioned table may or may not point to a directory under which individual partitions are stored. In fact, individual partitions may be located in totally unrelated directories. Before this change, ANALYZE TABLE table COMPUTE STATISTICS command calculated total size of a table by adding up sizes of files found under table's storage URI. This calculation could produce 0 if partitions are stored elsewhere.
This change uses storage URIs of individual partitions to calculate the sizes of all partitions of a table and adds these up to produce the total size of a table.
CC: wzhfy
## How was this patch tested?
Added unit test.
Ran ANALYZE TABLE xxx COMPUTE STATISTICS on a partitioned Hive table and verified that sizeInBytes is calculated correctly. Before this change, the size would be zero.
Author: Masha Basmanova <mbasmanova@fb.com>
Closes#18309 from mbasmanova/mbasmanova-analyze-part-table.
### What changes were proposed in this pull request?
```SQL
CREATE TABLE `tab1`
(`custom_fields` ARRAY<STRUCT<`id`: BIGINT, `value`: STRING>>)
USING parquet
INSERT INTO `tab1`
SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b'))
SELECT custom_fields.id, custom_fields.value FROM tab1
```
The above query always return the last struct of the array, because the rule `SimplifyCasts` incorrectly rewrites the query. The underlying cause is we always use the same `GenericInternalRow` object when doing the cast.
### How was this patch tested?
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18412 from gatorsmile/castStruct.
## What changes were proposed in this pull request?
This PR is to revert some code changes in the read path of https://github.com/apache/spark/pull/14377. The original fix is https://github.com/apache/spark/pull/17830
When merging this PR, please give the credit to gaborfeher
## How was this patch tested?
Added a test case to OracleIntegrationSuite.scala
Author: Gabor Feher <gabor.feher@lynxanalytics.com>
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18408 from gatorsmile/OracleType.
## What changes were proposed in this pull request?
This pr supported a DDL-formatted string in `DataStreamReader.schema`.
This fix could make users easily define a schema without importing the type classes.
For example,
```scala
scala> spark.readStream.schema("col0 INT, col1 DOUBLE").load("/tmp/abc").printSchema()
root
|-- col0: integer (nullable = true)
|-- col1: double (nullable = true)
```
## How was this patch tested?
Added tests in `DataStreamReaderWriterSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18373 from HyukjinKwon/SPARK-20431.
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050
Since SQL Generation is removed, we do not need to keep `isTableSample`.
## How was this patch tested?
The existing test cases
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18379 from gatorsmile/CleanSample.
## What changes were proposed in this pull request?
Currently we do a lot of validations for subquery in the Analyzer. We should move them to CheckAnalysis which is the framework to catch and report Analysis errors. This was mentioned as a review comment in SPARK-18874.
## How was this patch tested?
Exists tests + A few tests added to SQLQueryTestSuite.
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#17713 from dilipbiswal/subquery_checkanalysis.
## What changes were proposed in this pull request?
* Following the first few examples in this file, the remaining methods should also be methods of `df.na` not `df`.
* Filled in some missing parentheses
## How was this patch tested?
N/A
Author: Ong Ming Yang <me@ongmingyang.com>
Closes#18398 from ongmingyang/master.
## What changes were proposed in this pull request?
If the SQL conf for StateStore provider class is changed between restarts (i.e. query started with providerClass1 and attempted to restart using providerClass2), then the query will fail in a unpredictable way as files saved by one provider class cannot be used by the newer one.
Ideally, the provider class used to start the query should be used to restart the query, and the configuration in the session where it is being restarted should be ignored.
This PR saves the provider class config to OffsetSeqLog, in the same way # shuffle partitions is saved and recovered.
## How was this patch tested?
new unit tests
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#18402 from tdas/SPARK-21192.
## What changes were proposed in this pull request?
After wiring `SQLConf` in logical plan ([PR 18299](https://github.com/apache/spark/pull/18299)), we can remove the need of passing `conf` into `def stats` and `def computeStats`.
## How was this patch tested?
Covered by existing tests, plus some modified existing tests.
Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>
Closes#18391 from wzhfy/removeConf.
## What changes were proposed in this pull request?
The current master outputs unexpected results when the data schema and partition schema have the duplicate columns:
```
withTempPath { dir =>
val basePath = dir.getCanonicalPath
spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=1").toString)
spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=a").toString)
spark.read.parquet(basePath).show()
}
+---+
|foo|
+---+
| 1|
| 1|
| a|
| a|
| 1|
| a|
+---+
```
This patch added code to print a warning when the duplication found.
## How was this patch tested?
Manually checked.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#18375 from maropu/SPARK-21144-3.
## What changes were proposed in this pull request?
Current ColumnarBatchSuite has very simple test cases for `Array` and `Struct`. This pr wants to add some test suites for complicated cases in ColumnVector.
Author: jinxing <jinxing6042@126.com>
Closes#18327 from jinxing64/SPARK-21047.
## What changes were proposed in this pull request?
StateStoreProvider instances are loaded on-demand in a executor when a query is started. When a query is restarted, the loaded provider instance will get reused. Now, there is a non-trivial chance, that the task of the previous query run is still running, while the tasks of the restarted run has started. So for a stateful partition, there may be two concurrent tasks related to the same stateful partition, and there for using the same provider instance. This can lead to inconsistent results and possibly random failures, as state store implementations are not designed to be thread-safe.
To fix this, I have introduced a `StateStoreProviderId`, that unique identifies a provider loaded in an executor. It has the query run id in it, thus making sure that restarted queries will force the executor to load a new provider instance, thus avoiding two concurrent tasks (from two different runs) from reusing the same provider instance.
Additional minor bug fixes
- All state stores related to query run is marked as deactivated in the `StateStoreCoordinator` so that the executors can unload them and clear resources.
- Moved the code that determined the checkpoint directory of a state store from implementation-specific code (`HDFSBackedStateStoreProvider`) to non-specific code (StateStoreId), so that implementation do not accidentally get it wrong.
- Also added store name to the path, to support multiple stores per sql operator partition.
*Note:* This change does not address the scenario where two tasks of the same run (e.g. speculative tasks) are concurrently running in the same executor. The chance of this very small, because ideally speculative tasks should never run in the same executor.
## How was this patch tested?
Existing unit tests + new unit test.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#18355 from tdas/SPARK-21145.
## What changes were proposed in this pull request?
Currently the validation of sampling fraction in dataset is incomplete.
As an improvement, validate sampling fraction in logical operator level:
1) if with replacement: fraction should be nonnegative
2) else: fraction should be on interval [0, 1]
Also add test cases for the validation.
## How was this patch tested?
integration tests
gatorsmile cloud-fan
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Wang Gengliang <ltnwgl@gmail.com>
Closes#18387 from gengliangwang/sample_ratio_validate.
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.
Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).
## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly.
Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>
Closes#15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
## What changes were proposed in this pull request?
Currently, if we read a batch and want to display it on the console sink, it will lead a runtime exception.
Changes:
- In this PR, we add a match rule to check whether it is a ConsoleSinkProvider, we will display the Dataset
if using console format.
## How was this patch tested?
spark.read.schema().json(path).write.format("console").save
Author: Lubo Zhang <lubo.zhang@intel.com>
Author: lubozhan <lubo.zhang@intel.com>
Closes#18347 from lubozhan/dev.
## What changes were proposed in this pull request?
Fix incomplete documentation for `lpad`.
Author: actuaryzhang <actuaryzhang10@gmail.com>
Closes#18367 from actuaryzhang/SQLDoc.
## What changes were proposed in this pull request?
Decode the path generated by File sink to handle special characters.
## How was this patch tested?
The added unit test.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18381 from zsxwing/SPARK-21167.
## What changes were proposed in this pull request?
This PR proposes to throw an exception if a schema is provided by user to socket source as below:
**socket source**
```scala
import org.apache.spark.sql.types._
val userSpecifiedSchema = StructType(
StructField("name", StringType) ::
StructField("area", StringType) :: Nil)
val df = spark.readStream.format("socket").option("host", "localhost").option("port", 9999).schema(userSpecifiedSchema).load
df.printSchema
```
Before
```
root
|-- value: string (nullable = true)
```
After
```
org.apache.spark.sql.AnalysisException: The socket source does not support a user-specified schema.;
at org.apache.spark.sql.execution.streaming.TextSocketSourceProvider.sourceSchema(socket.scala:199)
at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
... 50 elided
```
**rate source**
```scala
spark.readStream.format("rate").schema(spark.range(1).schema).load().printSchema()
```
Before
```
root
|-- timestamp: timestamp (nullable = true)
|-- value: long (nullable = true)`
```
After
```
org.apache.spark.sql.AnalysisException: The rate source does not support a user-specified schema.;
at org.apache.spark.sql.execution.streaming.RateSourceProvider.sourceSchema(RateSourceProvider.scala:57)
at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
... 48 elided
```
## How was this patch tested?
Unit test in `TextSocketStreamSuite` and `RateSourceSuite`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18365 from HyukjinKwon/SPARK-21147.
## What changes were proposed in this pull request?
Currently we have several tens of test sqls in catalyst will fail at `SimpleAnalyzer.checkAnalysis`, we should make sure they are valid.
This PR makes the following changes:
1. Apply `checkAnalysis` on plans that tests `Optimizer` rules, but don't require the testcases for `Parser`/`Analyzer` pass `checkAnalysis`;
2. Fix testcases for `Optimizer` that would have fall.
## How was this patch tested?
Apply `SimpleAnalyzer.checkAnalysis` on plans in `PlanTest.comparePlans`, update invalid test cases.
Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Author: jiangxingbo <jiangxb1987@gmail.com>
Closes#15417 from jiangxb1987/cptest.
This patch adds DB2 specific data type mappings for decfloat, real, xml , and timestamp with time zone (DB2Z specific type) types on read and for byte, short data types on write to the to jdbc data source DB2 dialect. Default mapping does not work for these types when reading/writing from DB2 database.
Added docker test, and a JDBC unit test case.
Author: sureshthalamati <suresh.thalamati@gmail.com>
Closes#9162 from sureshthalamati/db2dialect_enhancements-spark-10655.
## What changes were proposed in this pull request?
This is a regression in Spark 2.2. In Spark 2.2, we introduced a new way to resolve persisted view: https://issues.apache.org/jira/browse/SPARK-18209 , but this makes the persisted view non case-preserving because we store the schema in hive metastore directly. We should follow data source table and store schema in table properties.
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18360 from cloud-fan/view.
## What changes were proposed in this pull request?
Fix some typo of the document.
## How was this patch tested?
Existing tests.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Xianyang Liu <xianyang.liu@intel.com>
Closes#18350 from ConeyLiu/fixtypo.
## What changes were proposed in this pull request?
This fix tries to address the issue in SPARK-19975 where we
have `map_keys` and `map_values` functions in SQL yet there
is no Python equivalent functions.
This fix adds `map_keys` and `map_values` functions to Python.
## How was this patch tested?
This fix is tested manually (See Python docs for examples).
Author: Yong Tang <yong.tang.github@outlook.com>
Closes#17328 from yongtang/SPARK-19975.
## What changes were proposed in this pull request?
Built-in SQL Function UnaryMinus/UnaryPositive support string type, if it's string type, convert it to double type, after this PR:
```sql
spark-sql> select positive('-1.11'), negative('-1.11');
-1.11 1.11
spark-sql>
```
## How was this patch tested?
unit tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#18173 from wangyum/SPARK-20948.
## What changes were proposed in this pull request?
Previous code mistakenly use `table.properties.get("comment")` to read the existing table comment, we should use `table.comment`
## How was this patch tested?
new regression test
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18325 from cloud-fan/unset.
## What changes were proposed in this pull request?
This PR adds built-in SQL function `BIT_LENGTH()`, `CHAR_LENGTH()`, and `OCTET_LENGTH()` functions.
`BIT_LENGTH()` returns the bit length of the given string or binary expression.
`CHAR_LENGTH()` returns the length of the given string or binary expression. (i.e. equal to `LENGTH()`)
`OCTET_LENGTH()` returns the byte length of the given string or binary expression.
## How was this patch tested?
Added new test suites for these three functions
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18046 from kiszk/SPARK-20749.
### What changes were proposed in this pull request?
`ALTER TABLE SET TBLPROPERTIES` should not overwrite `COMMENT` even if the input property does not have the property of `COMMENT`. This PR is to fix the issue.
### How was this patch tested?
Covered by the existing tests.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18318 from gatorsmile/fixTableComment.
## What changes were proposed in this pull request?
This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:
1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.
## How was this patch tested?
Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.
Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.
Author: ALeksander Eskilson <alek.eskilson@cerner.com>
Closes#18075 from bdrillard/class_splitting_only.
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.
### How was this patch tested?
N/A
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18202 from gatorsmile/renameCVSOption.
## What changes were proposed in this pull request?
It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.
The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.
In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).
## How was this patch tested?
Updated relevant tests for constraint propagation.
Author: Reynold Xin <rxin@databricks.com>
Closes#18299 from rxin/SPARK-21092.
### What changes were proposed in this pull request?
Since both table properties and storage properties share the same key values, table properties are not shown in the output of DESC EXTENDED/FORMATTED when the storage properties are not empty.
This PR is to fix the above issue by renaming them to different keys.
### How was this patch tested?
Added test cases.
Author: Xiao Li <gatorsmile@gmail.com>
Closes#18294 from gatorsmile/tableProperties.
## What changes were proposed in this pull request?
After PruneFileSourcePartitions rule, It needs reset table's statistics because PruneFileSourcePartitions can filter some unnecessary partitions. So the statistics need to be changed.
## How was this patch tested?
add unit test.
Author: lianhuiwang <lianhuiwang09@gmail.com>
Closes#18205 from lianhuiwang/SPARK-20986.
## What changes were proposed in this pull request?
When converting `string` to `number`(int, long or double), if the string has a space before or after,will lead to unnecessary mistakes.
## How was this patch tested?
unit test
Author: liuxian <liu.xian3@zte.com.cn>
Closes#18238 from 10110346/lx-wip-0608.
## What changes were proposed in this pull request?
This adds the average hash map probe metrics to hash aggregate.
`BytesToBytesMap` already has API to get the metrics, this PR adds an API to `UnsafeFixedWidthAggregationMap` to access it.
Preparing a test for this metrics seems tricky, because we don't know what collision keys are. For now, the test case generates random data large enough to have desired probe.
TODO in later PR: add hash map metrics to join.
## How was this patch tested?
Added test to SQLMetricsSuite.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes#18258 from viirya/SPARK-20953.
## What changes were proposed in this pull request?
To use treeAggregate instead of aggregate in DataFrame.stat.bloomFilter to parallelize the operation of merging the bloom filters
(Please fill in changes proposed in this fix)
## How was this patch tested?
unit tests passed
(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.
Author: Rishabh Bhardwaj <rbnext29@gmail.com>
Author: Rishabh Bhardwaj <admin@rishabh.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.Dlink>
Author: Rishabh Bhardwaj <admin@Admins-MacBook-Pro.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.local>
Closes#18263 from rishabhbhardwaj/SPARK-21039.
## What changes were proposed in this pull request?
Don't leave thread pool running from AlterTableRecoverPartitionsCommand DDL command
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#18216 from srowen/SPARK-20920.
## What changes were proposed in this pull request?
Since `stack` function generates a table with nullable columns, it should allow mixed null values.
```scala
scala> sql("select stack(3, 1, 2, 3)").printSchema
root
|-- col0: integer (nullable = true)
scala> sql("select stack(3, 1, 2, null)").printSchema
org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
```
## How was this patch tested?
Pass the Jenkins with a new test case.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#17251 from dongjoon-hyun/SPARK-19910.
## What changes were proposed in this pull request?
This PR adds RateSource for Structured Streaming so that the user can use it to generate data for tests and benchmark easily.
This source generates increment long values with timestamps. Each generated row has two columns: a timestamp column for the generated time and an auto increment long column starting with 0L.
It supports the following options:
- `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second.
- `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer seconds.
- `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. The source will try its best to reach `rowsPerSecond`, but the query may be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.
Here is a simple example that prints 10 rows per seconds:
```
spark.readStream
.format("rate")
.option("rowsPerSecond", "10")
.load()
.writeStream
.format("console")
.start()
```
The idea came from marmbrus and he did the initial work.
## How was this patch tested?
The added tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Author: Michael Armbrust <michael@databricks.com>
Closes#18199 from zsxwing/rate.
## What changes were proposed in this pull request?
Currently when a `ColumnVector` stores array type elements, we will use 2 arrays for lengths and offsets and implement them individually in on-heap and off-heap column vector.
In this PR, we use one array to represent both offsets and lengths, so that we can treat it as `ColumnVector` and all the logic can go to the base class `ColumnVector`
## How was this patch tested?
existing tests.
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18260 from cloud-fan/put.
## What changes were proposed in this pull request?
This PR fixes the inconsistency in `SparkSession.range`.
**BEFORE**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array(9223372036854775804, 9223372036854775805, 9223372036854775806)
```
**AFTER**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array()
```
## How was this patch tested?
Pass the Jenkins with newly added test cases.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#18257 from dongjoon-hyun/SPARK-21041.
## What changes were proposed in this pull request?
Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.
All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.
Interfaces/abstract classes use the following implementations:
* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`
## How was this patch tested?
```bash
build/mvn -DskipTests clean package && dev/run-tests
```
Additionally in Spark shell:
```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true
scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```
Author: Michal Senkyr <mike.senkyr@gmail.com>
Closes#18009 from michalsenkyr/dataset-java-lists.