Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
237 commits
Select commit Hold shift + click to select a range
4503632
[SPARK-17065][SQL] Improve the error message when encountering an inc…
zsxwing Aug 15, 2016
2e2c787
[SPARK-16964][SQL] Remove private[hive] from sql.hive.execution package
hvanhovell Aug 16, 2016
237ae54
Revert "[SPARK-16964][SQL] Remove private[hive] from sql.hive.executi…
rxin Aug 16, 2016
1c56971
[SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.ex…
hvanhovell Aug 16, 2016
022230c
[SPARK-16519][SPARKR] Handle SparkR RDD generics that create warnings…
felixcheung Aug 16, 2016
6cb3eab
[SPARK-17089][DOCS] Remove api doc link for mapReduceTriplets operator
phalodi Aug 16, 2016
3e0163b
[SPARK-17084][SQL] Rename ParserUtils.assert to validate
hvanhovell Aug 17, 2016
68a24d3
[MINOR][DOC] Fix the descriptions for `properties` argument in the do…
Aug 17, 2016
22c7660
[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grow…
kiszk Aug 17, 2016
394d598
[SPARK-17102][SQL] bypass UserDefinedGenerator for json format check
cloud-fan Aug 17, 2016
9406f82
[SPARK-17096][SQL][STREAMING] Improve exception string reported throu…
tdas Aug 17, 2016
585d1d9
[SPARK-17038][STREAMING] fix metrics retrieval source of 'lastReceive…
keypointt Aug 17, 2016
91aa532
[SPARK-16995][SQL] TreeNodeException when flat mapping RelationalGrou…
viirya Aug 18, 2016
5735b8b
[SPARK-16391][SQL] Support partial aggregation for reduceGroups
rxin Aug 18, 2016
ec5f157
[SPARK-17117][SQL] 1 / NULL should not fail analysis
petermaxlee Aug 18, 2016
176af17
[MINOR][SPARKR] R API documentation for "coltypes" is confusing
keypointt Aug 10, 2016
ea684b6
[SPARK-17069] Expose spark.range() as table-valued function in SQL
ericl Aug 18, 2016
c180d63
[SPARK-16947][SQL] Support type coercion and foldable expression for …
petermaxlee Aug 19, 2016
05b180f
HOTFIX: compilation broken due to protected ctor.
rxin Aug 19, 2016
d55d1f4
[SPARK-16961][CORE] Fixed off-by-one error that biased randomizeInPlace
nicklavers Aug 19, 2016
e0c60f1
[SPARK-16994][SQL] Whitelist operators for predicate pushdown
rxin Aug 19, 2016
d0707c6
[SPARK-11227][CORE] UnknownHostException can be thrown when NameNode …
sarutak Aug 19, 2016
3276ccf
[SPARK-16686][SQL] Remove PushProjectThroughSample since it is handle…
viirya Jul 26, 2016
ae89c8e
[SPARK-17113] [SHUFFLE] Job failure due to Executor OOM in offheap mode
Aug 19, 2016
efe8322
[SPARK-17149][SQL] array.sql for testing array related functions
petermaxlee Aug 20, 2016
379b127
[SPARK-17158][SQL] Change error message for out of range numeric lite…
srinathshankar Aug 20, 2016
f7458c7
[SPARK-17150][SQL] Support SQL generation for inline tables
petermaxlee Aug 20, 2016
4c4c275
[SPARK-17104][SQL] LogicalRelation.newInstance should follow the sema…
viirya Aug 20, 2016
24dd9a7
[SPARK-17124][SQL] RelationalGroupedDataset.agg should preserve order…
petermaxlee Aug 20, 2016
faff929
[SPARK-12666][CORE] SparkSubmit packages fix for when 'default' conf …
BryanCutler Aug 20, 2016
26d5a8b
[MINOR][R] add SparkR.Rcheck/ and SparkR_*.tar.gz to R/.gitignore
mengxr Aug 21, 2016
0297896
[SPARK-16508][SPARKR] Fix CRAN undocumented/duplicated arguments warn…
junyangq Aug 20, 2016
e62b29f
[SPARK-17098][SQL] Fix `NullPropagation` optimizer to handle `COUNT(N…
dongjoon-hyun Aug 21, 2016
49cc44d
[SPARK-17115][SQL] decrease the threshold when split expressions
Aug 22, 2016
2add45f
[SPARK-17085][STREAMING][DOCUMENTATION AND ACTUAL CODE DIFFERS - UNSU…
jagadeesanas2 Aug 22, 2016
7919598
[SPARKR][MINOR] Fix Cache Folder Path in Windows
junyangq Aug 22, 2016
94eff08
[SPARK-16320][DOC] Document G1 heap region's effect on spark 2.0 vs 1.6
srowen Aug 22, 2016
6dcc1a3
[SPARKR][MINOR] Add Xiangrui and Felix to maintainers
shivaram Aug 22, 2016
01a4d69
[SPARK-17162] Range does not support SQL generation
ericl Aug 22, 2016
b65b041
[SPARK-16508][SPARKR] doc updates and more CRAN check fixes
felixcheung Aug 22, 2016
ff2f873
[SPARK-16550][SPARK-17042][CORE] Certain classes fail to deserialize …
ericl Aug 22, 2016
2258989
[SPARK-16577][SPARKR] Add CRAN documentation checks to run-tests.sh
shivaram Aug 23, 2016
eaea1c8
[SPARK-17182][SQL] Mark Collect as non-deterministic
liancheng Aug 23, 2016
d16f9a0
[SPARKR][MINOR] Update R DESCRIPTION file
felixcheung Aug 23, 2016
811a2ce
[SPARK-13286] [SQL] add the next expression of SQLException as cause
Aug 23, 2016
cc40189
[SPARKR][MINOR] Remove reference link for common Windows environment …
junyangq Aug 23, 2016
a2a7506
[MINOR][DOC] Use standard quotes instead of "curly quote" marks from …
HyukjinKwon Aug 23, 2016
a772b4b
[SPARK-17194] Use single quotes when generating SQL for string literals
JoshRosen Aug 23, 2016
a6e6a04
[MINOR][SQL] Remove implemented functions from comments of 'HiveSessi…
weiqingy Aug 24, 2016
df87f16
[SPARK-17186][SQL] remove catalog table type INDEX
cloud-fan Aug 24, 2016
ce7dce1
[MINOR][BUILD] Fix Java CheckStyle Error
weiqingy Aug 24, 2016
33d79b5
[SPARK-17086][ML] Fix InvalidArgumentException issue in QuantileDiscr…
Aug 24, 2016
29091d7
[SPARKR][MINOR] Fix doc for show method
junyangq Aug 24, 2016
9f924a0
[SPARK-16781][PYSPARK] java launched by PySpark as gateway may not be…
srowen Aug 24, 2016
4327337
[SPARKR][MINOR] Add more examples to window function docs
junyangq Aug 24, 2016
9f363a6
[SPARKR][MINOR] Add installation message for remote master mode and i…
junyangq Aug 24, 2016
3258f27
[SPARK-16216][SQL][BRANCH-2.0] Backport Read/write dateFormat/timesta…
HyukjinKwon Aug 25, 2016
aa57083
[SPARK-17228][SQL] Not infer/propagate non-deterministic constraints
sameeragarwal Aug 25, 2016
c1c4980
[SPARK-17193][CORE] HadoopRDD NPE at DEBUG log level when getLocation…
srowen Aug 25, 2016
fb1c697
[SPARK-17061][SPARK-17093][SQL] MapObjects` should make copies of uns…
lw-lin Aug 25, 2016
88481ea
Revert "[SPARK-17061][SPARK-17093][SQL] MapObjects` should make copie…
hvanhovell Aug 25, 2016
184e78b
[SPARK-17061][SPARK-17093][SQL][BACKPORT] MapObjects should make copi…
lw-lin Aug 25, 2016
48ecf3d
[SPARK-16991][SPARK-17099][SPARK-17120][SQL] Fix Outer Join Eliminati…
gatorsmile Aug 25, 2016
2b32a44
[SPARK-17167][2.0][SQL] Issue Exceptions when Analyze Table on In-Mem…
gatorsmile Aug 25, 2016
356a359
[SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schema
Aug 15, 2016
55db262
[SPARK-15083][WEB UI] History Server can OOM due to unlimited TaskUIData
ajbozarth Aug 25, 2016
b3a4430
[SPARKR][BUILD] ignore cran-check.out under R folder
wangmiao1981 Aug 25, 2016
ff2e270
[SPARK-17205] Literal.sql should handle Infinity and NaN
JoshRosen Aug 25, 2016
73014a2
[SPARK-17231][CORE] Avoid building debug or trace log messages unless…
Aug 25, 2016
27ed6d5
[SPARK-17242][DOCUMENT] Update links of external dstream projects
zsxwing Aug 26, 2016
6f82d2d
[SPARKR][MINOR] Fix example of spark.naiveBayes
junyangq Aug 26, 2016
deb6a54
[SPARK-17165][SQL] FileStreamSource should not track the list of seen…
petermaxlee Aug 26, 2016
52feb3f
[SPARK-17246][SQL] Add BigDecimal literal
hvanhovell Aug 26, 2016
dfdfc30
[SPARK-17235][SQL] Support purging of old logs in MetadataLog
petermaxlee Aug 26, 2016
9c0ac6b
[SPARK-17244] Catalyst should not pushdown non-deterministic join con…
sameeragarwal Aug 26, 2016
94d52d7
[SPARK-17269][SQL] Move finish analysis optimization stage into its o…
rxin Aug 27, 2016
f91614f
[SPARK-17270][SQL] Move object optimization rules into its own file (…
rxin Aug 27, 2016
901ab06
[SPARK-17274][SQL] Move join optimizer rules into a separate file
rxin Aug 27, 2016
56a8426
[SPARK-15382][SQL] Fix a bug in sampling with replacement
maropu Aug 27, 2016
7306c5f
[ML][MLLIB] The require condition and message doesn't match in Sparse…
Aug 27, 2016
5487fa0
[SPARK-17216][UI] fix event timeline bars length
Aug 27, 2016
eec0371
[SPARK-16216][SQL][FOLLOWUP][BRANCH-2.0] Bacoport enabling timestamp …
HyukjinKwon Aug 28, 2016
3d283f6
[SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hiv…
Aug 29, 2016
976a43d
[SPARK-16581][SPARKR] Make JVM backend calling functions public
shivaram Aug 29, 2016
5903257
[SPARK-17301][SQL] Remove unused classTag field from AtomicType base …
JoshRosen Aug 30, 2016
f35b10a
[SPARK-17264][SQL] DataStreamWriter should document that it only supp…
srowen Aug 30, 2016
bc6c0d9
[SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with …
zsxwing Aug 31, 2016
021aa28
[SPARK-17243][WEB UI] Spark 2.0 History Server won't load with very l…
ajbozarth Aug 31, 2016
c17334e
[SPARK-17316][CORE] Make CoarseGrainedSchedulerBackend.removeExecutor…
zsxwing Aug 31, 2016
ad36892
[SPARK-17326][SPARKR] Fix tests with HiveContext in SparkR not to be …
HyukjinKwon Aug 31, 2016
d01251c
[SPARK-17316][TESTS] Fix MesosCoarseGrainedSchedulerBackendSuite
zsxwing Aug 31, 2016
8d15c1a
[SPARK-16581][SPARKR] Fix JVM API tests in SparkR
shivaram Aug 31, 2016
191d996
[SPARK-17180][SPARK-17309][SPARK-17323][SQL][2.0] create AlterViewAsC…
cloud-fan Sep 1, 2016
8711b45
[SPARKR][MINOR] Fix windowPartitionBy example
junyangq Sep 1, 2016
6281b74
[SPARK-17318][TESTS] Fix ReplSuite replicating blocks of object with …
zsxwing Sep 1, 2016
13bacd7
[SPARK-17271][SQL] Planner adds un-necessary Sort even if child orde…
tejasapatil Sep 1, 2016
ac22ab0
[SPARK-16926] [SQL] Remove partition columns from partition metadata.
bchocho Sep 1, 2016
dd377a5
[SPARK-17355] Workaround for HIVE-14684 / HiveResultSetMetaData.isSig…
JoshRosen Sep 1, 2016
f946323
[SPARK-17342][WEBUI] Style of event timeline is broken
sarutak Sep 2, 2016
171bdfd
[SPARK-16883][SPARKR] SQL decimal type is not properly cast to number…
wangmiao1981 Sep 2, 2016
d9d10ff
[SPARK-17352][WEBUI] Executor computing time can be negative-number b…
sarutak Sep 2, 2016
91a3cf1
[SPARK-16935][SQL] Verification of Function-related ExternalCatalog APIs
gatorsmile Sep 2, 2016
30e5c84
[SPARK-17261] [PYSPARK] Using HiveContext after re-creating SparkCont…
zjffdu Sep 2, 2016
29ac2f6
[SPARK-17376][SPARKR] Spark version should be available in R
felixcheung Sep 2, 2016
d4ae35d
[SPARKR][DOC] regexp_extract should doc that it returns empty string …
felixcheung Sep 2, 2016
03d9af6
[SPARK-17376][SPARKR] followup - change since version
felixcheung Sep 2, 2016
c9c36fa
[SPARK-17230] [SQL] Should not pass optimized query into QueryExecuti…
Sep 2, 2016
a3930c3
[SPARK-16334] Reusing same dictionary column for decoding consecutive…
sameeragarwal Sep 2, 2016
b8f65da
Fix build
davies Sep 2, 2016
c0ea770
Revert "[SPARK-16334] Reusing same dictionary column for decoding con…
davies Sep 2, 2016
12a2e2a
[SPARKR][MINOR] Fix docs for sparkR.session and count
junyangq Sep 3, 2016
949544d
[SPARK-17347][SQL][EXAMPLES] Encoder in Dataset example has incorrect…
CodingCat Sep 3, 2016
196d62e
[MINOR][SQL] Not dropping all necessary tables
techaddict Sep 3, 2016
a7f5e70
[SPARK-16959][SQL] Rebuild Table Comment when Retrieving Metadata fro…
gatorsmile Aug 10, 2016
3500dbc
[SPARK-16663][SQL] desc table should be consistent between data sourc…
cloud-fan Jul 26, 2016
704215d
[SPARK-17335][SQL] Fix ArrayType and MapType CatalogString.
hvanhovell Sep 3, 2016
e387c8b
[SPARK-17391][TEST][2.0] Fix Two Test Failures After Backport
gatorsmile Sep 5, 2016
f92d874
[SPARK-17353][SPARK-16943][SPARK-16942][BACKPORT-2.0][SQL] Fix multip…
gatorsmile Sep 6, 2016
7b1aa21
[SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException du…
clockfly Sep 6, 2016
dd27530
[SPARK-17358][SQL] Cached table(parquet/orc) should be shard between …
watermen Sep 6, 2016
f56b70f
Revert "[SPARK-17369][SQL] MetastoreRelation toJSON throws AssertExce…
yhuai Sep 6, 2016
286ccd6
[SPARK-17369][SQL][2.0] MetastoreRelation toJSON throws AssertExcepti…
clockfly Sep 6, 2016
c0f1f53
[SPARK-17356][SQL] Fix out of memory issue when generating JSON for T…
clockfly Sep 6, 2016
95e44dc
[SPARK-16922] [SPARK-17211] [SQL] make the address of values portable…
Sep 6, 2016
5343804
[SPARK-16334] [BACKPORT] Reusing same dictionary column for decoding …
sameeragarwal Sep 6, 2016
130a80f
[SPARK-17378][BUILD] Upgrade snappy-java to 1.1.2.6
a-roberts Sep 6, 2016
0ae9786
[SPARK-17299] TRIM/LTRIM/RTRIM should not strips characters other tha…
techaddict Sep 6, 2016
0157514
[SPARK-17110] Fix StreamCorruptionException in BlockManager.getRemote…
JoshRosen Sep 6, 2016
f3cfce0
[SPARK-17316][CORE] Fix the 'ask' type parameter in 'removeExecutor'
zsxwing Sep 6, 2016
a23d406
[SPARK-17279][SQL] better error message for exceptions during ScalaUD…
cloud-fan Sep 6, 2016
796577b
[SPARK-17372][SQL][STREAMING] Avoid serialization issues by using Arr…
tdas Sep 7, 2016
ee6301a
[SPARK-16785] R dapply doesn't return array or raw columns
clarkfitzg Sep 7, 2016
c8811ad
[SPARK-17296][SQL] Simplify parser join processing [BACKPORT 2.0]
hvanhovell Sep 7, 2016
e6caceb
[MINOR][SQL] Fixing the typo in unit test
Sep 7, 2016
078ac0e
[SPARK-17370] Shuffle service files not invalidated when a slave is lost
ericl Sep 7, 2016
067752c
[SPARK-16533][CORE] - backport driver deadlock fix to 2.0
Sep 7, 2016
28377da
[SPARK-17339][CORE][BRANCH-2.0] Do not use path to get a filesystem i…
HyukjinKwon Sep 8, 2016
e169085
[SPARK-16711] YarnShuffleService doesn't re-init properly on YARN rol…
Sep 8, 2016
c6e0dd1
[SPARK-17442][SPARKR] Additional arguments in write.df are not passed…
felixcheung Sep 8, 2016
a7f1c18
[SPARK-17456][CORE] Utility for parsing Spark versions
jkbradley Sep 9, 2016
6f02f40
[SPARK-17354] [SQL] Partitioning by dates/timestamps should work with…
HyukjinKwon Sep 9, 2016
c2378a6
[SPARK-17396][CORE] Share the task support between UnionRDD instances.
rdblue Sep 10, 2016
bde5452
[SPARK-17439][SQL] Fixing compression issues with approximate quantil…
thunterdb Sep 11, 2016
d293062
[SPARK-17336][PYSPARK] Fix appending multiple times to PYTHONPATH fro…
BryanCutler Sep 11, 2016
3052152
[SPARK-17486] Remove unused TaskMetricsUIData.updatedBlockStatuses field
JoshRosen Sep 12, 2016
0a36e36
[SPARK-17503][CORE] Fix memory leak in Memory store when unable to ca…
clockfly Sep 12, 2016
37f45bf
[SPARK-14818] Post-2.0 MiMa exclusion and build changes
JoshRosen Sep 12, 2016
a3fc576
[SPARK-17485] Prevent failed remote reads of cached blocks from faili…
JoshRosen Sep 12, 2016
1f72e77
[SPARK-17474] [SQL] fix python udf in TakeOrderedAndProjectExec
Sep 12, 2016
b17f10c
[SPARK-17515] CollectLimit.execute() should perform per-partition limits
JoshRosen Sep 13, 2016
c142645
[SPARK-17531] Don't initialize Hive Listeners for the Execution Client
brkyvz Sep 13, 2016
12ebfbe
[SPARK-17525][PYTHON] Remove SparkContext.clearFiles() from the PySpa…
sjakthol Sep 14, 2016
c6ea748
[SPARK-17480][SQL] Improve performance by removing or caching List.le…
seyfe Sep 14, 2016
5493107
[SPARK-17445][DOCS] Reference an ASF page as the main place to find t…
srowen Sep 14, 2016
6fe5972
[SPARK-17514] df.take(1) and df.limit(1).collect() should perform the…
JoshRosen Sep 14, 2016
fab77da
[SPARK-17511] Yarn Dynamic Allocation: Avoid marking released contain…
kishorvpatil Sep 14, 2016
fffcec9
[SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's v…
zsxwing Sep 14, 2016
bb2bdb4
[SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apa…
Sep 14, 2016
5c2bc83
[SPARK-17521] Error when I use sparkContext.makeRDD(Seq())
codlife Sep 15, 2016
a09c258
[SPARK-17317][SPARKR] Add SparkR vignette to branch 2.0
junyangq Sep 15, 2016
e77a437
[SPARK-17547] Ensure temp shuffle data file is cleaned up after error
JoshRosen Sep 15, 2016
62ab536
[SPARK-17114][SQL] Fix aggregates grouped by literals with empty input
hvanhovell Sep 15, 2016
abb89c4
[SPARK-17483] Refactoring in BlockManager status reporting and block …
JoshRosen Sep 12, 2016
0169c2e
[SPARK-17364][SQL] Antlr lexer wrongly treats full qualified identifi…
clockfly Sep 15, 2016
9c23f44
[SPARK-17484] Prevent invalid block locations from being reported aft…
JoshRosen Sep 15, 2016
5ad4395
[SPARK-17558] Bump Hadoop 2.7 version from 2.7.2 to 2.7.3
rxin Sep 16, 2016
3fce125
[SPARK-17549][SQL] Only collect table size stat in driver for cached …
Sep 16, 2016
9ff158b
Correct fetchsize property name in docs
darabos Sep 17, 2016
3ca0dc0
[SPARK-17567][DOCS] Use valid url to Spark RDD paper
keypointt Sep 17, 2016
c9bd67e
[SPARK-17561][DOCS] DataFrameWriter documentation formatting problems
srowen Sep 16, 2016
eb2675d
[SPARK-17548][MLLIB] Word2VecModel.findSynonyms no longer spuriously …
willb Sep 17, 2016
ec2b736
[SPARK-17575][DOCS] Remove extra table tags in configuration document
phalodi Sep 17, 2016
a3bba37
[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.leng…
HyukjinKwon Sep 17, 2016
bec0770
[SPARK-17491] Close serialization stream to fix wrong answer bug in p…
JoshRosen Sep 17, 2016
0cfc046
Revert "[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls L…
tdas Sep 17, 2016
5fd354b
[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.leng…
HyukjinKwon Sep 17, 2016
cf728b0
[SPARK-17541][SQL] fix some DDL bugs about table management when same…
cloud-fan Sep 18, 2016
5619f09
[SPARK-17546][DEPLOY] start-* scripts should use hostname -f
srowen Sep 18, 2016
6c67d86
[SPARK-17586][BUILD] Do not call static member via instance reference
HyukjinKwon Sep 18, 2016
151f808
[SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null value…
lw-lin Sep 18, 2016
27ce39c
[SPARK-17571][SQL] AssertOnQuery.condition should always return Boole…
petermaxlee Sep 18, 2016
ac06039
[SPARK-17297][DOCS] Clarify window/slide duration as absolute time, n…
srowen Sep 19, 2016
c4660d6
[SPARK-17589][TEST][2.0] Fix test case `create external table` in Met…
gatorsmile Sep 19, 2016
f56035b
[SPARK-17473][SQL] fixing docker integration tests error due to diffe…
sureshthalamati Sep 19, 2016
d6191a0
[SPARK-17438][WEBUI] Show Application.executorLimit in the applicatio…
zsxwing Sep 19, 2016
fef3ec1
[SPARK-16439] [SQL] bring back the separator in SQL UI
Sep 19, 2016
c02bc92
[SPARK-17100] [SQL] fix Python udf in filter on top of outer join
Sep 19, 2016
7026eb8
[SPARK-17160] Properly escape field names in code-generated error mes…
JoshRosen Sep 20, 2016
5456a1b
[SPARK-17513][SQL] Make StreamExecution garbage-collect its metadata
petermaxlee Sep 20, 2016
643f161
Revert "[SPARK-17513][SQL] Make StreamExecution garbage-collect its m…
cloud-fan Sep 20, 2016
e76f4f4
[SPARK-17051][SQL] we should use hadoopConf in InsertIntoHiveTable
cloud-fan Sep 20, 2016
2bd37ce
[SPARK-17549][SQL] Revert "[] Only collect table size stat in driver …
yhuai Sep 20, 2016
8d8e233
[SPARK-15698][SQL][STREAMING] Add the ability to remove the old Metad…
jerryshao Sep 20, 2016
726f057
[SPARK-17513][SQL] Make StreamExecution garbage-collect its metadata
petermaxlee Sep 21, 2016
65295ba
[SPARK-17617][SQL] Remainder(%) expression.eval returns incorrect res…
clockfly Sep 21, 2016
45bccdd
[BACKPORT 2.0][MINOR][BUILD] Fix CheckStyle Error
weiqingy Sep 21, 2016
cd0bd89
[SPARK-17418] Prevent kinesis-asl-assembly artifacts from being publi…
JoshRosen Sep 21, 2016
59e6ab1
[SPARK-17512][CORE] Avoid formatting to python path for yarn and meso…
jerryshao Sep 21, 2016
966abd6
[SPARK-17627] Mark Streaming Providers Experimental
marmbrus Sep 22, 2016
ec377e7
[SPARK-17494][SQL] changePrecision() on compact decimal should respec…
Sep 22, 2016
053b20a
Bump doc version for release 2.0.1.
rxin Sep 22, 2016
00f2e28
Preparing Spark release v2.0.1-rc1
pwendell Sep 22, 2016
e8b26be
Preparing development version 2.0.2-SNAPSHOT
pwendell Sep 22, 2016
b25a8e6
[SPARK-17421][DOCS] Documenting the current treatment of MAVEN_OPTS.
frreiss Sep 22, 2016
f14f47f
Skip building R vignettes if Spark is not built
shivaram Sep 22, 2016
243bdb1
[SPARK-17613] S3A base paths with no '/' at the end return empty Data…
brkyvz Sep 22, 2016
47fc0b9
[SPARK-17638][STREAMING] Stop JVM StreamingContext when the Python pr…
zsxwing Sep 22, 2016
0a593db
[SPARK-17616][SQL] Support a single distinct aggregate combined with …
hvanhovell Sep 22, 2016
c2cb841
[SPARK-17599][SPARK-17569] Backport and to Spark 2.0 branch
brkyvz Sep 23, 2016
04141ad
Preparing Spark release v2.0.1-rc2
pwendell Sep 23, 2016
c393d86
Preparing development version 2.0.2-SNAPSHOT
pwendell Sep 23, 2016
22216d6
[SPARK-17502][17609][SQL][BACKPORT][2.0] Fix Multiple Bugs in DDL Sta…
gatorsmile Sep 23, 2016
54d4eee
[SPARK-16240][ML] ML persistence backward compatibility for LDA - 2.0…
GayathriMurali Sep 23, 2016
d3f90e7
[SPARK-17640][SQL] Avoid using -1 as the default batchId for FileStre…
zsxwing Sep 23, 2016
1a8ea00
[SPARK-17210][SPARKR] sparkr.zip is not distributed to executors when…
zjffdu Sep 23, 2016
452e468
[SPARK-17577][CORE][2.0 BACKPORT] Update SparkContext.addFile to make…
yanboliang Sep 23, 2016
b111a81
[SPARK-17651][SPARKR] Set R package version number along with mvn
shivaram Sep 23, 2016
9d28cc1
Preparing Spark release v2.0.1-rc3
pwendell Sep 23, 2016
5bc5b49
Preparing development version 2.0.2-SNAPSHOT
pwendell Sep 23, 2016
9e91a10
[SPARK-15703][SCHEDULER][CORE][WEBUI] Make ListenerBus event queue si…
dhruve Sep 23, 2016
ed54576
[SPARK-10835][ML] Word2Vec should accept non-null string array, in ad…
srowen Sep 24, 2016
88ba2e1
[SPARK-17650] malformed url's throw exceptions before bricking Executors
brkyvz Sep 26, 2016
cf53241
[SPARK-17649][CORE] Log how many Spark events got dropped in LiveList…
zsxwing Sep 26, 2016
8a58f2e
[SPARK-17652] Fix confusing exception message while reserving capacity
sameeragarwal Sep 26, 2016
f459490
[Docs] Update spark-standalone.md to fix link
ammills01 Sep 26, 2016
98bbc44
[SPARK-17618] Guard against invalid comparisons between UnsafeRow and…
JoshRosen Sep 27, 2016
2cd327e
[SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in Mem…
viirya Sep 27, 2016
1b02f88
[SPARK-17666] Ensure that RecordReaders are closed by data source fil…
JoshRosen Sep 28, 2016
4d73d5c
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
HyukjinKwon Sep 28, 2016
4c694e4
[SPARK-17644][CORE] Do not add failedStages when abortStage for fetch…
scwf Sep 28, 2016
d358298
[SPARK-17673][SQL] Incorrect exchange reuse with RowDataSourceScan (b…
ericl Sep 28, 2016
0a69477
[SPARK-17641][SQL] Collect_list/Collect_set should not collect null v…
hvanhovell Sep 28, 2016
933d2c1
Preparing Spark release v2.0.1-rc4
pwendell Sep 28, 2016
191341d
Merge tag 'v2.0.1' into snappy/v2.0.1
Oct 18, 2016
c0b73ca
MemoryStore changes for our changes
Oct 19, 2016
c71a5eb
Merge branch 'snappy/branch-2.0' into snappy/v2.0.1
Oct 19, 2016
ad90a7b
Added SnappyLicense
Oct 21, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-16947][SQL] Support type coercion and foldable expression for …
…inline tables

This patch improves inline table support with the following:

1. Support type coercion.
2. Support using foldable expressions. Previously only literals were supported.
3. Improve error message handling.
4. Improve test coverage.

Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql.

Author: petermaxlee <[email protected]>

Closes apache#14676 from petermaxlee/SPARK-16947.

(cherry picked from commit f5472dd)
Signed-off-by: Reynold Xin <[email protected]>
  • Loading branch information
petermaxlee authored and rxin committed Aug 19, 2016
commit c180d637a3caca0d4e46f4980c10d1005eb453bc
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ class Analyzer(
GlobalAggregates ::
ResolveAggregateFunctions ::
TimeWindowing ::
ResolveInlineTables ::
TypeCoercion.typeCoercionRules ++
extendedResolutionRules : _*),
Batch("Nondeterministic", Once,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.analysis

import scala.util.control.NonFatal

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Cast
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.types.{StructField, StructType}

/**
* An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]].
*/
object ResolveInlineTables extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case table: UnresolvedInlineTable if table.expressionsResolved =>
validateInputDimension(table)
validateInputEvaluable(table)
convert(table)
}

/**
* Validates the input data dimension:
* 1. All rows have the same cardinality.
* 2. The number of column aliases defined is consistent with the number of columns in data.
*
* This is package visible for unit testing.
*/
private[analysis] def validateInputDimension(table: UnresolvedInlineTable): Unit = {
if (table.rows.nonEmpty) {
val numCols = table.names.size
table.rows.zipWithIndex.foreach { case (row, ri) =>
if (row.size != numCols) {
table.failAnalysis(s"expected $numCols columns but found ${row.size} columns in row $ri")
}
}
}
}

/**
* Validates that all inline table data are valid expressions that can be evaluated
* (in this they must be foldable).
*
* This is package visible for unit testing.
*/
private[analysis] def validateInputEvaluable(table: UnresolvedInlineTable): Unit = {
table.rows.foreach { row =>
row.foreach { e =>
// Note that nondeterministic expressions are not supported since they are not foldable.
if (!e.resolved || !e.foldable) {
e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline table definition")
}
}
}
}

/**
* Convert a valid (with right shape and foldable inputs) [[UnresolvedInlineTable]]
* into a [[LocalRelation]].
*
* This function attempts to coerce inputs into consistent types.
*
* This is package visible for unit testing.
*/
private[analysis] def convert(table: UnresolvedInlineTable): LocalRelation = {
// For each column, traverse all the values and find a common data type and nullability.
val fields = table.rows.transpose.zip(table.names).map { case (column, name) =>
val inputTypes = column.map(_.dataType)
val tpe = TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
table.failAnalysis(s"incompatible types found in column $name for inline table")
}
StructField(name, tpe, nullable = column.exists(_.nullable))
}
val attributes = StructType(fields).toAttributes
assert(fields.size == table.names.size)

val newRows: Seq[InternalRow] = table.rows.map { row =>
InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) =>
val targetType = fields(ci).dataType
try {
if (e.dataType.sameType(targetType)) {
e.eval()
} else {
Cast(e, targetType).eval()
}
} catch {
case NonFatal(ex) =>
table.failAnalysis(s"failed to evaluate expression ${e.sql}: ${ex.getMessage}")
}
})
}

LocalRelation(attributes, newRows)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ object TypeCoercion {
* [[findTightestCommonType]], but can handle decimal types. If the wider decimal type exceeds
* system limitation, this rule will truncate the decimal type before return it.
*/
private def findWiderTypeWithoutStringPromotion(types: Seq[DataType]): Option[DataType] = {
def findWiderTypeWithoutStringPromotion(types: Seq[DataType]): Option[DataType] = {
types.foldLeft[Option[DataType]](Some(NullType))((r, c) => r match {
case Some(d) => findTightestCommonTypeOfTwo(d, c).orElse((d, c) match {
case (t1: DecimalType, t2: DecimalType) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,10 +50,30 @@ case class UnresolvedRelation(
}

/**
* Holds a table-valued function call that has yet to be resolved.
* An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into
* a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]].
*
* @param names list of column names
* @param rows expressions for the data
*/
case class UnresolvedInlineTable(
names: Seq[String],
rows: Seq[Seq[Expression]])
extends LeafNode {

lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved))
override lazy val resolved = false
override def output: Seq[Attribute] = Nil
}

/**
* A table-valued function, e.g.
* {{{
* select * from range(10);
* }}}
*/
case class UnresolvedTableValuedFunction(
functionName: String, functionArgs: Seq[Expression]) extends LeafNode {
case class UnresolvedTableValuedFunction(functionName: String, functionArgs: Seq[Expression])
extends LeafNode {

override def output: Seq[Attribute] = Nil

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -665,39 +665,24 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
*/
override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
// Get the backing expressions.
val expressions = ctx.expression.asScala.map { eCtx =>
val e = expression(eCtx)
validate(e.foldable, "All expressions in an inline table must be constants.", eCtx)
e
}

// Validate and evaluate the rows.
val (structType, structConstructor) = expressions.head.dataType match {
case st: StructType =>
(st, (e: Expression) => e)
case dt =>
val st = CreateStruct(Seq(expressions.head)).dataType
(st, (e: Expression) => CreateStruct(Seq(e)))
}
val rows = expressions.map {
case expression =>
val safe = Cast(structConstructor(expression), structType)
safe.eval().asInstanceOf[InternalRow]
val rows = ctx.expression.asScala.map { e =>
expression(e) match {
// inline table comes in two styles:
// style 1: values (1), (2), (3) -- multiple columns are supported
// style 2: values 1, 2, 3 -- only a single column is supported here
case CreateStruct(children) => children // style 1
case child => Seq(child) // style 2
}
}

// Construct attributes.
val baseAttributes = structType.toAttributes.map(_.withNullability(true))
val attributes = if (ctx.identifierList != null) {
val aliases = visitIdentifierList(ctx.identifierList)
validate(aliases.size == baseAttributes.size,
"Number of aliases must match the number of fields in an inline table.", ctx)
baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
val aliases = if (ctx.identifierList != null) {
visitIdentifierList(ctx.identifierList)
} else {
baseAttributes
Seq.tabulate(rows.head.size)(i => s"col${i + 1}")
}

// Create plan and add an alias if a name has been defined.
LocalRelation(attributes, rows).optionalMap(ctx.identifier)(aliasPlan)
val table = UnresolvedInlineTable(aliases, rows)
table.optionalMap(ctx.identifier)(aliasPlan)
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.analysis

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.{Literal, Rand}
import org.apache.spark.sql.catalyst.expressions.aggregate.Count
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.types.{LongType, NullType}

/**
* Unit tests for [[ResolveInlineTables]]. Note that there are also test cases defined in
* end-to-end tests (in sql/core module) for verifying the correct error messages are shown
* in negative cases.
*/
class ResolveInlineTablesSuite extends PlanTest with BeforeAndAfter {

private def lit(v: Any): Literal = Literal(v)

test("validate inputs are foldable") {
ResolveInlineTables.validateInputEvaluable(
UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)))))

// nondeterministic (rand) should not work
intercept[AnalysisException] {
ResolveInlineTables.validateInputEvaluable(
UnresolvedInlineTable(Seq("c1"), Seq(Seq(Rand(1)))))
}

// aggregate should not work
intercept[AnalysisException] {
ResolveInlineTables.validateInputEvaluable(
UnresolvedInlineTable(Seq("c1"), Seq(Seq(Count(lit(1))))))
}

// unresolved attribute should not work
intercept[AnalysisException] {
ResolveInlineTables.validateInputEvaluable(
UnresolvedInlineTable(Seq("c1"), Seq(Seq(UnresolvedAttribute("A")))))
}
}

test("validate input dimensions") {
ResolveInlineTables.validateInputDimension(
UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2)))))

// num alias != data dimension
intercept[AnalysisException] {
ResolveInlineTables.validateInputDimension(
UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)), Seq(lit(2)))))
}

// num alias == data dimension, but data themselves are inconsistent
intercept[AnalysisException] {
ResolveInlineTables.validateInputDimension(
UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(21), lit(22)))))
}
}

test("do not fire the rule if not all expressions are resolved") {
val table = UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(UnresolvedAttribute("A"))))
assert(ResolveInlineTables(table) == table)
}

test("convert") {
val table = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L))))
val converted = ResolveInlineTables.convert(table)

