Skip to content
Merged
Changes from 1 commit
Commits
Show all changes
202 commits
Select commit Hold shift + click to select a range
17881a4
[SPARK-19784][SPARK-25403][SQL] Refresh the table even table stats is…
wangyum Jan 7, 2020
da07615
[SPARK-30433][SQL] Make conflict attributes resolution more scalable …
Ngone51 Jan 7, 2020
93d3ab8
[SPARK-30338][SQL] Avoid unnecessary InternalRow copies in ParquetRow…
JoshRosen Jan 7, 2020
7a1a5db
[SPARK-30414][SQL] ParquetRowConverter optimizations: arrays, maps, p…
JoshRosen Jan 7, 2020
866b7df
[SPARK-30335][SQL][DOCS] Add a note first, last, collect_list and col…
HyukjinKwon Jan 7, 2020
314e70f
[SPARK-30214][SQL] V2 commands resolves namespaces with new resolutio…
imback82 Jan 7, 2020
8c121b0
[SPARK-30431][SQL] Update SqlBase.g4 to create commentSpec pattern li…
yaooqinn Jan 7, 2020
f399d65
[SPARK-30173] Tweak stale PR message
nchammas Jan 7, 2020
9479887
[SPARK-30039][SQL] CREATE FUNCTION should do multi-catalog resolution
planga82 Jan 7, 2020
2be5286
[SPARK-30382][SQL] Remove Hive LogUtils usage to prevent ClassNotFoun…
ajithme Jan 7, 2020
ed8a260
[SPARK-30450][INFRA] Exclude .git folder for python linter
ericfchang Jan 7, 2020
ed73ed8
[SPARK-28825][SQL][DOC] Documentation for Explain Command
PavithraRamachandran Jan 8, 2020
d7c7e37
[SPARK-30381][ML] Refactor GBT to reuse treePoints for all trees
zhengruifeng Jan 8, 2020
9535776
[SPARK-30302][SQL] Complete info for show create table for views
wzhfy Jan 8, 2020
390e6bd
[SPARK-30453][BUILD][R] Update AppVeyor R version to 3.6.2
HyukjinKwon Jan 8, 2020
1160457
[SPARK-30429][SQL] Optimize catalogString and usage in ValidateExtern…
viirya Jan 8, 2020
0d589f4
[SPARK-30267][SQL][FOLLOWUP] Use while loop in Avro Array Deserializer
gengliangwang Jan 8, 2020
b2ed6d0
[SPARK-30214][SQL][FOLLOWUP] Remove statement logical plans for names…
imback82 Jan 8, 2020
047bff0
[SPARK-30215][SQL] Remove PrunedInMemoryFileIndex and merge its funct…
fuwhu Jan 8, 2020
fa36966
[SPARK-30410][SQL] Calculating size of table with large number of par…
wzhfy Jan 8, 2020
b3c2d73
[MINOR][CORE] Process bar should print new line to avoid polluting logs
Ngone51 Jan 8, 2020
a93b996
[MINOR][ML][INT] Array.fill(0) -> Array.ofDim; Array.empty -> Array.e…
zhengruifeng Jan 8, 2020
0a72dba
[SPARK-30445][CORE] Accelerator aware scheduling handle setting confi…
tgravescs Jan 8, 2020
bd7510b
[SPARK-30281][SS] Consider partitioned/recursive option while verifyi…
HeartSaVioR Jan 8, 2020
c49abf8
[SPARK-30417][CORE] Task speculation numTaskThreshold should be great…
yuchenhuo Jan 8, 2020
af2d3d0
[SPARK-30315][SQL] Add adaptive execution context
maryannxue Jan 9, 2020
18daa37
[SPARK-30440][CORE][TESTS] Avoid race condition in TaskSetManagerSuit…
ajithme Jan 9, 2020
ee8d661
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into '…
HyukjinKwon Jan 9, 2020
92a0877
[SPARK-30464][PYTHON][DOCS] Explicitly note that we don't add "pandas…
HyukjinKwon Jan 9, 2020
c373123
[SPARK-30183][SQL] Disallow to specify reserved properties in CREATE/…
yaooqinn Jan 9, 2020
5c71304
[SPARK-30450][INFRA][FOLLOWUP] Fix git folder regex for windows file …
ericfchang Jan 9, 2020
dcdc9a8
[SPARK-28198][PYTHON][FOLLOW-UP] Run the tests of MAP ITER UDF in Jen…
HyukjinKwon Jan 9, 2020
94fc0e3
[SPARK-30428][SQL] File source V2: support partition pruning
gengliangwang Jan 9, 2020
c88124a
[SPARK-30452][ML][PYSPARK] Add predict and numFeatures in Python Isot…
huaxingao Jan 9, 2020
f8d5957
[SPARK-29219][SQL] Introduce SupportsCatalogOptions for TableProvider
brkyvz Jan 9, 2020
c0e9f9f
[SPARK-30459][SQL] Fix ignoreMissingFiles/ignoreCorruptFiles in data …
Ngone51 Jan 9, 2020
4d23938
[MINOR][SQL][TEST-HIVE1.2] Fix scalastyle error due to length line in…
shaneknapp Jan 9, 2020
1ffa627
[SPARK-30416][SQL] Log a warning for deprecated SQL config in `set()`…
MaxGekk Jan 10, 2020
0ec0355
[SPARK-30439][SQL] Support non-nullable column in CREATE TABLE, ADD C…
cloud-fan Jan 10, 2020
afd70a0
[SPARK-30480][PYSPARK][TESTS] Fix 'test_memory_limit' on pyspark test
HeartSaVioR Jan 10, 2020
bcf07cb
[SPARK-30018][SQL] Support ALTER DATABASE SET OWNER syntax
yaooqinn Jan 10, 2020
418f7dc
[SPARK-30447][SQL] Constant propagation nullability issue
peter-toth Jan 10, 2020
d0983af
Revert "[SPARK-30480][PYSPARK][TESTS] Fix 'test_memory_limit' on pysp…
HyukjinKwon Jan 10, 2020
2a629e5
[SPARK-30234][SQL] ADD FILE cannot add directories from sql CLI
iRakson Jan 10, 2020
d6532c7
[SPARK-30448][CORE] accelerator aware scheduling enforce cores as lim…
tgravescs Jan 10, 2020
b942832
[SPARK-30343][SQL] Skip unnecessary checks in RewriteDistinctAggregates
maropu Jan 10, 2020
2bd8731
[SPARK-30468][SQL] Use multiple lines to display data columns for sho…
wzhfy Jan 10, 2020
7fb17f5
[SPARK-29779][CORE] Compact old event log files and cleanup
HeartSaVioR Jan 10, 2020
b5bc3e1
[SPARK-30312][SQL] Preserve path permission and acl when truncate table
viirya Jan 10, 2020
f372d1c
[SPARK-29748][PYTHON][SQL] Remove Row field sorting in PySpark for ve…
BryanCutler Jan 10, 2020
582509b
[SPARK-30489][BUILD] Make build delete pyspark.zip file properly
jeff303 Jan 11, 2020
b044071
[SPARK-30312][SQL][FOLLOWUP] Use inequality check instead to be robust
viirya Jan 11, 2020
26ad8f8
[SPARK-30478][CORE][DOCS] Fix Memory Package documentation
sddyljsx Jan 12, 2020
65b603d
[SPARK-30458][WEBUI] Fix Wrong Executor Computing Time in Time Line o…
sddyljsx Jan 12, 2020
823e3d3
[SPARK-30353][SQL] Add IsNotNull check in SimplifyBinaryComparison op…
ulysses-you Jan 12, 2020
1f50a58
[SPARK-27296][SQL] Allows Aggregator to be registered as a UDF
erikerlandson Jan 12, 2020
f5118f8
[SPARK-30409][SPARK-29173][SQL][TESTS] Use `NoOp` datasource in SQL b…
MaxGekk Jan 12, 2020
361583d
[SPARK-30409][TEST][FOLLOWUP][HOTFIX] Remove dangling JSONBenchmark-j…
dongjoon-hyun Jan 12, 2020
150d493
[SPARK-28752][BUILD][DOCS][FOLLOW-UP] Render examples imported from J…
HyukjinKwon Jan 13, 2020
308ae28
[SPARK-30457][ML] Use PeriodicRDDCheckpointer instead of NodeIdCache
zhengruifeng Jan 13, 2020
8ce7962
[SPARK-30245][SQL] Add cache for Like and RLike when pattern is not s…
ulysses-you Jan 13, 2020
28fc043
[SPARK-28152][SQL][FOLLOWUP] Add a legacy conf for old MsSqlServerDia…
dongjoon-hyun Jan 13, 2020
eefcc7d
[SPARK-21869][SS][DOCS][FOLLOWUP] Document Kafka producer pool config…
HeartSaVioR Jan 13, 2020
0823aec
[SPARK-30480][PYTHON][TESTS] Increases the memory limit being tested …
HyukjinKwon Jan 13, 2020
6502c66
[SPARK-30493][PYTHON][ML] Remove OneVsRestModel setClassifier, setLab…
zero323 Jan 13, 2020
d6e28f2
[SPARK-30377][ML] Make Regressors extend abstract class Regressor
huaxingao Jan 13, 2020
f77dcfc
[SPARK-30351][ML][PYSPARK] BisectingKMeans support instance weighting
huaxingao Jan 13, 2020
b389b8c
[SPARK-30188][SQL] Resolve the failed unit tests when enable AQE
Jan 13, 2020
81e1a21
[SPARK-30234][SQL][DOCS][FOLOWUP] Update Documentation for ADD FILE a…
iRakson Jan 14, 2020
6646b3e
Revert "[SPARK-28670][SQL] create function should thrown Exception if…
HyukjinKwon Jan 14, 2020
1846b02
[SPARK-30500][SPARK-30501][SQL] Remove SQL configs deprecated in Spar…
MaxGekk Jan 14, 2020
88fc8db
[SPARK-30482][SQL][CORE][TESTS] Add sub-class of `AppenderSkeleton` r…
MaxGekk Jan 14, 2020
e0efd21
[SPARK-30292][SQL] Throw Exception when invalid string is cast to num…
iRakson Jan 14, 2020
4462756
[SPARK-30325][CORE] markPartitionCompleted cause task status inconsis…
Jan 14, 2020
2688fae
[SPARK-30498][ML][PYSPARK] Fix some ml parity issues between python a…
huaxingao Jan 14, 2020
a2aa966
[SPARK-29544][SQL] optimize skewed partition based on data size
Jan 14, 2020
176b696
[SPARK-30423][SQL] Deprecate UserDefinedAggregateFunction
erikerlandson Jan 14, 2020
9320011
[SPARK-9478][ML][PYSPARK] Add sample weights to Random Forest
zhengruifeng Jan 14, 2020
0c6bd3b
[SPARK-27142][SQL] Provide REST API for SQL information
ajithme Jan 14, 2020
db7262a
[SPARK-30509][SQL] Fix deprecation log warning in Avro schema inferring
MaxGekk Jan 14, 2020
0f3d744
[MINOR][TESTS] Remove unsupported `header` option in AvroSuite
MaxGekk Jan 14, 2020
3668291
[SPARK-30452][ML][PYSPARK][FOLLOWUP] Change IsotonicRegressionModel.n…
zero323 Jan 15, 2020
51d2917
[SPARK-30505][DOCS] Deprecate Avro option `ignoreExtension` in sql-da…
MaxGekk Jan 15, 2020
240840f
[SPARK-30515][SQL] Refactor SimplifyBinaryComparison to reduce the ti…
gengliangwang Jan 15, 2020
5f6cd61
[SPARK-29708][SQL] Correct aggregated values when grouping sets are d…
maropu Jan 15, 2020
525c569
[SPARK-30504][PYTHON][ML] Set weightCol in OneVsRest(Model) _to_java …
zero323 Jan 15, 2020
990a2be
[SPARK-30378][ML][PYSPARK][FOLLOWUP] Remove Param fields provided by …
zero323 Jan 15, 2020
e751bc6
[SPARK-30479][SQL] Apply compaction of event log to SQL events
HeartSaVioR Jan 15, 2020
6c178a5
[SPARK-30495][SS] Consider spark.security.credentials.kafka.enabled a…
gaborgsomogyi Jan 15, 2020
d42cf45
[SPARK-30246][CORE] OneForOneStreamManager might leak memory in conne…
hensg Jan 15, 2020
8a926e4
[SPARK-26736][SQL] Partition pruning through nondeterministic express…
maropu Jan 15, 2020
883ae33
[SPARK-30497][SQL] migrate DESCRIBE TABLE to the new framework
cloud-fan Jan 16, 2020
a3a42b3
[SPARK-27986][SQL][FOLLOWUP] Respect filter in sql/toString of Aggreg…
maropu Jan 16, 2020
5a55a5a
[SPARK-30518][SQL] Precision and scale should be same for values betw…
Ngone51 Jan 16, 2020
aec55cd
[SPARK-30502][ML][CORE] PeriodicRDDCheckpointer support storageLevel
zhengruifeng Jan 16, 2020
0a95eb0
[SPARK-30434][FOLLOW-UP][PYTHON][SQL] Make the parameter list consist…
HyukjinKwon Jan 16, 2020
be4d825
[SPARK-30312][SQL][FOLLOWUP] Rename conf by adding `.enabled`
viirya Jan 16, 2020
4e50f02
[SPARK-30323][SQL] Support filters pushdown in CSV datasource
MaxGekk Jan 16, 2020
f888741
[SPARK-30491][INFRA] Enable dependency audit files to tell dependency…
mengCareers Jan 16, 2020
018bdcc
[SPARK-30521][SQL][TESTS] Eliminate deprecation warnings for Expressi…
MaxGekk Jan 16, 2020
1ef1d6c
[SPARK-29565][FOLLOWUP] add setInputCol/setOutputCol in OHEModel
huaxingao Jan 16, 2020
82f25f5
[SPARK-30507][SQL] TableCalalog reserved properties shoudn't be chang…
yaooqinn Jan 16, 2020
6e5b4bf
[SPARK-30524][SQL] Disable OptimizeSkewedJoin rule when introducing a…
Jan 16, 2020
dca8380
[SPARK-29950][K8S] Blacklist deleted executors in K8S with dynamic al…
Jan 16, 2020
3848999
[SPARK-30534][INFRA] Use mvn in `dev/scalastyle`
dongjoon-hyun Jan 17, 2020
92dd7c9
[MINOR][ML] Change DecisionTreeClassifier to FMClassifier in OneVsRes…
huaxingao Jan 17, 2020
1a9de8c
[SPARK-30499][SQL] Remove SQL config spark.sql.execution.pandas.respe…
MaxGekk Jan 17, 2020
4398dfa
[SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution f…
HyukjinKwon Jan 17, 2020
0bd7a3d
[SPARK-29572][SQL] add v1 read fallback API in DS v2
cloud-fan Jan 17, 2020
1881caa
[SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution f…
HyukjinKwon Jan 17, 2020
64fe192
[SPARK-30282][SQL] Migrate SHOW TBLPROPERTIES to new framework
imback82 Jan 17, 2020
6dbfa2b
[SPARK-29306][CORE] Stage Level Sched: Executors need to track what R…
tgravescs Jan 17, 2020
f5f05d5
[SPARK-30310][CORE] Resolve missing match case in SparkUncaughtExcept…
tinhto-000 Jan 17, 2020
fd308ad
[SPARK-30041][SQL][WEBUI] Add Codegen Stage Id to Stage DAG visualiza…
LucaCanali Jan 17, 2020
830e635
[SPARK-27868][CORE][FOLLOWUP] Recover the default value to -1 again
xCASx Jan 17, 2020
abf759a
[SPARK-29876][SS] Delete/archive file source completed files in separ…
gaborgsomogyi Jan 17, 2020
fdbded3
[SPARK-30312][DOCS][FOLLOWUP] Add a migration guide
dongjoon-hyun Jan 17, 2020
96a3445
[SPARK-25993][SQL][TESTS] Add test cases for CREATE EXTERNAL TABLE wi…
kevinyu98 Jan 18, 2020
505693c
[SPARK-28152][DOCS][FOLLOWUP] Add a migration guide for MsSQLServer J…
dongjoon-hyun Jan 18, 2020
3228732
[SPARK-30533][ML][PYSPARK] Add classes to represent Java Regressors a…
zero323 Jan 18, 2020
a3357df
[SPARK-30544][BUILD] Upgrade the version of Genjavadoc to 0.15
sarutak Jan 18, 2020
a6bdea3
[SPARK-30539][PYTHON][SQL] Add DataFrame.tail in PySpark
HyukjinKwon Jan 18, 2020
ef1af43
[MINOR][DOCS] Remove note about -T for parallel build
srowen Jan 18, 2020
789a4ab
[MINOR][HIVE] Pick up HIVE-22708 HTTP transport fix
srowen Jan 18, 2020
0d99d7e
[SPARK-30524] [SQL] follow up SPARK-30524 to resolve comments
Jan 19, 2020
17857f9
[SPARK-30551][SQL] Disable comparison for interval type
yaooqinn Jan 19, 2020
d4c6ec6
[SPARK-30530][SQL] Fix filter pushdown for bad CSV records
MaxGekk Jan 19, 2020
f14061c
[SPARK-30371][K8S] Add spark.kubernetes.driver.master conf
wackxu Jan 19, 2020
19a1059
[SPARK-30282][DOCS][FOLLOWUP] Update SQL migration guide for SHOW TBL…
imback82 Jan 19, 2020
3858e94
[SPARK-30566][BUILD] Iterator doesn't refer outer identifier named "i…
sarutak Jan 20, 2020
c992716
[SPARK-30572][BUILD] Add a fallback Maven repository
dongjoon-hyun Jan 20, 2020
a2081ae
[SPARK-29290][CORE] Update to chill 0.9.5
srowen Jan 20, 2020
775fae4
[SPARK-30486][BUILD] Bump lz4-java version to 1.7.1
maropu Jan 20, 2020
d50f8df
[SPARK-30413][SQL] Avoid WrappedArray roundtrip in GenericArrayData c…
JoshRosen Jan 20, 2020
4806cc5
[SPARK-30547][SQL] Add unstable annotation to the CalendarInterval class
yaooqinn Jan 20, 2020
00039cc
[SPARK-30554][SQL] Return `Iterable` from `FailureSafeParser.rawParser`
MaxGekk Jan 20, 2020
ab04899
[SPARK-30558][SQL] Avoid rebuilding `AvroOptions` per each partition
MaxGekk Jan 20, 2020
b5cb9ab
[SPARK-30535][SQL] Migrate ALTER TABLE commands to the new framework
imback82 Jan 20, 2020
f5b345c
[SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in O…
Ngone51 Jan 20, 2020
fd69533
[SPARK-30482][CORE][SQL][TESTS][FOLLOW-UP] Output caller info in log …
MaxGekk Jan 21, 2020
14bc2a2
[SPARK-30530][SQL][FOLLOW-UP] Remove unnecessary codes and fix commen…
HyukjinKwon Jan 21, 2020
24efa43
[SPARK-30019][SQL] Add the owner property to v2 table
yaooqinn Jan 21, 2020
0388b7a
[SPARK-30568][SQL] Invalidate interval type as a field table schema
yaooqinn Jan 21, 2020
94284c8
[SPARK-30587][SQL][TESTS] Add test suites for CSV and JSON v1
MaxGekk Jan 21, 2020
2d59ca4
[SPARK-30475][SQL] File source V2: Push data filters for file listing
guykhazma Jan 21, 2020
78df532
[SPARK-30433][SQL][FOLLOW-UP] Optimize collect conflict plans
Ngone51 Jan 21, 2020
595cdb0
[SPARK-30571][CORE] fix splitting shuffle fetch requests
cloud-fan Jan 21, 2020
a94a4fc
[MINOR][DOCS] Fix Jenkins build image and link in README.md
HyukjinKwon Jan 21, 2020
e170422
Revert "[SPARK-30534][INFRA] Use mvn in `dev/scalastyle`"
HyukjinKwon Jan 21, 2020
730388b
[SPARK-30547][SQL][FOLLOWUP] Update since anotation for CalendarInter…
yaooqinn Jan 21, 2020
af70542
[SPARK-30593][SQL] Revert interval ISO/ANSI SQL Standard output since…
yaooqinn Jan 21, 2020
ff39c92
[SPARK-30252][SQL] Disallow negative scale of Decimal
Ngone51 Jan 21, 2020
cfb1706
[SPARK-15616][SQL] Add optimizer rule PruneHiveTablePartitions
fuwhu Jan 21, 2020
a131031
[SPARK-30599][CORE][TESTS] Increase the maximum number of log events …
MaxGekk Jan 21, 2020
3c4e619
[SPARK-30553][DOCS] fix structured-streaming java example error
Jan 22, 2020
ab0890b
[SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and renam…
HyukjinKwon Jan 22, 2020
f2d71f5
[SPARK-30591][SQL] Remove the nonstandard SET OWNER syntax for namesp…
yaooqinn Jan 22, 2020
b8cb52a
[SPARK-30555][SQL] MERGE INTO insert action should only access column…
cloud-fan Jan 22, 2020
1c46bd9
[SPARK-30503][ML] OnlineLDAOptimizer does not handle persistance corr…
zhengruifeng Jan 22, 2020
8097b7e
[SPARK-30573][DOC] Document WHERE Clause of SELECT statement in SQL R…
dilipbiswal Jan 22, 2020
a6030ef
[SPARK-30575][DOC] Document HAVING Clause of SELECT statement in SQL …
dilipbiswal Jan 22, 2020
8f7f4d5
[SPARK-30583][DOC] Document LIMIT Clause of SELECT statement in SQL R…
dilipbiswal Jan 22, 2020
8e280ce
[SPARK-30592][SQL] Interval support for csv and json funtions
yaooqinn Jan 22, 2020
6dfaa07
[SPARK-30549][SQL] Fix the subquery shown issue in UI When enable AQE
Jan 22, 2020
84f1154
[SPARK-30604][CORE] Fix a log message by including hostLocalBlockByte…
Udbhav30 Jan 22, 2020
4ca31b4
[SPARK-30606][SQL] Fix the `like` function with 2 parameters
MaxGekk Jan 22, 2020
2e74dba
[SPARK-30574][DOC] Document GROUP BY Clause of SELECT statement in SQ…
dilipbiswal Jan 23, 2020
38f4e59
[SPARK-28801][DOC] Document SELECT statement in SQL Reference (Main p…
dilipbiswal Jan 23, 2020
eccae13
[SPARK-30531][WEB UI] Do not render plan viz when it exists already
EnricoMi Jan 23, 2020
bbab2bb
[SPARK-30556][SQL] Copy sparkContext.localproperties to child thread …
ajithme Jan 23, 2020
d2bca8f
[SPARK-30609] Allow default merge command resolution to be bypassed b…
tdas Jan 23, 2020
db528e4
[SPARK-30535][SQL] Revert "[] Migrate ALTER TABLE commands to the new…
brkyvz Jan 23, 2020
cd9ccdc
[SPARK-30601][BUILD] Add a Google Maven Central as a primary repository
HyukjinKwon Jan 23, 2020
2330a56
[SPARK-30607][SQL][PYSPARK][SPARKR] Add overlay wrappers for SparkR a…
zero323 Jan 23, 2020
f35f352
[SPARK-30543][ML][PYSPARK][R] RandomForest add Param bootstrap to con…
zhengruifeng Jan 23, 2020
d0bf447
[SPARK-30575][DOCS][FOLLOWUP] Fix typos in documents
huaxingao Jan 23, 2020
ffd435b
[SPARK-27871][SQL][FOLLOW-UP] Remove the conf spark.sql.optimizer.rea…
gatorsmile Jan 23, 2020
dbed4c7
[SPARK-30605][SQL] move defaultNamespace from SupportsNamespace to Ca…
cloud-fan Jan 23, 2020
705fc6a
[SPARK-30188][SQL][TESTS][FOLLOW-UP] Remove `sorted` in asserts of co…
MaxGekk Jan 23, 2020
3d7359a
[SPARK-29175][SQL][FOLLOW-UP] Rename the config name to spark.sql.mav…
xuanyuanking Jan 23, 2020
3c8b360
[SPARK-30620][SQL] avoid unnecessary serialization in AggregateExpres…
cloud-fan Jan 23, 2020
afe70b3
[SPARK-28794][SQL][DOC] Documentation for Create table Command
PavithraRamachandran Jan 23, 2020
843224e
[SPARK-30570][BUILD] Update scalafmt plugin to 1.0.3 with onlyChanged…
koeninger Jan 23, 2020
976946a
[SPARK-29947][SQL][FOLLOWUP] Fix table lookup cache
cloud-fan Jan 23, 2020
3228d72
[SPARK-30603][SQL] Move RESERVED_PROPERTIES from SupportsNamespaces a…
yaooqinn Jan 23, 2020
4847f73
[SPARK-30298][SQL] Respect aliases in output partitioning of projects…
imback82 Jan 23, 2020
3f76bd4
[SPARK-27083][SQL][FOLLOW-UP] Rename spark.sql.subquery.reuse to spar…
gatorsmile Jan 23, 2020
ddf8315
[SPARK-28962][SQL][FOLLOW-UP] Add the parameter description for the S…
gatorsmile Jan 24, 2020
53fd83a
[MINOR][DOCS] Fix src/dest type documentation for `to_timestamp`
deepyaman Jan 24, 2020
ed44926
[SPARK-30627][SQL] Disable all the V2 file sources by default
gengliangwang Jan 24, 2020
d1a673a
[SPARK-29924][DOCS] Document Apache Arrow JDK11 requirement
dongjoon-hyun Jan 24, 2020
f86a1b9
[SPARK-30626][K8S] Add SPARK_APPLICATION_ID into driver pod env
Jeffwan Jan 24, 2020
2f8e4d0
[SPARK-30630][ML] Remove numTrees in GBT in 3.0.0
huaxingao Jan 24, 2020
a0e63b6
[SPARK-29721][SQL] Prune unnecessary nested fields from Generate with…
viirya Jan 25, 2020
8629597
[SPARK-30639][BUILD] Upgrade Jersey to 2.30
dongjoon-hyun Jan 25, 2020
d5b92b2
[SPARK-30579][DOC] Document ORDER BY Clause of SELECT statement in SQ…
dilipbiswal Jan 26, 2020
40b1f4d
[SPARK-30645][SPARKR][TESTS][WINDOWS] Move Unicode test data to exter…
zero323 Jan 26, 2020
c3f17fa
[SPARK-29777][FOLLOW-UP][SPARKR] Remove no longer valid test for recu…
zero323 Jan 26, 2020
d69ed9a
Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_ti…
gatorsmile Jan 26, 2020
48f6478
[SPARK-30644][SQL][TEST] Remove query index from the golden files of …
gatorsmile Jan 26, 2020
d0800fc
[SPARK-30314] Add identifier and catalog information to DataSourceV2R…
yuchenhuo Jan 26, 2020
43d9c7e
[SPARK-30640][PYTHON][SQL] Prevent unnecessary copies of data during …
BryanCutler Jan 26, 2020
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-30440][CORE][TESTS] Avoid race condition in TaskSetManagerSuit…
…e by not using resourceOffer

### What changes were proposed in this pull request?
There is a race condition in test case introduced in SPARK-30359 between reviveOffers in org.apache.spark.scheduler.TaskSchedulerImpl#submitTasks and org.apache.spark.scheduler.TaskSetManager#resourceOffer, in the testcase

No need to do resourceOffers as submitTask will revive offers from task set

### Why are the changes needed?
Fix flaky test

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Test case can pass after the change

Closes apache#27115 from ajithme/testflaky.

Authored-by: Ajith <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
  • Loading branch information
ajithme authored and dongjoon-hyun committed Jan 9, 2020
commit 18daa37cdb9839740816c0d1426a1d27aed218e3
Original file line number Diff line number Diff line change
Expand Up @@ -1940,27 +1940,27 @@ class TaskSetManagerSuite
val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend]

TestUtils.waitUntilExecutorsUp(sc, 2, 60000)
val Seq(exec0, exec1) = backend.getExecutorIds()

val taskSet = FakeTask.createTaskSet(2)
val tasks = Array.tabulate[Task[_]](2)(partition => new FakeLongTasks(stageId = 0, partition))
val taskSet: TaskSet = new TaskSet(tasks, stageId = 0, stageAttemptId = 0, priority = 0, null)
val stageId = taskSet.stageId
val stageAttemptId = taskSet.stageAttemptId
sched.submitTasks(taskSet)
val taskSetManagers = PrivateMethod[mutable.HashMap[Int, mutable.HashMap[Int, TaskSetManager]]](
Symbol("taskSetsByStageIdAndAttempt"))
val taskSetManagers =
PrivateMethod[mutable.HashMap[Int, mutable.HashMap[Int, TaskSetManager]]](
Symbol("taskSetsByStageIdAndAttempt"))
// get the TaskSetManager
val manager = sched.invokePrivate(taskSetManagers()).get(stageId).get(stageAttemptId)

val task0 = manager.resourceOffer(exec0, "localhost", TaskLocality.NO_PREF)
val task1 = manager.resourceOffer(exec1, "localhost", TaskLocality.NO_PREF)
assert(task0.isDefined && task1.isDefined)
val (taskId0, index0) = (task0.get.taskId, task0.get.index)
val (taskId1, index1) = (task1.get.taskId, task1.get.index)
val (task0, task1) = eventually(timeout(10.seconds), interval(100.milliseconds)) {
(manager.taskInfos(0), manager.taskInfos(1))
}

val (taskId0, index0, exec0) = (task0.taskId, task0.index, task0.executorId)
val (taskId1, index1, exec1) = (task1.taskId, task1.index, task1.executorId)
// set up two running tasks
assert(manager.taskInfos(taskId0).running)
assert(manager.taskInfos(taskId0).executorId === exec0)
assert(manager.taskInfos(taskId1).running)
assert(manager.taskInfos(taskId1).executorId === exec1)

val numFailures = PrivateMethod[Array[Int]](Symbol("numFailures"))
// no task failures yet
Expand All @@ -1981,3 +1981,13 @@ class TaskSetManagerSuite
}
}
}

class FakeLongTasks(stageId: Int, partitionId: Int) extends FakeTask(stageId, partitionId) {

override def runTask(context: TaskContext): Int = {
while (true) {
Thread.sleep(10000)
}
0
}
}