assert(converted.output.map(_.dataType) == Seq(LongType))
assert(converted.data.size == 2)
assert(converted.data(0).getLong(0) == 1L)
assert(converted.data(1).getLong(0) == 2L)
}

test("nullability inference in convert") {
val table1 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(lit(2L))))
val converted1 = ResolveInlineTables.convert(table1)
assert(!converted1.schema.fields(0).nullable)

val table2 = UnresolvedInlineTable(Seq("c1"), Seq(Seq(lit(1)), Seq(Literal(null, NullType))))
val converted2 = ResolveInlineTables.convert(table2)
assert(converted2.schema.fields(0).nullable)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,8 @@

package org.apache.spark.sql.catalyst.parser

import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.{UnresolvedGenerator, UnresolvedTableValuedFunction}
import org.apache.spark.sql.catalyst.analysis.{UnresolvedGenerator, UnresolvedInlineTable, UnresolvedTableValuedFunction}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
Expand Down Expand Up @@ -430,19 +429,14 @@ class PlanParserSuite extends PlanTest {
}

test("inline table") {
assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
Seq('col1.int),
Seq(1, 2, 3, 4).map(x => Row(x))))
assertEqual("values 1, 2, 3, 4",
UnresolvedInlineTable(Seq("col1"), Seq(1, 2, 3, 4).map(x => Seq(Literal(x)))))

assertEqual(
"values (1, 'a'), (2, 'b'), (3, 'c') as tbl(a, b)",
LocalRelation.fromExternalRows(
Seq('a.int, 'b.string),
Seq((1, "a"), (2, "b"), (3, "c")).map(x => Row(x._1, x._2))).as("tbl"))
intercept("values (a, 'a'), (b, 'b')",
"All expressions in an inline table must be constants.")
intercept("values (1, 'a'), (2, 'b') as tbl(a, b, c)",
"Number of aliases must match the number of fields in an inline table.")
intercept[ArrayIndexOutOfBoundsException](parsePlan("values (1, 'a'), (2, 'b', 5Y)"))
"values (1, 'a'), (2, 'b') as tbl(a, b)",
UnresolvedInlineTable(
Seq("a", "b"),
Seq(Literal(1), Literal("a")) :: Seq(Literal(2), Literal("b")) :: Nil).as("tbl"))
}

test("simple select query with !> and !<") {
Expand Down
Loading