aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorJosh Rosen <joshrosen@databricks.com>2016-04-14 16:43:28 -0700
committerJosh Rosen <joshrosen@databricks.com>2016-04-14 16:43:28 -0700
commitee4090b60e8b6a350913d1d5049f0770c251cd4a (patch)
tree7e082fa815430c23e0387461be0726cc3e4d04b5 /core
parent2407f5b14edcdcf750113766d82e78732f9852d6 (diff)
parentd7e124edfe2578ecdf8e816a4dda3ce430a09172 (diff)
downloadspark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.tar.gz
spark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.tar.bz2
spark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.zip
Merge remote-tracking branch 'origin/master' into build-for-2.12
Diffstat (limited to 'core')
-rw-r--r--core/src/main/java/org/apache/spark/JavaSparkListener.java88
-rw-r--r--core/src/main/java/org/apache/spark/SparkExecutorInfo.java33
-rw-r--r--core/src/main/java/org/apache/spark/SparkFirehoseListener.java2
-rw-r--r--core/src/main/java/org/apache/spark/api/java/StorageLevels.java6
-rw-r--r--core/src/main/java/org/apache/spark/io/LZ4BlockInputStream.java (renamed from core/src/main/scala/org/apache/spark/io/LZ4BlockInputStream.java)14
-rw-r--r--core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java6
-rw-r--r--core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java7
-rw-r--r--core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java105
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java7
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java7
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java2
-rw-r--r--core/src/main/resources/org/apache/spark/log4j-defaults.properties4
-rw-r--r--core/src/main/resources/org/apache/spark/ui/static/webui.css8
-rw-r--r--core/src/main/scala/org/apache/spark/ContextCleaner.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/FutureAction.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/HttpServer.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/InternalAccumulator.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/SSLOptions.scala57
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala85
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/SparkStatusTracker.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/StatusAPIImpl.scala33
-rw-r--r--core/src/main/scala/org/apache/spark/TaskContext.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/TaskContextImpl.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala60
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala80
-rw-r--r--core/src/main/scala/org/apache/spark/api/r/RRDD.scala328
-rw-r--r--core/src/main/scala/org/apache/spark/api/r/RRunner.scala368
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala29
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala44
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala52
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala6
-rwxr-xr-xcore/src/main/scala/org/apache/spark/deploy/worker/Worker.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala22
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/executor/InputMetrics.scala32
-rw-r--r--core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala27
-rw-r--r--core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala44
-rw-r--r--core/src/main/scala/org/apache/spark/internal/config/package.scala50
-rw-r--r--core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/memory/MemoryManager.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala22
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/network/BlockTransferService.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala36
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala (renamed from core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDDState.scala)6
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDD.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobListener.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala251
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Stage.scala19
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala199
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Task.scala39
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala70
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala24
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/Serializer.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/api.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala85
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageLevel.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageUtils.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala171
-rw-r--r--core/src/main/scala/org/apache/spark/ui/JettyUtils.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/ui/SparkUI.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/WebUI.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala85
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/util/CausedBy.scala36
-rw-r--r--core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala38
-rw-r--r--core/src/main/scala/org/apache/spark/util/EventLoop.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/util/JsonProtocol.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/util/SizeEstimator.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala112
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala50
-rw-r--r--core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala (renamed from core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala)38
-rw-r--r--core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala203
-rw-r--r--core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java4
-rw-r--r--core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java64
-rw-r--r--core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json24
-rw-r--r--core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json24
-rw-r--r--core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json4
-rw-r--r--core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json3
-rw-r--r--core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json6
-rw-r--r--core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json18
-rw-r--r--core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json3
-rw-r--r--core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json6
-rw-r--r--core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json10
-rw-r--r--core/src/test/resources/log4j.properties3
-rw-r--r--core/src/test/scala/org/apache/spark/AccumulatorSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/DistributedSuite.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/ShuffleSuite.scala13
-rw-r--r--core/src/test/scala/org/apache/spark/Smuggle.scala46
-rw-r--r--core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/UnpersistSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala19
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala9
-rw-r--r--core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala28
-rw-r--r--core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala27
-rw-r--r--core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala3
-rw-r--r--core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala107
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala15
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala3
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala19
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala24
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala36
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala32
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala26
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala61
-rw-r--r--core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala22
-rw-r--r--core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala67
-rw-r--r--core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala10
-rw-r--r--core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala58
-rw-r--r--core/src/test/scala/org/apache/spark/util/CausedBySuite.scala56
-rw-r--r--core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala72
-rw-r--r--core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala159
-rw-r--r--core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala (renamed from core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala)47
-rw-r--r--core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala197
200 files changed, 3317 insertions, 1946 deletions
diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java
deleted file mode 100644
index 23bc9a2e81..0000000000
--- a/core/src/main/java/org/apache/spark/JavaSparkListener.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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;
-
-import org.apache.spark.scheduler.*;
-
-/**
- * Java clients should extend this class instead of implementing
- * SparkListener directly. This is to prevent java clients
- * from breaking when new events are added to the SparkListener
- * trait.
- *
- * This is a concrete class instead of abstract to enforce
- * new events get added to both the SparkListener and this adapter
- * in lockstep.
- */
-public class JavaSparkListener implements SparkListener {
-
- @Override
- public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { }
-
- @Override
- public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { }
-
- @Override
- public void onTaskStart(SparkListenerTaskStart taskStart) { }
-
- @Override
- public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { }
-
- @Override
- public void onTaskEnd(SparkListenerTaskEnd taskEnd) { }
-
- @Override
- public void onJobStart(SparkListenerJobStart jobStart) { }
-
- @Override
- public void onJobEnd(SparkListenerJobEnd jobEnd) { }
-
- @Override
- public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { }
-
- @Override
- public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { }
-
- @Override
- public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { }
-
- @Override
- public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { }
-
- @Override
- public void onApplicationStart(SparkListenerApplicationStart applicationStart) { }
-
- @Override
- public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { }
-
- @Override
- public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { }
-
- @Override
- public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { }
-
- @Override
- public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { }
-
- @Override
- public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { }
-
- @Override
- public void onOtherEvent(SparkListenerEvent event) { }
-
-}
diff --git a/core/src/main/java/org/apache/spark/SparkExecutorInfo.java b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java
new file mode 100644
index 0000000000..dc3e826475
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+/**
+ * Exposes information about Spark Executors.
+ *
+ * This interface is not designed to be implemented outside of Spark. We may add additional methods
+ * which may break binary compatibility with outside implementations.
+ */
+public interface SparkExecutorInfo extends Serializable {
+ String host();
+ int port();
+ long cacheSize();
+ int numRunningTasks();
+}
diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
index e6b24afd88..97eed611e8 100644
--- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
+++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
@@ -28,7 +28,7 @@ import org.apache.spark.scheduler.*;
* this was a concrete Scala class, default implementations of new event handlers would be inherited
* from the SparkListener trait).
*/
-public class SparkFirehoseListener implements SparkListener {
+public class SparkFirehoseListener implements SparkListenerInterface {
public void onEvent(SparkListenerEvent event) { }
diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java
index 666c797738..3fcb52f615 100644
--- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java
+++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java
@@ -34,13 +34,13 @@ public class StorageLevels {
public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2);
public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1);
public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2);
- public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1);
+ public static final StorageLevel OFF_HEAP = create(true, true, true, false, 1);
/**
* Create a new StorageLevel object.
* @param useDisk saved to disk, if true
- * @param useMemory saved to memory, if true
- * @param useOffHeap saved to Tachyon, if true
+ * @param useMemory saved to on-heap memory, if true
+ * @param useOffHeap saved to off-heap memory, if true
* @param deserialized saved as deserialized objects, if true
* @param replication replication factor
*/
diff --git a/core/src/main/scala/org/apache/spark/io/LZ4BlockInputStream.java b/core/src/main/java/org/apache/spark/io/LZ4BlockInputStream.java
index 27b6f0d4a3..8783b5f56e 100644
--- a/core/src/main/scala/org/apache/spark/io/LZ4BlockInputStream.java
+++ b/core/src/main/java/org/apache/spark/io/LZ4BlockInputStream.java
@@ -20,20 +20,17 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.zip.Checksum;
-import net.jpountz.lz4.LZ4BlockOutputStream;
import net.jpountz.lz4.LZ4Exception;
import net.jpountz.lz4.LZ4Factory;
import net.jpountz.lz4.LZ4FastDecompressor;
import net.jpountz.util.SafeUtils;
-import net.jpountz.xxhash.StreamingXXHash32;
-import net.jpountz.xxhash.XXHash32;
import net.jpountz.xxhash.XXHashFactory;
/**
* {@link InputStream} implementation to decode data written with
- * {@link LZ4BlockOutputStream}. This class is not thread-safe and does not
+ * {@link net.jpountz.lz4.LZ4BlockOutputStream}. This class is not thread-safe and does not
* support {@link #mark(int)}/{@link #reset()}.
- * @see LZ4BlockOutputStream
+ * @see net.jpountz.lz4.LZ4BlockOutputStream
*
* This is based on net.jpountz.lz4.LZ4BlockInputStream
*
@@ -90,12 +87,13 @@ public final class LZ4BlockInputStream extends FilterInputStream {
}
/**
- * Create a new instance using {@link XXHash32} for checksuming.
+ * Create a new instance using {@link net.jpountz.xxhash.XXHash32} for checksuming.
* @see #LZ4BlockInputStream(InputStream, LZ4FastDecompressor, Checksum)
- * @see StreamingXXHash32#asChecksum()
+ * @see net.jpountz.xxhash.StreamingXXHash32#asChecksum()
*/
public LZ4BlockInputStream(InputStream in, LZ4FastDecompressor decompressor) {
- this(in, decompressor, XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum());
+ this(in, decompressor,
+ XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum());
}
/**
diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
index 81ee7ab58a..3c2980e442 100644
--- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
@@ -215,8 +215,6 @@ final class ShuffleExternalSorter extends MemoryConsumer {
}
}
- inMemSorter.reset();
-
if (!isLastFile) { // i.e. this is a spill file
// The current semantics of `shuffleRecordsWritten` seem to be that it's updated when records
// are written to disk, not when they enter the shuffle sorting code. DiskBlockObjectWriter
@@ -255,6 +253,10 @@ final class ShuffleExternalSorter extends MemoryConsumer {
writeSortedFile(false);
final long spillSize = freeMemory();
+ inMemSorter.reset();
+ // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the
+ // records. Otherwise, if the task is over allocated memory, then without freeing the memory pages,
+ // we might not be able to get memory for the pointer array.
taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
return spillSize;
}
diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
index fe79ff0e30..76b0e6a304 100644
--- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
@@ -51,9 +51,12 @@ final class ShuffleInMemorySorter {
*/
private int pos = 0;
+ private int initialSize;
+
ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize) {
this.consumer = consumer;
assert (initialSize > 0);
+ this.initialSize = initialSize;
this.array = consumer.allocateArray(initialSize);
this.sorter = new Sorter<>(ShuffleSortDataFormat.INSTANCE);
}
@@ -70,6 +73,10 @@ final class ShuffleInMemorySorter {
}
public void reset() {
+ if (consumer != null) {
+ consumer.freeArray(array);
+ this.array = consumer.allocateArray(initialSize);
+ }
pos = 0;
}
diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
index 9aacb084f6..6807710f9f 100644
--- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
+++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java
@@ -56,9 +56,10 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter;
* Bytes 4 to 8: len(k)
* Bytes 8 to 8 + len(k): key data
* Bytes 8 + len(k) to 8 + len(k) + len(v): value data
+ * Bytes 8 + len(k) + len(v) to 8 + len(k) + len(v) + 8: pointer to next pair
*
* This means that the first four bytes store the entire record (key + value) length. This format
- * is consistent with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter},
+ * is compatible with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter},
* so we can pass records from this map directly into the sorter to sort records in place.
*/
public final class BytesToBytesMap extends MemoryConsumer {
@@ -132,7 +133,12 @@ public final class BytesToBytesMap extends MemoryConsumer {
/**
* Number of keys defined in the map.
*/
- private int numElements;
+ private int numKeys;
+
+ /**
+ * Number of values defined in the map. A key could have multiple values.
+ */
+ private int numValues;
/**
* The map will be expanded once the number of keys exceeds this threshold.
@@ -223,7 +229,12 @@ public final class BytesToBytesMap extends MemoryConsumer {
/**
* Returns the number of keys defined in the map.
*/
- public int numElements() { return numElements; }
+ public int numKeys() { return numKeys; }
+
+ /**
+ * Returns the number of values defined in the map. A key could have multiple values.
+ */
+ public int numValues() { return numValues; }
public final class MapIterator implements Iterator<Location> {
@@ -311,7 +322,8 @@ public final class BytesToBytesMap extends MemoryConsumer {
if (currentPage != null) {
int totalLength = Platform.getInt(pageBaseObject, offsetInPage);
loc.with(currentPage, offsetInPage);
- offsetInPage += 4 + totalLength;
+ // [total size] [key size] [key] [value] [pointer to next]
+ offsetInPage += 4 + totalLength + 8;
recordsInPage --;
return loc;
} else {
@@ -361,7 +373,7 @@ public final class BytesToBytesMap extends MemoryConsumer {
while (numRecords > 0) {
int length = Platform.getInt(base, offset);
writer.write(base, offset + 4, length, 0);
- offset += 4 + length;
+ offset += 4 + length + 8;
numRecords--;
}
writer.close();
@@ -395,7 +407,7 @@ public final class BytesToBytesMap extends MemoryConsumer {
* `lookup()`, the behavior of the returned iterator is undefined.
*/
public MapIterator iterator() {
- return new MapIterator(numElements, loc, false);
+ return new MapIterator(numValues, loc, false);
}
/**
@@ -409,7 +421,7 @@ public final class BytesToBytesMap extends MemoryConsumer {
* `lookup()`, the behavior of the returned iterator is undefined.
*/
public MapIterator destructiveIterator() {
- return new MapIterator(numElements, loc, true);
+ return new MapIterator(numValues, loc, true);
}
/**
@@ -560,6 +572,20 @@ public final class BytesToBytesMap extends MemoryConsumer {
}
/**
+ * Find the next pair that has the same key as current one.
+ */
+ public boolean nextValue() {
+ assert isDefined;
+ long nextAddr = Platform.getLong(baseObject, valueOffset + valueLength);
+ if (nextAddr == 0) {
+ return false;
+ } else {
+ updateAddressesAndSizes(nextAddr);
+ return true;
+ }
+ }
+
+ /**
* Returns the memory page that contains the current record.
* This is only valid if this is returned by {@link BytesToBytesMap#iterator()}.
*/
@@ -625,10 +651,9 @@ public final class BytesToBytesMap extends MemoryConsumer {
}
/**
- * Store a new key and value. This method may only be called once for a given key; if you want
- * to update the value associated with a key, then you can directly manipulate the bytes stored
- * at the value address. The return value indicates whether the put succeeded or whether it
- * failed because additional memory could not be acquired.
+ * Append a new value for the key. This method could be called multiple times for a given key.
+ * The return value indicates whether the put succeeded or whether it failed because additional
+ * memory could not be acquired.
* <p>
* It is only valid to call this method immediately after calling `lookup()` using the same key.
* </p>
@@ -637,7 +662,7 @@ public final class BytesToBytesMap extends MemoryConsumer {
* </p>
* <p>
* After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length`
- * will return information on the data stored by this `putNewKey` call.
+ * will return information on the data stored by this `append` call.
* </p>
* <p>
* As an example usage, here's the proper way to store a new key:
@@ -645,7 +670,7 @@ public final class BytesToBytesMap extends MemoryConsumer {
* <pre>
* Location loc = map.lookup(keyBase, keyOffset, keyLength);
* if (!loc.isDefined()) {
- * if (!loc.putNewKey(keyBase, keyOffset, keyLength, ...)) {
+ * if (!loc.append(keyBase, keyOffset, keyLength, ...)) {
* // handle failure to grow map (by spilling, for example)
* }
* }
@@ -657,26 +682,23 @@ public final class BytesToBytesMap extends MemoryConsumer {
* @return true if the put() was successful and false if the put() failed because memory could
* not be acquired.
*/
- public boolean putNewKey(Object keyBase, long keyOffset, int keyLength,
- Object valueBase, long valueOffset, int valueLength) {
- assert (!isDefined) : "Can only set value once for a key";
- assert (keyLength % 8 == 0);
- assert (valueLength % 8 == 0);
- assert(longArray != null);
+ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff, int vlen) {
+ assert (klen % 8 == 0);
+ assert (vlen % 8 == 0);
+ assert (longArray != null);
-
- if (numElements == MAX_CAPACITY
+ if (numKeys == MAX_CAPACITY
// The map could be reused from last spill (because of no enough memory to grow),
// then we don't try to grow again if hit the `growthThreshold`.
- || !canGrowArray && numElements > growthThreshold) {
+ || !canGrowArray && numKeys > growthThreshold) {
return false;
}
// Here, we'll copy the data into our data pages. Because we only store a relative offset from
// the key address instead of storing the absolute address of the value, the key and value
// must be stored in the same memory page.
- // (8 byte key length) (key) (value)
- final long recordLength = 8 + keyLength + valueLength;
+ // (8 byte key length) (key) (value) (8 byte pointer to next value)
+ final long recordLength = 8 + klen + vlen + 8;
if (currentPage == null || currentPage.size() - pageCursor < recordLength) {
if (!acquireNewPage(recordLength + 4L)) {
return false;
@@ -687,30 +709,36 @@ public final class BytesToBytesMap extends MemoryConsumer {
final Object base = currentPage.getBaseObject();
long offset = currentPage.getBaseOffset() + pageCursor;
final long recordOffset = offset;
- Platform.putInt(base, offset, keyLength + valueLength + 4);
- Platform.putInt(base, offset + 4, keyLength);
+ Platform.putInt(base, offset, klen + vlen + 4);
+ Platform.putInt(base, offset + 4, klen);
offset += 8;
- Platform.copyMemory(keyBase, keyOffset, base, offset, keyLength);
- offset += keyLength;
- Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength);
+ Platform.copyMemory(kbase, koff, base, offset, klen);
+ offset += klen;
+ Platform.copyMemory(vbase, voff, base, offset, vlen);
+ offset += vlen;
+ // put this value at the beginning of the list
+ Platform.putLong(base, offset, isDefined ? longArray.get(pos * 2) : 0);
// --- Update bookkeeping data structures ----------------------------------------------------
offset = currentPage.getBaseOffset();
Platform.putInt(base, offset, Platform.getInt(base, offset) + 1);
pageCursor += recordLength;
- numElements++;
final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset(
currentPage, recordOffset);
longArray.set(pos * 2, storedKeyAddress);
- longArray.set(pos * 2 + 1, keyHashcode);
updateAddressesAndSizes(storedKeyAddress);
- isDefined = true;
+ numValues++;
+ if (!isDefined) {
+ numKeys++;
+ longArray.set(pos * 2 + 1, keyHashcode);
+ isDefined = true;
- if (numElements > growthThreshold && longArray.size() < MAX_CAPACITY) {
- try {
- growAndRehash();
- } catch (OutOfMemoryError oom) {
- canGrowArray = false;
+ if (numKeys > growthThreshold && longArray.size() < MAX_CAPACITY) {
+ try {
+ growAndRehash();
+ } catch (OutOfMemoryError oom) {
+ canGrowArray = false;
+ }
}
}
return true;
@@ -866,7 +894,8 @@ public final class BytesToBytesMap extends MemoryConsumer {
* Reset this map to initialized state.
*/
public void reset() {
- numElements = 0;
+ numKeys = 0;
+ numValues = 0;
longArray.zeroOut();
while (dataPages.size() > 0) {
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index cf7c9a299f..dc9a8db9c5 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -200,14 +200,17 @@ public final class UnsafeExternalSorter extends MemoryConsumer {
spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix());
}
spillWriter.close();
-
- inMemSorter.reset();
}
final long spillSize = freeMemory();
// Note that this is more-or-less going to be a multiple of the page size, so wasted space in
// pages will currently be counted as memory spilled even though that space isn't actually
// written to disk. This also counts the space needed to store the sorter's pointer array.
+ inMemSorter.reset();
+ // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the
+ // records. Otherwise, if the task is over allocated memory, then without freeing the memory pages,
+ // we might not be able to get memory for the pointer array.
+
taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
return spillSize;
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index 145c3a1950..01eae0e8dc 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -84,6 +84,8 @@ public final class UnsafeInMemorySorter {
*/
private int pos = 0;
+ private long initialSize;
+
public UnsafeInMemorySorter(
final MemoryConsumer consumer,
final TaskMemoryManager memoryManager,
@@ -102,6 +104,7 @@ public final class UnsafeInMemorySorter {
LongArray array) {
this.consumer = consumer;
this.memoryManager = memoryManager;
+ this.initialSize = array.size();
if (recordComparator != null) {
this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
@@ -123,6 +126,10 @@ public final class UnsafeInMemorySorter {
}
public void reset() {
+ if (consumer != null) {
+ consumer.freeArray(array);
+ this.array = consumer.allocateArray(initialSize);
+ }
pos = 0;
}
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
index 2b1c860e55..01aed95878 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
@@ -45,7 +45,7 @@ final class UnsafeSorterSpillMerger {
}
}
};
- priorityQueue = new PriorityQueue<UnsafeSorterIterator>(numSpills, comparator);
+ priorityQueue = new PriorityQueue<>(numSpills, comparator);
}
/**
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
index 0750488e4a..89a7963a86 100644
--- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties
+++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
@@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}:
log4j.logger.org.apache.spark.repl.Main=WARN
# Settings to quiet third party logs that are too verbose
-log4j.logger.org.spark-project.jetty=WARN
-log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
+log4j.logger.org.spark_project.jetty=WARN
+log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 48f86d1536..47dd9162a1 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/webui.css
+++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css
@@ -106,21 +106,22 @@ pre {
line-height: 18px;
padding: 6px;
margin: 0;
+ word-break: break-word;
border-radius: 3px;
}
.stage-details {
- max-height: 100px;
overflow-y: auto;
margin: 0;
+ display: block;
transition: max-height 0.25s ease-out, padding 0.25s ease-out;
}
.stage-details.collapsed {
- max-height: 0;
padding-top: 0;
padding-bottom: 0;
border: none;
+ display: none;
}
.description-input {
@@ -143,14 +144,15 @@ pre {
max-height: 300px;
overflow-y: auto;
margin: 0;
+ display: block;
transition: max-height 0.25s ease-out, padding 0.25s ease-out;
}
.stacktrace-details.collapsed {
- max-height: 0;
padding-top: 0;
padding-bottom: 0;
border: none;
+ display: none;
}
span.expand-additional-metrics, span.expand-dag-viz {
diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala
index 8fc657c5eb..76692ccec8 100644
--- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala
@@ -278,9 +278,9 @@ private object ContextCleaner {
* Listener class used for testing when any item has been cleaned by the Cleaner class.
*/
private[spark] trait CleanerListener {
- def rddCleaned(rddId: Int)
- def shuffleCleaned(shuffleId: Int)
- def broadcastCleaned(broadcastId: Long)
- def accumCleaned(accId: Long)
- def checkpointCleaned(rddId: Long)
+ def rddCleaned(rddId: Int): Unit
+ def shuffleCleaned(shuffleId: Int): Unit
+ def broadcastCleaned(broadcastId: Long): Unit
+ def accumCleaned(accId: Long): Unit
+ def checkpointCleaned(rddId: Long): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index 842bfdbadc..8baddf45bf 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -23,6 +23,10 @@ package org.apache.spark
*/
private[spark] trait ExecutorAllocationClient {
+
+ /** Get the list of currently active executors */
+ private[spark] def getExecutorIds(): Seq[String]
+
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index 509f5082f1..882d2b21cf 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -41,7 +41,7 @@ trait FutureAction[T] extends Future[T] {
/**
* Cancels the execution of this action.
*/
- def cancel()
+ def cancel(): Unit
/**
* Blocks until this action completes.
@@ -65,7 +65,7 @@ trait FutureAction[T] extends Future[T] {
* When this action is completed, either through an exception, or a value, applies the provided
* function.
*/
- def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext)
+ def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext): Unit
/**
* Returns whether the action has already been completed with a value or an exception.
@@ -156,16 +156,16 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
/**
- * Handle via which a "run" function passed to a [[ComplexFutureAction]]
- * can submit jobs for execution.
- */
+ * Handle via which a "run" function passed to a [[ComplexFutureAction]]
+ * can submit jobs for execution.
+ */
@DeveloperApi
trait JobSubmitter {
/**
- * Submit a job for execution and return a FutureAction holding the result.
- * This is a wrapper around the same functionality provided by SparkContext
- * to enable cancellation.
- */
+ * Submit a job for execution and return a FutureAction holding the result.
+ * This is a wrapper around the same functionality provided by SparkContext
+ * to enable cancellation.
+ */
def submitJob[T, U, R](
rdd: RDD[T],
processPartition: Iterator[T] => U,
diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
index e8748dd80a..2bdbd3fae9 100644
--- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
+++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
@@ -56,7 +56,7 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean)
* Lives in the driver to receive heartbeats from executors..
*/
private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
- extends ThreadSafeRpcEndpoint with SparkListener with Logging {
+ extends SparkListener with ThreadSafeRpcEndpoint with Logging {
def this(sc: SparkContext) {
this(sc, new SystemClock)
@@ -220,6 +220,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
}
}
-object HeartbeatReceiver {
+
+private[spark] object HeartbeatReceiver {
val ENDPOINT_NAME = "HeartbeatReceiver"
}
diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala
index 9fad1f6786..982b6d6b61 100644
--- a/core/src/main/scala/org/apache/spark/HttpServer.scala
+++ b/core/src/main/scala/org/apache/spark/HttpServer.scala
@@ -25,6 +25,7 @@ import org.eclipse.jetty.server.Server
import org.eclipse.jetty.server.bio.SocketConnector
import org.eclipse.jetty.server.ssl.SslSocketConnector
import org.eclipse.jetty.servlet.{DefaultServlet, ServletContextHandler, ServletHolder}
+import org.eclipse.jetty.util.component.LifeCycle
import org.eclipse.jetty.util.security.{Constraint, Password}
import org.eclipse.jetty.util.thread.QueuedThreadPool
@@ -155,6 +156,12 @@ private[spark] class HttpServer(
throw new ServerStateException("Server is already stopped")
} else {
server.stop()
+ // Stop the ThreadPool if it supports stop() method (through LifeCycle).
+ // It is needed because stopping the Server won't stop the ThreadPool it uses.
+ val threadPool = server.getThreadPool
+ if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) {
+ threadPool.asInstanceOf[LifeCycle].stop
+ }
port = -1
server = null
}
diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala
index 7aa9057858..0dd4ec656f 100644
--- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala
+++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala
@@ -187,7 +187,7 @@ private[spark] object InternalAccumulator {
* add to the same set of accumulators. We do this to report the distribution of accumulator
* values across all tasks within each stage.
*/
- def create(sc: SparkContext): Seq[Accumulator[_]] = {
+ def createAll(sc: SparkContext): Seq[Accumulator[_]] = {
val accums = createAll()
accums.foreach { accum =>
Accumulators.register(accum)
diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala
index 30db6ccbf4..719905a2c9 100644
--- a/core/src/main/scala/org/apache/spark/SSLOptions.scala
+++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala
@@ -132,34 +132,35 @@ private[spark] case class SSLOptions(
private[spark] object SSLOptions extends Logging {
- /** Resolves SSLOptions settings from a given Spark configuration object at a given namespace.
- *
- * The following settings are allowed:
- * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively
- * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory
- * $ - `[ns].keyStorePassword` - a password to the key-store file
- * $ - `[ns].keyPassword` - a password to the private key
- * $ - `[ns].keyStoreType` - the type of the key-store
- * $ - `[ns].needClientAuth` - whether SSL needs client authentication
- * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current
- * directory
- * $ - `[ns].trustStorePassword` - a password to the trust-store file
- * $ - `[ns].trustStoreType` - the type of trust-store
- * $ - `[ns].protocol` - a protocol name supported by a particular Java version
- * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers
- *
- * For a list of protocols and ciphers supported by particular Java versions, you may go to
- * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
- * blog page]].
- *
- * You can optionally specify the default configuration. If you do, for each setting which is
- * missing in SparkConf, the corresponding setting is used from the default configuration.
- *
- * @param conf Spark configuration object where the settings are collected from
- * @param ns the namespace name
- * @param defaults the default configuration
- * @return [[org.apache.spark.SSLOptions]] object
- */
+ /**
+ * Resolves SSLOptions settings from a given Spark configuration object at a given namespace.
+ *
+ * The following settings are allowed:
+ * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively
+ * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory
+ * $ - `[ns].keyStorePassword` - a password to the key-store file
+ * $ - `[ns].keyPassword` - a password to the private key
+ * $ - `[ns].keyStoreType` - the type of the key-store
+ * $ - `[ns].needClientAuth` - whether SSL needs client authentication
+ * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current
+ * directory
+ * $ - `[ns].trustStorePassword` - a password to the trust-store file
+ * $ - `[ns].trustStoreType` - the type of trust-store
+ * $ - `[ns].protocol` - a protocol name supported by a particular Java version
+ * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers
+ *
+ * For a list of protocols and ciphers supported by particular Java versions, you may go to
+ * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
+ * blog page]].
+ *
+ * You can optionally specify the default configuration. If you do, for each setting which is
+ * missing in SparkConf, the corresponding setting is used from the default configuration.
+ *
+ * @param conf Spark configuration object where the settings are collected from
+ * @param ns the namespace name
+ * @param defaults the default configuration
+ * @return [[org.apache.spark.SSLOptions]] object
+ */
def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = {
val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled))
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 5da2e98f1f..acce6bc24f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -419,8 +419,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
*/
private[spark] def getenv(name: String): String = System.getenv(name)
- /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not
- * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */
+ /**
+ * Checks for illegal or deprecated config settings. Throws an exception for the former. Not
+ * idempotent - may mutate this conf object to convert deprecated settings to supported ones.
+ */
private[spark] def validateSettings() {
if (contains("spark.local.dir")) {
val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " +
@@ -454,9 +456,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
"Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit."
throw new Exception(msg)
}
- if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) {
- val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " +
- "Use spark.executor.memory instead."
+ if (javaOpts.contains("-Xmx")) {
+ val msg = s"$executorOptsKey is not allowed to specify max heap memory settings " +
+ s"(was '$javaOpts'). Use spark.executor.memory instead."
throw new Exception(msg)
}
}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index dcb41f3a40..e41088f7c8 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -147,8 +147,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
appName: String,
sparkHome: String = null,
jars: Seq[String] = Nil,
- environment: Map[String, String] = Map()) =
- {
+ environment: Map[String, String] = Map()) = {
this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment))
}
@@ -603,8 +602,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
}
/**
- * Set a local property that affects jobs submitted from this thread, such as the
- * Spark fair scheduler pool.
+ * Set a local property that affects jobs submitted from this thread, such as the Spark fair
+ * scheduler pool. User-defined properties may also be set here. These properties are propagated
+ * through to worker tasks and can be accessed there via
+ * [[org.apache.spark.TaskContext#getLocalProperty]].
*/
def setLocalProperty(key: String, value: String) {
if (value == null) {
@@ -722,7 +723,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
(safeEnd - safeStart) / step + 1
}
}
- parallelize(0 until numSlices, numSlices).mapPartitionsWithIndex((i, _) => {
+ parallelize(0 until numSlices, numSlices).mapPartitionsWithIndex { (i, _) =>
val partitionStart = (i * numElements) / numSlices * step + start
val partitionEnd = (((i + 1) * numElements) / numSlices) * step + start
def getSafeMargin(bi: BigInt): Long =
@@ -761,7 +762,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
ret
}
}
- })
+ }
}
/** Distribute a local Scala collection to form an RDD.
@@ -774,9 +775,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
parallelize(seq, numSlices)
}
- /** Distribute a local Scala collection to form an RDD, with one or more
- * location preferences (hostnames of Spark nodes) for each object.
- * Create a new partition for each collection item. */
+ /**
+ * Distribute a local Scala collection to form an RDD, with one or more
+ * location preferences (hostnames of Spark nodes) for each object.
+ * Create a new partition for each collection item.
+ */
def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope {
assertNotStopped()
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
@@ -1096,14 +1099,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
new NewHadoopRDD(this, fClass, kClass, vClass, jconf)
}
- /** Get an RDD for a Hadoop SequenceFile with given key and value types.
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle
- * operation will create many references to the same object.
- * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
- * copy them using a `map` function.
- */
+ /**
+ * Get an RDD for a Hadoop SequenceFile with given key and value types.
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle
+ * operation will create many references to the same object.
+ * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
+ * copy them using a `map` function.
+ */
def sequenceFile[K, V](path: String,
keyClass: Class[K],
valueClass: Class[V],
@@ -1114,14 +1118,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)
}
- /** Get an RDD for a Hadoop SequenceFile with given key and value types.
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle
- * operation will create many references to the same object.
- * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
- * copy them using a `map` function.
- * */
+ /**
+ * Get an RDD for a Hadoop SequenceFile with given key and value types.
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle
+ * operation will create many references to the same object.
+ * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
+ * copy them using a `map` function.
+ */
def sequenceFile[K, V](
path: String,
keyClass: Class[K],
@@ -1353,10 +1358,20 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* Register a listener to receive up-calls from events that happen during execution.
*/
@DeveloperApi
- def addSparkListener(listener: SparkListener) {
+ def addSparkListener(listener: SparkListenerInterface) {
listenerBus.addListener(listener)
}
+ private[spark] override def getExecutorIds(): Seq[String] = {
+ schedulerBackend match {
+ case b: CoarseGrainedSchedulerBackend =>
+ b.getExecutorIds()
+ case _ =>
+ logWarning("Requesting executors is only supported in coarse-grained mode")
+ Nil
+ }
+ }
+
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
@@ -1994,7 +2009,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
// Use reflection to find the right constructor
val constructors = {
val listenerClass = Utils.classForName(className)
- listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]]
+ listenerClass
+ .getConstructors
+ .asInstanceOf[Array[Constructor[_ <: SparkListenerInterface]]]
}
val constructorTakingSparkConf = constructors.find { c =>
c.getParameterTypes.sameElements(Array(classOf[SparkConf]))
@@ -2002,7 +2019,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
lazy val zeroArgumentConstructor = constructors.find { c =>
c.getParameterTypes.isEmpty
}
- val listener: SparkListener = {
+ val listener: SparkListenerInterface = {
if (constructorTakingSparkConf.isDefined) {
constructorTakingSparkConf.get.newInstance(conf)
} else if (zeroArgumentConstructor.isDefined) {
@@ -2380,9 +2397,8 @@ object SparkContext extends Logging {
} catch {
// TODO: Enumerate the exact reasons why it can fail
// But irrespective of it, it means we cannot proceed !
- case e: Exception => {
+ case e: Exception =>
throw new SparkException("YARN mode not available ?", e)
- }
}
val backend = try {
val clazz =
@@ -2390,9 +2406,8 @@ object SparkContext extends Logging {
val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
} catch {
- case e: Exception => {
+ case e: Exception =>
throw new SparkException("YARN mode not available ?", e)
- }
}
scheduler.initialize(backend)
(backend, scheduler)
@@ -2404,9 +2419,8 @@ object SparkContext extends Logging {
cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
} catch {
- case e: Exception => {
+ case e: Exception =>
throw new SparkException("YARN mode not available ?", e)
- }
}
val backend = try {
@@ -2415,9 +2429,8 @@ object SparkContext extends Logging {
val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
} catch {
- case e: Exception => {
+ case e: Exception =>
throw new SparkException("YARN mode not available ?", e)
- }
}
scheduler.initialize(backend)
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 700e2cb3f9..3d11db7461 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -101,14 +101,13 @@ class SparkEnv (
// We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the
// current working dir in executor which we do not need to delete.
driverTmpDirToDelete match {
- case Some(path) => {
+ case Some(path) =>
try {
Utils.deleteRecursively(new File(path))
} catch {
case e: Exception =>
logWarning(s"Exception while deleting Spark temp dir: $path", e)
}
- }
case None => // We just need to delete tmp dir created by driver, so do nothing on executor
}
}
@@ -314,7 +313,8 @@ object SparkEnv extends Logging {
UnifiedMemoryManager(conf, numUsableCores)
}
- val blockTransferService = new NettyBlockTransferService(conf, securityManager, numUsableCores)
+ val blockTransferService =
+ new NettyBlockTransferService(conf, securityManager, hostname, numUsableCores)
val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint(
BlockManagerMaster.DRIVER_ENDPOINT_NAME,
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
index 34ee3a48f8..52c4656c27 100644
--- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
+++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
@@ -17,6 +17,8 @@
package org.apache.spark
+import org.apache.spark.scheduler.TaskSchedulerImpl
+
/**
* Low-level status reporting APIs for monitoring job and stage progress.
*
@@ -104,4 +106,22 @@ class SparkStatusTracker private[spark] (sc: SparkContext) {
}
}
}
+
+ /**
+ * Returns information of all known executors, including host, port, cacheSize, numRunningTasks.
+ */
+ def getExecutorInfos: Array[SparkExecutorInfo] = {
+ val executorIdToRunningTasks: Map[String, Int] =
+ sc.taskScheduler.asInstanceOf[TaskSchedulerImpl].runningTasksByExecutors()
+
+ sc.getExecutorStorageStatus.map { status =>
+ val bmId = status.blockManagerId
+ new SparkExecutorInfoImpl(
+ bmId.host,
+ bmId.port,
+ status.cacheSize,
+ executorIdToRunningTasks.getOrElse(bmId.executorId, 0)
+ )
+ }
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
index e5c7c8d0db..c1f24a6377 100644
--- a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
+++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
@@ -18,18 +18,25 @@
package org.apache.spark
private class SparkJobInfoImpl (
- val jobId: Int,
- val stageIds: Array[Int],
- val status: JobExecutionStatus)
- extends SparkJobInfo
+ val jobId: Int,
+ val stageIds: Array[Int],
+ val status: JobExecutionStatus)
+ extends SparkJobInfo
private class SparkStageInfoImpl(
- val stageId: Int,
- val currentAttemptId: Int,
- val submissionTime: Long,
- val name: String,
- val numTasks: Int,
- val numActiveTasks: Int,
- val numCompletedTasks: Int,
- val numFailedTasks: Int)
- extends SparkStageInfo
+ val stageId: Int,
+ val currentAttemptId: Int,
+ val submissionTime: Long,
+ val name: String,
+ val numTasks: Int,
+ val numActiveTasks: Int,
+ val numCompletedTasks: Int,
+ val numFailedTasks: Int)
+ extends SparkStageInfo
+
+private class SparkExecutorInfoImpl(
+ val host: String,
+ val port: Int,
+ val cacheSize: Long,
+ val numRunningTasks: Int)
+ extends SparkExecutorInfo
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index 0c1a1dec30..5b2fca4b2d 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -18,6 +18,7 @@
package org.apache.spark
import java.io.Serializable
+import java.util.Properties
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
@@ -64,7 +65,7 @@ object TaskContext {
* An empty task context that does not represent an actual task.
*/
private[spark] def empty(): TaskContextImpl = {
- new TaskContextImpl(0, 0, 0, 0, null, null)
+ new TaskContextImpl(0, 0, 0, 0, null, new Properties, null)
}
}
@@ -170,6 +171,12 @@ abstract class TaskContext extends Serializable {
*/
def taskAttemptId(): Long
+ /**
+ * Get a local property set upstream in the driver, or null if it is missing. See also
+ * [[org.apache.spark.SparkContext.setLocalProperty]].
+ */
+ def getLocalProperty(key: String): String
+
@DeveloperApi
def taskMetrics(): TaskMetrics
diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
index 87dc7f30e7..8b407f9771 100644
--- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
@@ -17,6 +17,8 @@
package org.apache.spark
+import java.util.Properties
+
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.executor.TaskMetrics
@@ -32,6 +34,7 @@ private[spark] class TaskContextImpl(
override val taskAttemptId: Long,
override val attemptNumber: Int,
override val taskMemoryManager: TaskMemoryManager,
+ localProperties: Properties,
@transient private val metricsSystem: MetricsSystem,
initialAccumulators: Seq[Accumulator[_]] = InternalAccumulator.createAll())
extends TaskContext
@@ -119,6 +122,8 @@ private[spark] class TaskContextImpl(
override def isInterrupted(): Boolean = interrupted
+ override def getLocalProperty(key: String): String = localProperties.getProperty(key)
+
override def getMetricsSources(sourceName: String): Seq[Source] =
metricsSystem.getSourcesByName(sourceName)
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index e080f91f50..2897272a8b 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -461,10 +461,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])
fromRDD(rdd.partitionBy(partitioner))
/**
- * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
- * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
- * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
- */
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
+ */
def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] =
fromRDD(rdd.join(other, partitioner))
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index d362c40b7a..dfd91ae338 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -295,13 +295,14 @@ class JavaSparkContext(val sc: SparkContext)
new JavaRDD(sc.binaryRecords(path, recordLength))
}
- /** Get an RDD for a Hadoop SequenceFile with given key and value types.
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD will create many references to the same object.
- * If you plan to directly cache Hadoop writable objects, you should first copy them using
- * a `map` function.
- * */
+ /**
+ * Get an RDD for a Hadoop SequenceFile with given key and value types.
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD will create many references to the same object.
+ * If you plan to directly cache Hadoop writable objects, you should first copy them using
+ * a `map` function.
+ */
def sequenceFile[K, V](path: String,
keyClass: Class[K],
valueClass: Class[V],
@@ -312,13 +313,14 @@ class JavaSparkContext(val sc: SparkContext)
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minPartitions))
}
- /** Get an RDD for a Hadoop SequenceFile.
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD will create many references to the same object.
- * If you plan to directly cache Hadoop writable objects, you should first copy them using
- * a `map` function.
- */
+ /**
+ * Get an RDD for a Hadoop SequenceFile.
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD will create many references to the same object.
+ * If you plan to directly cache Hadoop writable objects, you should first copy them using
+ * a `map` function.
+ */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]):
JavaPairRDD[K, V] = {
implicit val ctagK: ClassTag[K] = ClassTag(keyClass)
@@ -411,13 +413,14 @@ class JavaSparkContext(val sc: SparkContext)
new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
- /** Get an RDD for a Hadoop file with an arbitrary InputFormat.
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD will create many references to the same object.
- * If you plan to directly cache Hadoop writable objects, you should first copy them using
- * a `map` function.
- */
+ /**
+ * Get an RDD for a Hadoop file with an arbitrary InputFormat.
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD will create many references to the same object.
+ * If you plan to directly cache Hadoop writable objects, you should first copy them using
+ * a `map` function.
+ */
def hadoopFile[K, V, F <: InputFormat[K, V]](
path: String,
inputFormatClass: Class[F],
@@ -431,13 +434,14 @@ class JavaSparkContext(val sc: SparkContext)
new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]])
}
- /** Get an RDD for a Hadoop file with an arbitrary InputFormat
- *
- * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
- * record, directly caching the returned RDD will create many references to the same object.
- * If you plan to directly cache Hadoop writable objects, you should first copy them using
- * a `map` function.
- */
+ /**
+ * Get an RDD for a Hadoop file with an arbitrary InputFormat
+ *
+ * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each
+ * record, directly caching the returned RDD will create many references to the same object.
+ * If you plan to directly cache Hadoop writable objects, you should first copy them using
+ * a `map` function.
+ */
def hadoopFile[K, V, F <: InputFormat[K, V]](
path: String,
inputFormatClass: Class[F],
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
index 6f6730690f..6259bead3e 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
@@ -134,11 +134,10 @@ private[python] class JavaToWritableConverter extends Converter[Any, Writable] {
mapWritable.put(convertToWritable(k), convertToWritable(v))
}
mapWritable
- case array: Array[Any] => {
+ case array: Array[Any] =>
val arrayWriteable = new ArrayWritable(classOf[Writable])
arrayWriteable.set(array.map(convertToWritable(_)))
arrayWriteable
- }
case other => throw new SparkException(
s"Data of type ${other.getClass.getName} cannot be used")
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index f423b2ee56..ab5b6c8380 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -59,7 +59,7 @@ private[spark] class PythonRDD(
val asJavaRDD: JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
- val runner = new PythonRunner(func, bufferSize, reuse_worker)
+ val runner = PythonRunner(func, bufferSize, reuse_worker)
runner.compute(firstParent.iterator(split, context), split.index, context)
}
}
@@ -78,17 +78,41 @@ private[spark] case class PythonFunction(
accumulator: Accumulator[JList[Array[Byte]]])
/**
- * A helper class to run Python UDFs in Spark.
+ * A wrapper for chained Python functions (from bottom to top).
+ * @param funcs
+ */
+private[spark] case class ChainedPythonFunctions(funcs: Seq[PythonFunction])
+
+private[spark] object PythonRunner {
+ def apply(func: PythonFunction, bufferSize: Int, reuse_worker: Boolean): PythonRunner = {
+ new PythonRunner(
+ Seq(ChainedPythonFunctions(Seq(func))), bufferSize, reuse_worker, false, Array(Array(0)))
+ }
+}
+
+/**
+ * A helper class to run Python mapPartition/UDFs in Spark.
+ *
+ * funcs is a list of independent Python functions, each one of them is a list of chained Python
+ * functions (from bottom to top).
*/
private[spark] class PythonRunner(
- func: PythonFunction,
+ funcs: Seq[ChainedPythonFunctions],
bufferSize: Int,
- reuse_worker: Boolean)
+ reuse_worker: Boolean,
+ isUDF: Boolean,
+ argOffsets: Array[Array[Int]])
extends Logging {
- private val envVars = func.envVars
- private val pythonExec = func.pythonExec
- private val accumulator = func.accumulator
+ require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs")
+
+ // All the Python functions should have the same exec, version and envvars.
+ private val envVars = funcs.head.funcs.head.envVars
+ private val pythonExec = funcs.head.funcs.head.pythonExec
+ private val pythonVer = funcs.head.funcs.head.pythonVer
+
+ // TODO: support accumulator in multiple UDF
+ private val accumulator = funcs.head.funcs.head.accumulator
def compute(
inputIterator: Iterator[_],
@@ -228,10 +252,8 @@ private[spark] class PythonRunner(
@volatile private var _exception: Exception = null
- private val pythonVer = func.pythonVer
- private val pythonIncludes = func.pythonIncludes
- private val broadcastVars = func.broadcastVars
- private val command = func.command
+ private val pythonIncludes = funcs.flatMap(_.funcs.flatMap(_.pythonIncludes.asScala)).toSet
+ private val broadcastVars = funcs.flatMap(_.funcs.flatMap(_.broadcastVars.asScala))
setDaemon(true)
@@ -256,13 +278,13 @@ private[spark] class PythonRunner(
// sparkFilesDir
PythonRDD.writeUTF(SparkFiles.getRootDirectory(), dataOut)
// Python includes (*.zip and *.egg files)
- dataOut.writeInt(pythonIncludes.size())
- for (include <- pythonIncludes.asScala) {
+ dataOut.writeInt(pythonIncludes.size)
+ for (include <- pythonIncludes) {
PythonRDD.writeUTF(include, dataOut)
}
// Broadcast variables
val oldBids = PythonRDD.getWorkerBroadcasts(worker)
- val newBids = broadcastVars.asScala.map(_.id).toSet
+ val newBids = broadcastVars.map(_.id).toSet
// number of different broadcasts
val toRemove = oldBids.diff(newBids)
val cnt = toRemove.size + newBids.diff(oldBids).size
@@ -272,7 +294,7 @@ private[spark] class PythonRunner(
dataOut.writeLong(- bid - 1) // bid >= 0
oldBids.remove(bid)
}
- for (broadcast <- broadcastVars.asScala) {
+ for (broadcast <- broadcastVars) {
if (!oldBids.contains(broadcast.id)) {
// send new broadcast
dataOut.writeLong(broadcast.id)
@@ -282,8 +304,26 @@ private[spark] class PythonRunner(
}
dataOut.flush()
// Serialized command:
- dataOut.writeInt(command.length)
- dataOut.write(command)
+ if (isUDF) {
+ dataOut.writeInt(1)
+ dataOut.writeInt(funcs.length)
+ funcs.zip(argOffsets).foreach { case (chained, offsets) =>
+ dataOut.writeInt(offsets.length)
+ offsets.foreach { offset =>
+ dataOut.writeInt(offset)
+ }
+ dataOut.writeInt(chained.funcs.length)
+ chained.funcs.foreach { f =>
+ dataOut.writeInt(f.command.length)
+ dataOut.write(f.command)
+ }
+ }
+ } else {
+ dataOut.writeInt(0)
+ val command = funcs.head.funcs.head.command
+ dataOut.writeInt(command.length)
+ dataOut.write(command)
+ }
// Data values
PythonRDD.writeIteratorToStream(inputIterator, dataOut)
dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION)
@@ -413,6 +453,10 @@ private[spark] object PythonRDD extends Logging {
serveIterator(rdd.collect().iterator, s"serve RDD ${rdd.id}")
}
+ def toLocalIteratorAndServe[T](rdd: RDD[T]): Int = {
+ serveIterator(rdd.toLocalIterator, s"serve toLocalIterator")
+ }
+
def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int):
JavaRDD[Array[Byte]] = {
val file = new DataInputStream(new FileInputStream(filename))
@@ -426,7 +470,7 @@ private[spark] object PythonRDD extends Logging {
objs.append(obj)
}
} catch {
- case eof: EOFException => {}
+ case eof: EOFException => // No-op
}
JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
} finally {
diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
index 588a57e65f..606ba6ef86 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala
@@ -17,21 +17,16 @@
package org.apache.spark.api.r
-import java.io._
-import java.net.{InetAddress, ServerSocket}
-import java.util.{Arrays, Map => JMap}
+import java.util.{Map => JMap}
import scala.collection.JavaConverters._
-import scala.io.Source
import scala.reflect.ClassTag
-import scala.util.Try
import org.apache.spark._
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
-import org.apache.spark.util.Utils
private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
parent: RDD[T],
@@ -42,188 +37,16 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
packageNames: Array[Byte],
broadcastVars: Array[Broadcast[Object]])
extends RDD[U](parent) with Logging {
- protected var dataStream: DataInputStream = _
- private var bootTime: Double = _
override def getPartitions: Array[Partition] = parent.partitions
override def compute(partition: Partition, context: TaskContext): Iterator[U] = {
-
- // Timing start
- bootTime = System.currentTimeMillis / 1000.0
+ val runner = new RRunner[U](
+ func, deserializer, serializer, packageNames, broadcastVars, numPartitions)
// The parent may be also an RRDD, so we should launch it first.
val parentIterator = firstParent[T].iterator(partition, context)
- // we expect two connections
- val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost"))
- val listenPort = serverSocket.getLocalPort()
-
- // The stdout/stderr is shared by multiple tasks, because we use one daemon
- // to launch child process as worker.
- val errThread = RRDD.createRWorker(listenPort)
-
- // We use two sockets to separate input and output, then it's easy to manage
- // the lifecycle of them to avoid deadlock.
- // TODO: optimize it to use one socket
-
- // the socket used to send out the input of task
- serverSocket.setSoTimeout(10000)
- val inSocket = serverSocket.accept()
- startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index)
-
- // the socket used to receive the output of task
- val outSocket = serverSocket.accept()
- val inputStream = new BufferedInputStream(outSocket.getInputStream)
- dataStream = new DataInputStream(inputStream)
- serverSocket.close()
-
- try {
-
- return new Iterator[U] {
- def next(): U = {
- val obj = _nextObj
- if (hasNext) {
- _nextObj = read()
- }
- obj
- }
-
- var _nextObj = read()
-
- def hasNext(): Boolean = {
- val hasMore = (_nextObj != null)
- if (!hasMore) {
- dataStream.close()
- }
- hasMore
- }
- }
- } catch {
- case e: Exception =>
- throw new SparkException("R computation failed with\n " + errThread.getLines())
- }
- }
-
- /**
- * Start a thread to write RDD data to the R process.
- */
- private def startStdinThread[T](
- output: OutputStream,
- iter: Iterator[T],
- partition: Int): Unit = {
-
- val env = SparkEnv.get
- val taskContext = TaskContext.get()
- val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
- val stream = new BufferedOutputStream(output, bufferSize)
-
- new Thread("writer for R") {
- override def run(): Unit = {
- try {
- SparkEnv.set(env)
- TaskContext.setTaskContext(taskContext)
- val dataOut = new DataOutputStream(stream)
- dataOut.writeInt(partition)
-
- SerDe.writeString(dataOut, deserializer)
- SerDe.writeString(dataOut, serializer)
-
- dataOut.writeInt(packageNames.length)
- dataOut.write(packageNames)
-
- dataOut.writeInt(func.length)
- dataOut.write(func)
-
- dataOut.writeInt(broadcastVars.length)
- broadcastVars.foreach { broadcast =>
- // TODO(shivaram): Read a Long in R to avoid this cast
- dataOut.writeInt(broadcast.id.toInt)
- // TODO: Pass a byte array from R to avoid this cast ?
- val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]]
- dataOut.writeInt(broadcastByteArr.length)
- dataOut.write(broadcastByteArr)
- }
-
- dataOut.writeInt(numPartitions)
-
- if (!iter.hasNext) {
- dataOut.writeInt(0)
- } else {
- dataOut.writeInt(1)
- }
-
- val printOut = new PrintStream(stream)
-
- def writeElem(elem: Any): Unit = {
- if (deserializer == SerializationFormats.BYTE) {
- val elemArr = elem.asInstanceOf[Array[Byte]]
- dataOut.writeInt(elemArr.length)
- dataOut.write(elemArr)
- } else if (deserializer == SerializationFormats.ROW) {
- dataOut.write(elem.asInstanceOf[Array[Byte]])
- } else if (deserializer == SerializationFormats.STRING) {
- // write string(for StringRRDD)
- // scalastyle:off println
- printOut.println(elem)
- // scalastyle:on println
- }
- }
-
- for (elem <- iter) {
- elem match {
- case (key, value) =>
- writeElem(key)
- writeElem(value)
- case _ =>
- writeElem(elem)
- }
- }
- stream.flush()
- } catch {
- // TODO: We should propogate this error to the task thread
- case e: Exception =>
- logError("R Writer thread got an exception", e)
- } finally {
- Try(output.close())
- }
- }
- }.start()
- }
-
- protected def readData(length: Int): U
-
- protected def read(): U = {
- try {
- val length = dataStream.readInt()
-
- length match {
- case SpecialLengths.TIMING_DATA =>
- // Timing data from R worker
- val boot = dataStream.readDouble - bootTime
- val init = dataStream.readDouble
- val broadcast = dataStream.readDouble
- val input = dataStream.readDouble
- val compute = dataStream.readDouble
- val output = dataStream.readDouble
- logInfo(
- ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " +
- "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " +
- "total = %.3f s").format(
- boot,
- init,
- broadcast,
- input,
- compute,
- output,
- boot + init + broadcast + input + compute + output))
- read()
- case length if length >= 0 =>
- readData(length)
- }
- } catch {
- case eof: EOFException =>
- throw new SparkException("R worker exited unexpectedly (cranshed)", eof)
- }
+ runner.compute(parentIterator, partition.index, context)
}
}
@@ -242,19 +65,6 @@ private class PairwiseRRDD[T: ClassTag](
parent, numPartitions, hashFunc, deserializer,
SerializationFormats.BYTE, packageNames,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
-
- override protected def readData(length: Int): (Int, Array[Byte]) = {
- length match {
- case length if length == 2 =>
- val hashedKey = dataStream.readInt()
- val contentPairsLength = dataStream.readInt()
- val contentPairs = new Array[Byte](contentPairsLength)
- dataStream.readFully(contentPairs)
- (hashedKey, contentPairs)
- case _ => null
- }
- }
-
lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this)
}
@@ -271,17 +81,6 @@ private class RRDD[T: ClassTag](
extends BaseRRDD[T, Array[Byte]](
parent, -1, func, deserializer, serializer, packageNames,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
-
- override protected def readData(length: Int): Array[Byte] = {
- length match {
- case length if length > 0 =>
- val obj = new Array[Byte](length)
- dataStream.readFully(obj)
- obj
- case _ => null
- }
- }
-
lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
}
@@ -297,55 +96,10 @@ private class StringRRDD[T: ClassTag](
extends BaseRRDD[T, String](
parent, -1, func, deserializer, SerializationFormats.STRING, packageNames,
broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) {
-
- override protected def readData(length: Int): String = {
- length match {
- case length if length > 0 =>
- SerDe.readStringBytes(dataStream, length)
- case _ => null
- }
- }
-
lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this)
}
-private object SpecialLengths {
- val TIMING_DATA = -1
-}
-
-private[r] class BufferedStreamThread(
- in: InputStream,
- name: String,
- errBufferSize: Int) extends Thread(name) with Logging {
- val lines = new Array[String](errBufferSize)
- var lineIdx = 0
- override def run() {
- for (line <- Source.fromInputStream(in).getLines) {
- synchronized {
- lines(lineIdx) = line
- lineIdx = (lineIdx + 1) % errBufferSize
- }
- logInfo(line)
- }
- }
-
- def getLines(): String = synchronized {
- (0 until errBufferSize).filter { x =>
- lines((x + lineIdx) % errBufferSize) != null
- }.map { x =>
- lines((x + lineIdx) % errBufferSize)
- }.mkString("\n")
- }
-}
-
private[r] object RRDD {
- // Because forking processes from Java is expensive, we prefer to launch
- // a single R daemon (daemon.R) and tell it to fork new workers for our tasks.
- // This daemon currently only works on UNIX-based systems now, so we should
- // also fall back to launching workers (worker.R) directly.
- private[this] var errThread: BufferedStreamThread = _
- private[this] var daemonChannel: DataOutputStream = _
-
def createSparkContext(
master: String,
appName: String,
@@ -353,7 +107,6 @@ private[r] object RRDD {
jars: Array[String],
sparkEnvirMap: JMap[Object, Object],
sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = {
-
val sparkConf = new SparkConf().setAppName(appName)
.setSparkHome(sparkHome)
@@ -381,83 +134,10 @@ private[r] object RRDD {
}
/**
- * Start a thread to print the process's stderr to ours
- */
- private def startStdoutThread(proc: Process): BufferedStreamThread = {
- val BUFFER_SIZE = 100
- val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE)
- thread.setDaemon(true)
- thread.start()
- thread
- }
-
- private def createRProcess(port: Int, script: String): BufferedStreamThread = {
- // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command",
- // but kept here for backward compatibility.
- val sparkConf = SparkEnv.get.conf
- var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript")
- rCommand = sparkConf.get("spark.r.command", rCommand)
-
- val rOptions = "--vanilla"
- val rLibDir = RUtils.sparkRPackagePath(isDriver = false)
- val rExecScript = rLibDir(0) + "/SparkR/worker/" + script
- val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript))
- // Unset the R_TESTS environment variable for workers.
- // This is set by R CMD check as startup.Rs
- // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R)
- // and confuses worker script which tries to load a non-existent file
- pb.environment().put("R_TESTS", "")
- pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(","))
- pb.environment().put("SPARKR_WORKER_PORT", port.toString)
- pb.redirectErrorStream(true) // redirect stderr into stdout
- val proc = pb.start()
- val errThread = startStdoutThread(proc)
- errThread
- }
-
- /**
- * ProcessBuilder used to launch worker R processes.
- */
- def createRWorker(port: Int): BufferedStreamThread = {
- val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true)
- if (!Utils.isWindows && useDaemon) {
- synchronized {
- if (daemonChannel == null) {
- // we expect one connections
- val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost"))
- val daemonPort = serverSocket.getLocalPort
- errThread = createRProcess(daemonPort, "daemon.R")
- // the socket used to send out the input of task
- serverSocket.setSoTimeout(10000)
- val sock = serverSocket.accept()
- daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
- serverSocket.close()
- }
- try {
- daemonChannel.writeInt(port)
- daemonChannel.flush()
- } catch {
- case e: IOException =>
- // daemon process died
- daemonChannel.close()
- daemonChannel = null
- errThread = null
- // fail the current task, retry by scheduler
- throw e
- }
- errThread
- }
- } else {
- createRProcess(port, "worker.R")
- }
- }
-
- /**
* Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is
* called from R.
*/
def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = {
JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length))
}
-
}
diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
new file mode 100644
index 0000000000..07d1fa2c4a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala
@@ -0,0 +1,368 @@
+/*
+ * 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.api.r
+
+import java.io._
+import java.net.{InetAddress, ServerSocket}
+import java.util.Arrays
+
+import scala.io.Source
+import scala.util.Try
+
+import org.apache.spark._
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * A helper class to run R UDFs in Spark.
+ */
+private[spark] class RRunner[U](
+ func: Array[Byte],
+ deserializer: String,
+ serializer: String,
+ packageNames: Array[Byte],
+ broadcastVars: Array[Broadcast[Object]],
+ numPartitions: Int = -1)
+ extends Logging {
+ private var bootTime: Double = _
+ private var dataStream: DataInputStream = _
+ val readData = numPartitions match {
+ case -1 =>
+ serializer match {
+ case SerializationFormats.STRING => readStringData _
+ case _ => readByteArrayData _
+ }
+ case _ => readShuffledData _
+ }
+
+ def compute(
+ inputIterator: Iterator[_],
+ partitionIndex: Int,
+ context: TaskContext): Iterator[U] = {
+ // Timing start
+ bootTime = System.currentTimeMillis / 1000.0
+
+ // we expect two connections
+ val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost"))
+ val listenPort = serverSocket.getLocalPort()
+
+ // The stdout/stderr is shared by multiple tasks, because we use one daemon
+ // to launch child process as worker.
+ val errThread = RRunner.createRWorker(listenPort)
+
+ // We use two sockets to separate input and output, then it's easy to manage
+ // the lifecycle of them to avoid deadlock.
+ // TODO: optimize it to use one socket
+
+ // the socket used to send out the input of task
+ serverSocket.setSoTimeout(10000)
+ val inSocket = serverSocket.accept()
+ startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex)
+
+ // the socket used to receive the output of task
+ val outSocket = serverSocket.accept()
+ val inputStream = new BufferedInputStream(outSocket.getInputStream)
+ dataStream = new DataInputStream(inputStream)
+ serverSocket.close()
+
+ try {
+ return new Iterator[U] {
+ def next(): U = {
+ val obj = _nextObj
+ if (hasNext) {
+ _nextObj = read()
+ }
+ obj
+ }
+
+ var _nextObj = read()
+
+ def hasNext(): Boolean = {
+ val hasMore = (_nextObj != null)
+ if (!hasMore) {
+ dataStream.close()
+ }
+ hasMore
+ }
+ }
+ } catch {
+ case e: Exception =>
+ throw new SparkException("R computation failed with\n " + errThread.getLines())
+ }
+ }
+
+ /**
+ * Start a thread to write RDD data to the R process.
+ */
+ private def startStdinThread(
+ output: OutputStream,
+ iter: Iterator[_],
+ partitionIndex: Int): Unit = {
+ val env = SparkEnv.get
+ val taskContext = TaskContext.get()
+ val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+ val stream = new BufferedOutputStream(output, bufferSize)
+
+ new Thread("writer for R") {
+ override def run(): Unit = {
+ try {
+ SparkEnv.set(env)
+ TaskContext.setTaskContext(taskContext)
+ val dataOut = new DataOutputStream(stream)
+ dataOut.writeInt(partitionIndex)
+
+ SerDe.writeString(dataOut, deserializer)
+ SerDe.writeString(dataOut, serializer)
+
+ dataOut.writeInt(packageNames.length)
+ dataOut.write(packageNames)
+
+ dataOut.writeInt(func.length)
+ dataOut.write(func)
+
+ dataOut.writeInt(broadcastVars.length)
+ broadcastVars.foreach { broadcast =>
+ // TODO(shivaram): Read a Long in R to avoid this cast
+ dataOut.writeInt(broadcast.id.toInt)
+ // TODO: Pass a byte array from R to avoid this cast ?
+ val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]]
+ dataOut.writeInt(broadcastByteArr.length)
+ dataOut.write(broadcastByteArr)
+ }
+
+ dataOut.writeInt(numPartitions)
+
+ if (!iter.hasNext) {
+ dataOut.writeInt(0)
+ } else {
+ dataOut.writeInt(1)
+ }
+
+ val printOut = new PrintStream(stream)
+
+ def writeElem(elem: Any): Unit = {
+ if (deserializer == SerializationFormats.BYTE) {
+ val elemArr = elem.asInstanceOf[Array[Byte]]
+ dataOut.writeInt(elemArr.length)
+ dataOut.write(elemArr)
+ } else if (deserializer == SerializationFormats.ROW) {
+ dataOut.write(elem.asInstanceOf[Array[Byte]])
+ } else if (deserializer == SerializationFormats.STRING) {
+ // write string(for StringRRDD)
+ // scalastyle:off println
+ printOut.println(elem)
+ // scalastyle:on println
+ }
+ }
+
+ for (elem <- iter) {
+ elem match {
+ case (key, value) =>
+ writeElem(key)
+ writeElem(value)
+ case _ =>
+ writeElem(elem)
+ }
+ }
+ stream.flush()
+ } catch {
+ // TODO: We should propagate this error to the task thread
+ case e: Exception =>
+ logError("R Writer thread got an exception", e)
+ } finally {
+ Try(output.close())
+ }
+ }
+ }.start()
+ }
+
+ private def read(): U = {
+ try {
+ val length = dataStream.readInt()
+
+ length match {
+ case SpecialLengths.TIMING_DATA =>
+ // Timing data from R worker
+ val boot = dataStream.readDouble - bootTime
+ val init = dataStream.readDouble
+ val broadcast = dataStream.readDouble
+ val input = dataStream.readDouble
+ val compute = dataStream.readDouble
+ val output = dataStream.readDouble
+ logInfo(
+ ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " +
+ "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " +
+ "total = %.3f s").format(
+ boot,
+ init,
+ broadcast,
+ input,
+ compute,
+ output,
+ boot + init + broadcast + input + compute + output))
+ read()
+ case length if length >= 0 =>
+ readData(length).asInstanceOf[U]
+ }
+ } catch {
+ case eof: EOFException =>
+ throw new SparkException("R worker exited unexpectedly (cranshed)", eof)
+ }
+ }
+
+ private def readShuffledData(length: Int): (Int, Array[Byte]) = {
+ length match {
+ case length if length == 2 =>
+ val hashedKey = dataStream.readInt()
+ val contentPairsLength = dataStream.readInt()
+ val contentPairs = new Array[Byte](contentPairsLength)
+ dataStream.readFully(contentPairs)
+ (hashedKey, contentPairs)
+ case _ => null
+ }
+ }
+
+ private def readByteArrayData(length: Int): Array[Byte] = {
+ length match {
+ case length if length > 0 =>
+ val obj = new Array[Byte](length)
+ dataStream.readFully(obj)
+ obj
+ case _ => null
+ }
+ }
+
+ private def readStringData(length: Int): String = {
+ length match {
+ case length if length > 0 =>
+ SerDe.readStringBytes(dataStream, length)
+ case _ => null
+ }
+ }
+}
+
+private object SpecialLengths {
+ val TIMING_DATA = -1
+}
+
+private[r] class BufferedStreamThread(
+ in: InputStream,
+ name: String,
+ errBufferSize: Int) extends Thread(name) with Logging {
+ val lines = new Array[String](errBufferSize)
+ var lineIdx = 0
+ override def run() {
+ for (line <- Source.fromInputStream(in).getLines) {
+ synchronized {
+ lines(lineIdx) = line
+ lineIdx = (lineIdx + 1) % errBufferSize
+ }
+ logInfo(line)
+ }
+ }
+
+ def getLines(): String = synchronized {
+ (0 until errBufferSize).filter { x =>
+ lines((x + lineIdx) % errBufferSize) != null
+ }.map { x =>
+ lines((x + lineIdx) % errBufferSize)
+ }.mkString("\n")
+ }
+}
+
+private[r] object RRunner {
+ // Because forking processes from Java is expensive, we prefer to launch
+ // a single R daemon (daemon.R) and tell it to fork new workers for our tasks.
+ // This daemon currently only works on UNIX-based systems now, so we should
+ // also fall back to launching workers (worker.R) directly.
+ private[this] var errThread: BufferedStreamThread = _
+ private[this] var daemonChannel: DataOutputStream = _
+
+ /**
+ * Start a thread to print the process's stderr to ours
+ */
+ private def startStdoutThread(proc: Process): BufferedStreamThread = {
+ val BUFFER_SIZE = 100
+ val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE)
+ thread.setDaemon(true)
+ thread.start()
+ thread
+ }
+
+ private def createRProcess(port: Int, script: String): BufferedStreamThread = {
+ // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command",
+ // but kept here for backward compatibility.
+ val sparkConf = SparkEnv.get.conf
+ var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript")
+ rCommand = sparkConf.get("spark.r.command", rCommand)
+
+ val rOptions = "--vanilla"
+ val rLibDir = RUtils.sparkRPackagePath(isDriver = false)
+ val rExecScript = rLibDir(0) + "/SparkR/worker/" + script
+ val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript))
+ // Unset the R_TESTS environment variable for workers.
+ // This is set by R CMD check as startup.Rs
+ // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R)
+ // and confuses worker script which tries to load a non-existent file
+ pb.environment().put("R_TESTS", "")
+ pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(","))
+ pb.environment().put("SPARKR_WORKER_PORT", port.toString)
+ pb.redirectErrorStream(true) // redirect stderr into stdout
+ val proc = pb.start()
+ val errThread = startStdoutThread(proc)
+ errThread
+ }
+
+ /**
+ * ProcessBuilder used to launch worker R processes.
+ */
+ def createRWorker(port: Int): BufferedStreamThread = {
+ val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true)
+ if (!Utils.isWindows && useDaemon) {
+ synchronized {
+ if (daemonChannel == null) {
+ // we expect one connections
+ val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost"))
+ val daemonPort = serverSocket.getLocalPort
+ errThread = createRProcess(daemonPort, "daemon.R")
+ // the socket used to send out the input of task
+ serverSocket.setSoTimeout(10000)
+ val sock = serverSocket.accept()
+ daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
+ serverSocket.close()
+ }
+ try {
+ daemonChannel.writeInt(port)
+ daemonChannel.flush()
+ } catch {
+ case e: IOException =>
+ // daemon process died
+ daemonChannel.close()
+ daemonChannel = null
+ errThread = null
+ // fail the current task, retry by scheduler
+ throw e
+ }
+ errThread
+ }
+ } else {
+ createRProcess(port, "worker.R")
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index e5e6a9e4a8..632b0ae9c2 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -30,7 +30,7 @@ import org.apache.spark.io.CompressionCodec
import org.apache.spark.serializer.Serializer
import org.apache.spark.storage.{BlockId, BroadcastBlockId, StorageLevel}
import org.apache.spark.util.{ByteBufferInputStream, Utils}
-import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer}
+import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream}
/**
* A BitTorrent-like implementation of [[org.apache.spark.broadcast.Broadcast]].
@@ -228,12 +228,12 @@ private object TorrentBroadcast extends Logging {
blockSize: Int,
serializer: Serializer,
compressionCodec: Option[CompressionCodec]): Array[ByteBuffer] = {
- val bos = new ByteArrayChunkOutputStream(blockSize)
- val out: OutputStream = compressionCodec.map(c => c.compressedOutputStream(bos)).getOrElse(bos)
+ val cbbos = new ChunkedByteBufferOutputStream(blockSize, ByteBuffer.allocate)
+ val out = compressionCodec.map(c => c.compressedOutputStream(cbbos)).getOrElse(cbbos)
val ser = serializer.newInstance()
val serOut = ser.serializeStream(out)
serOut.writeObject[T](obj).close()
- bos.toArrays.map(ByteBuffer.wrap)
+ cbbos.toChunkedByteBuffer.getChunks()
}
def unBlockifyObject[T: ClassTag](
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 06b7b388ca..cda9d38c6a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -74,13 +74,12 @@ class SparkHadoopUtil extends Logging {
}
}
+
/**
- * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop
- * subsystems.
+ * Appends S3-specific, spark.hadoop.*, and spark.buffer.size configurations to a Hadoop
+ * configuration.
*/
- def newConfiguration(conf: SparkConf): Configuration = {
- val hadoopConf = new Configuration()
-
+ def appendS3AndSparkHadoopConfigurations(conf: SparkConf, hadoopConf: Configuration): Unit = {
// Note: this null check is around more than just access to the "conf" object to maintain
// the behavior of the old implementation of this code, for backwards compatibility.
if (conf != null) {
@@ -106,7 +105,15 @@ class SparkHadoopUtil extends Logging {
val bufferSize = conf.get("spark.buffer.size", "65536")
hadoopConf.set("io.file.buffer.size", bufferSize)
}
+ }
+ /**
+ * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop
+ * subsystems.
+ */
+ def newConfiguration(conf: SparkConf): Configuration = {
+ val hadoopConf = new Configuration()
+ appendS3AndSparkHadoopConfigurations(conf, hadoopConf)
hadoopConf
}
@@ -145,10 +152,9 @@ class SparkHadoopUtil extends Logging {
val baselineBytesRead = f()
Some(() => f() - baselineBytesRead)
} catch {
- case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => {
+ case e @ (_: NoSuchMethodException | _: ClassNotFoundException) =>
logDebug("Couldn't find method for retrieving thread-level FileSystem input data", e)
None
- }
}
}
@@ -167,10 +173,9 @@ class SparkHadoopUtil extends Logging {
val baselineBytesWritten = f()
Some(() => f() - baselineBytesWritten)
} catch {
- case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => {
+ case e @ (_: NoSuchMethodException | _: ClassNotFoundException) =>
logDebug("Couldn't find method for retrieving thread-level FileSystem output data", e)
None
- }
}
}
@@ -308,7 +313,7 @@ class SparkHadoopUtil extends Logging {
*/
def substituteHadoopVariables(text: String, hadoopConf: Configuration): String = {
text match {
- case HADOOP_CONF_PATTERN(matched) => {
+ case HADOOP_CONF_PATTERN(matched) =>
logDebug(text + " matched " + HADOOP_CONF_PATTERN)
val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. }
val eval = Option[String](hadoopConf.get(key))
@@ -323,11 +328,9 @@ class SparkHadoopUtil extends Logging {
// Continue to substitute more variables.
substituteHadoopVariables(eval.get, hadoopConf)
}
- }
- case _ => {
+ case _ =>
logDebug(text + " didn't match " + HADOOP_CONF_PATTERN)
text
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 4049fc0c41..926e1ff7a8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -441,7 +441,6 @@ object SparkSubmit {
OptionAssigner(args.deployMode, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
sysProp = "spark.submit.deployMode"),
OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"),
- OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"),
OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars.ivy"),
OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT,
sysProp = "spark.driver.memory"),
@@ -452,27 +451,15 @@ object SparkSubmit {
OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
sysProp = "spark.driver.extraLibraryPath"),
- // Yarn client only
- OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"),
+ // Yarn only
+ OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.queue"),
OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES,
sysProp = "spark.executor.instances"),
- OptionAssigner(args.files, YARN, CLIENT, sysProp = "spark.yarn.dist.files"),
- OptionAssigner(args.archives, YARN, CLIENT, sysProp = "spark.yarn.dist.archives"),
- OptionAssigner(args.principal, YARN, CLIENT, sysProp = "spark.yarn.principal"),
- OptionAssigner(args.keytab, YARN, CLIENT, sysProp = "spark.yarn.keytab"),
-
- // Yarn cluster only
- OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"),
- OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"),
- OptionAssigner(args.driverCores, YARN, CLUSTER, clOption = "--driver-cores"),
- OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"),
- OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"),
- OptionAssigner(args.executorCores, YARN, CLUSTER, clOption = "--executor-cores"),
- OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"),
- OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"),
- OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"),
- OptionAssigner(args.principal, YARN, CLUSTER, clOption = "--principal"),
- OptionAssigner(args.keytab, YARN, CLUSTER, clOption = "--keytab"),
+ OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.jars"),
+ OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"),
+ OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.archives"),
+ OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"),
+ OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"),
// Other options
OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES,
@@ -483,10 +470,11 @@ object SparkSubmit {
sysProp = "spark.cores.max"),
OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES,
sysProp = "spark.files"),
- OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"),
- OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER,
+ OptionAssigner(args.jars, LOCAL, CLIENT, sysProp = "spark.jars"),
+ OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.jars"),
+ OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER,
sysProp = "spark.driver.memory"),
- OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER,
+ OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER,
sysProp = "spark.driver.cores"),
OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER,
sysProp = "spark.driver.supervise"),
@@ -550,6 +538,10 @@ object SparkSubmit {
if (args.isPython) {
sysProps.put("spark.yarn.isPython", "true")
}
+
+ if (args.pyFiles != null) {
+ sysProps("spark.submit.pyFiles") = args.pyFiles
+ }
}
// assure a keytab is available from any place in a JVM
@@ -576,9 +568,6 @@ object SparkSubmit {
childMainClass = "org.apache.spark.deploy.yarn.Client"
if (args.isPython) {
childArgs += ("--primary-py-file", args.primaryResource)
- if (args.pyFiles != null) {
- childArgs += ("--py-files", args.pyFiles)
- }
childArgs += ("--class", "org.apache.spark.deploy.PythonRunner")
} else if (args.isR) {
val mainFile = new Path(args.primaryResource).getName
@@ -627,7 +616,8 @@ object SparkSubmit {
"spark.jars",
"spark.files",
"spark.yarn.dist.files",
- "spark.yarn.dist.archives")
+ "spark.yarn.dist.archives",
+ "spark.yarn.dist.jars")
pathConfigs.foreach { config =>
// Replace old URIs with resolved URIs, if they exist
sysProps.get(config).foreach { oldValue =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
index e584952a9a..94506a0cbb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
@@ -33,7 +33,8 @@ private[spark] trait AppClientListener {
/** An application death is an unrecoverable failure condition. */
def dead(reason: String): Unit
- def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int)
+ def executorAdded(
+ fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
index 70f21fbe0d..52e2854961 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
@@ -32,8 +32,8 @@ trait LeaderElectionAgent {
@DeveloperApi
trait LeaderElectable {
- def electedLeader()
- def revokedLeadership()
+ def electedLeader(): Unit
+ def revokedLeadership(): Unit
}
/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 01901bbf85..b443e8f051 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -217,7 +217,7 @@ private[deploy] class Master(
}
override def receive: PartialFunction[Any, Unit] = {
- case ElectedLeader => {
+ case ElectedLeader =>
val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData(rpcEnv)
state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) {
RecoveryState.ALIVE
@@ -233,16 +233,14 @@ private[deploy] class Master(
}
}, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS)
}
- }
case CompleteRecovery => completeRecovery()
- case RevokedLeadership => {
+ case RevokedLeadership =>
logError("Leadership has been revoked -- master shutting down.")
System.exit(0)
- }
- case RegisterApplication(description, driver) => {
+ case RegisterApplication(description, driver) =>
// TODO Prevent repeated registrations from some driver
if (state == RecoveryState.STANDBY) {
// ignore, don't send response
@@ -255,12 +253,11 @@ private[deploy] class Master(
driver.send(RegisteredApplication(app.id, self))
schedule()
}
- }
- case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
+ case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId))
execOption match {
- case Some(exec) => {
+ case Some(exec) =>
val appInfo = idToApp(appId)
val oldState = exec.state
exec.state = state
@@ -298,22 +295,19 @@ private[deploy] class Master(
}
}
}
- }
case None =>
logWarning(s"Got status update for unknown executor $appId/$execId")
}
- }
- case DriverStateChanged(driverId, state, exception) => {
+ case DriverStateChanged(driverId, state, exception) =>
state match {
case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED =>
removeDriver(driverId, state, exception)
case _ =>
throw new Exception(s"Received unexpected state update for driver $driverId: $state")
}
- }
- case Heartbeat(workerId, worker) => {
+ case Heartbeat(workerId, worker) =>
idToWorker.get(workerId) match {
case Some(workerInfo) =>
workerInfo.lastHeartbeat = System.currentTimeMillis()
@@ -327,9 +321,8 @@ private[deploy] class Master(
" This worker was never registered, so ignoring the heartbeat.")
}
}
- }
- case MasterChangeAcknowledged(appId) => {
+ case MasterChangeAcknowledged(appId) =>
idToApp.get(appId) match {
case Some(app) =>
logInfo("Application has been re-registered: " + appId)
@@ -339,9 +332,8 @@ private[deploy] class Master(
}
if (canCompleteRecovery) { completeRecovery() }
- }
- case WorkerSchedulerStateResponse(workerId, executors, driverIds) => {
+ case WorkerSchedulerStateResponse(workerId, executors, driverIds) =>
idToWorker.get(workerId) match {
case Some(worker) =>
logInfo("Worker has been re-registered: " + workerId)
@@ -367,7 +359,6 @@ private[deploy] class Master(
}
if (canCompleteRecovery) { completeRecovery() }
- }
case WorkerLatestState(workerId, executors, driverIds) =>
idToWorker.get(workerId) match {
@@ -397,9 +388,8 @@ private[deploy] class Master(
logInfo(s"Received unregister request from application $applicationId")
idToApp.get(applicationId).foreach(finishApplication)
- case CheckForWorkerTimeOut => {
+ case CheckForWorkerTimeOut =>
timeOutDeadWorkers()
- }
case AttachCompletedRebuildUI(appId) =>
// An asyncRebuildSparkUI has completed, so need to attach to master webUi
@@ -408,7 +398,7 @@ private[deploy] class Master(
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
case RegisterWorker(
- id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl) => {
+ id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl) =>
logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
workerHost, workerPort, cores, Utils.megabytesToString(memory)))
if (state == RecoveryState.STANDBY) {
@@ -430,9 +420,8 @@ private[deploy] class Master(
+ workerAddress))
}
}
- }
- case RequestSubmitDriver(description) => {
+ case RequestSubmitDriver(description) =>
if (state != RecoveryState.ALIVE) {
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
"Can only accept driver submissions in ALIVE state."
@@ -451,9 +440,8 @@ private[deploy] class Master(
context.reply(SubmitDriverResponse(self, true, Some(driver.id),
s"Driver successfully submitted as ${driver.id}"))
}
- }
- case RequestKillDriver(driverId) => {
+ case RequestKillDriver(driverId) =>
if (state != RecoveryState.ALIVE) {
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
s"Can only kill drivers in ALIVE state."
@@ -484,9 +472,8 @@ private[deploy] class Master(
context.reply(KillDriverResponse(self, driverId, success = false, msg))
}
}
- }
- case RequestDriverStatus(driverId) => {
+ case RequestDriverStatus(driverId) =>
if (state != RecoveryState.ALIVE) {
val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " +
"Can only request driver status in ALIVE state."
@@ -501,18 +488,15 @@ private[deploy] class Master(
context.reply(DriverStatusResponse(found = false, None, None, None, None))
}
}
- }
- case RequestMasterState => {
+ case RequestMasterState =>
context.reply(MasterStateResponse(
address.host, address.port, restServerBoundPort,
workers.toArray, apps.toArray, completedApps.toArray,
drivers.toArray, completedDrivers.toArray, state))
- }
- case BoundPortsRequest => {
+ case BoundPortsRequest =>
context.reply(BoundPortsResponse(address.port, webUi.boundPort, restServerBoundPort))
- }
case RequestExecutors(appId, requestedTotal) =>
context.reply(handleRequestExecutors(appId, requestedTotal))
@@ -859,10 +843,10 @@ private[deploy] class Master(
addressToApp -= app.driver.address
if (completedApps.size >= RETAINED_APPLICATIONS) {
val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1)
- completedApps.take(toRemove).foreach( a => {
+ completedApps.take(toRemove).foreach { a =>
Option(appIdToUI.remove(a.id)).foreach { ui => webUi.detachSparkUI(ui) }
applicationMetricsSystem.removeSource(a.appSource)
- })
+ }
completedApps.trimStart(toRemove)
}
completedApps += app // Remember it in our history
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 9cd7458ba0..585e0839d0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -78,7 +78,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
case ("--help") :: tail =>
printUsageAndExit(0)
- case Nil => {}
+ case Nil => // No-op
case _ =>
printUsageAndExit(1)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
index dddf2be57e..b30bc821b7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
@@ -40,12 +40,12 @@ abstract class PersistenceEngine {
* Defines how the object is serialized and persisted. Implementation will
* depend on the store used.
*/
- def persist(name: String, obj: Object)
+ def persist(name: String, obj: Object): Unit
/**
* Defines how the object referred by its name is removed from the store.
*/
- def unpersist(name: String)
+ def unpersist(name: String): Unit
/**
* Gives all objects, matching a prefix. This defines how objects are
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 79f77212fe..af850e4871 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -70,11 +70,10 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer
try {
Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData)))
} catch {
- case e: Exception => {
+ case e: Exception =>
logWarning("Exception while reading persisted file, deleting", e)
zk.delete().forPath(WORKING_DIR + "/" + filename)
None
- }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
index b97805a28b..11e13441ee 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
@@ -76,14 +76,13 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
case ("--help") :: tail =>
printUsageAndExit(0)
- case Nil => {
+ case Nil =>
if (masterUrl == null) {
// scalastyle:off println
System.err.println("--master is required")
// scalastyle:on println
printUsageAndExit(1)
}
- }
case _ =>
printUsageAndExit(1)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index a4efafcb27..cba4aaffe2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -29,7 +29,7 @@ import org.apache.spark.launcher.WorkerCommandBuilder
import org.apache.spark.util.Utils
/**
- ** Utilities for running commands with the spark classpath.
+ * Utilities for running commands with the spark classpath.
*/
private[deploy]
object CommandUtils extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 9c6bc5c62f..aad2e91b25 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -218,7 +218,7 @@ private[deploy] class DriverRunner(
}
private[deploy] trait Sleeper {
- def sleep(seconds: Int)
+ def sleep(seconds: Int): Unit
}
// Needed because ProcessBuilder is a final class and cannot be mocked
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index f9c92c3bb9..06066248ea 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -179,16 +179,14 @@ private[deploy] class ExecutorRunner(
val message = "Command exited with code " + exitCode
worker.send(ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)))
} catch {
- case interrupted: InterruptedException => {
+ case interrupted: InterruptedException =>
logInfo("Runner thread for executor " + fullId + " interrupted")
state = ExecutorState.KILLED
killProcess(None)
- }
- case e: Exception => {
+ case e: Exception =>
logError("Error running executor", e)
state = ExecutorState.FAILED
killProcess(Some(e.toString))
- }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 1b7637a39c..449beb0811 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -480,7 +480,7 @@ private[deploy] class Worker(
memoryUsed += memory_
sendToMaster(ExecutorStateChanged(appId, execId, manager.state, None, None))
} catch {
- case e: Exception => {
+ case e: Exception =>
logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e)
if (executors.contains(appId + "/" + execId)) {
executors(appId + "/" + execId).kill()
@@ -488,7 +488,6 @@ private[deploy] class Worker(
}
sendToMaster(ExecutorStateChanged(appId, execId, ExecutorState.FAILED,
Some(e.toString), None))
- }
}
}
@@ -509,7 +508,7 @@ private[deploy] class Worker(
}
}
- case LaunchDriver(driverId, driverDesc) => {
+ case LaunchDriver(driverId, driverDesc) =>
logInfo(s"Asked to launch driver $driverId")
val driver = new DriverRunner(
conf,
@@ -525,9 +524,8 @@ private[deploy] class Worker(
coresUsed += driverDesc.cores
memoryUsed += driverDesc.mem
- }
- case KillDriver(driverId) => {
+ case KillDriver(driverId) =>
logInfo(s"Asked to kill driver $driverId")
drivers.get(driverId) match {
case Some(runner) =>
@@ -535,11 +533,9 @@ private[deploy] class Worker(
case None =>
logError(s"Asked to kill unknown driver $driverId")
}
- }
- case driverStateChanged @ DriverStateChanged(driverId, state, exception) => {
+ case driverStateChanged @ DriverStateChanged(driverId, state, exception) =>
handleDriverStateChanged(driverStateChanged)
- }
case ReregisterWithMaster =>
reregisterWithMaster()
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 391eb41190..777020d4d5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -165,12 +165,11 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
}
// scalastyle:on classforname
} catch {
- case e: Exception => {
+ case e: Exception =>
totalMb = 2*1024
// scalastyle:off println
System.out.println("Failed to get total physical memory. Using " + totalMb + " MB")
// scalastyle:on println
- }
}
// Leave out 1 GB for the operating system, but don't return a negative memory size
math.max(totalMb - 1024, Utils.DEFAULT_DRIVER_MEM_MB)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
index 6500cab73b..e75c0cec4a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala
@@ -107,20 +107,18 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with
}
val content =
- <html>
- <body>
- {linkToMaster}
- <div>
- <div style="float:left; margin-right:10px">{backButton}</div>
- <div style="float:left;">{range}</div>
- <div style="float:right; margin-left:10px">{nextButton}</div>
- </div>
- <br />
- <div style="height:500px; overflow:auto; padding:5px;">
- <pre>{logText}</pre>
- </div>
- </body>
- </html>
+ <div>
+ {linkToMaster}
+ <div>
+ <div style="float:left; margin-right:10px">{backButton}</div>
+ <div style="float:left;">{range}</div>
+ <div style="float:right; margin-left:10px">{nextButton}</div>
+ </div>
+ <br />
+ <div style="height:500px; overflow:auto; padding:5px;">
+ <pre>{logText}</pre>
+ </div>
+ </div>
UIUtils.basicSparkPage(content, logType + " log page for " + pageName)
}
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 320a20033d..71b4ad160d 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -57,16 +57,14 @@ private[spark] class CoarseGrainedExecutorBackend(
rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref =>
// This is a very fast action so we can use "ThreadUtils.sameThread"
driver = Some(ref)
- ref.ask[RegisterExecutorResponse](RegisterExecutor(executorId, self, cores, extractLogUrls))
+ ref.ask[Boolean](RegisterExecutor(executorId, self, cores, extractLogUrls))
}(ThreadUtils.sameThread).onComplete {
// This is a very fast action so we can use "ThreadUtils.sameThread"
- case Success(msg) => Utils.tryLogNonFatalError {
- Option(self).foreach(_.send(msg)) // msg must be RegisterExecutorResponse
- }
- case Failure(e) => {
+ case Success(msg) =>
+ // Always receive `true`. Just ignore it
+ case Failure(e) =>
logError(s"Cannot register with driver: $driverUrl", e)
System.exit(1)
- }
}(ThreadUtils.sameThread)
}
@@ -113,9 +111,15 @@ private[spark] class CoarseGrainedExecutorBackend(
case Shutdown =>
stopping.set(true)
- executor.stop()
- stop()
- rpcEnv.shutdown()
+ new Thread("CoarseGrainedExecutorBackend-stop-executor") {
+ override def run(): Unit = {
+ // executor.stop() will call `SparkEnv.stop()` which waits until RpcEnv stops totally.
+ // However, if `executor.stop()` runs in some thread of RpcEnv, RpcEnv won't be able to
+ // stop until `executor.stop()` returns, which becomes a dead-lock (See SPARK-14180).
+ // Therefore, we put this line in a new thread.
+ executor.stop()
+ }
+ }.start()
}
override def onDisconnected(remoteAddress: RpcAddress): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index 3201463b8c..9f94fdef24 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -21,6 +21,7 @@ import java.io.{File, NotSerializableException}
import java.lang.management.ManagementFactory
import java.net.URL
import java.nio.ByteBuffer
+import java.util.Properties
import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
import scala.collection.JavaConverters._
@@ -206,9 +207,16 @@ private[spark] class Executor(
startGCTime = computeTotalGcTime()
try {
- val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask)
+ val (taskFiles, taskJars, taskProps, taskBytes) =
+ Task.deserializeWithDependencies(serializedTask)
+
+ // Must be set before updateDependencies() is called, in case fetching dependencies
+ // requires access to properties contained within (e.g. for access control).
+ Executor.taskDeserializationProps.set(taskProps)
+
updateDependencies(taskFiles, taskJars)
task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
+ task.localProperties = taskProps
task.setTaskMemoryManager(taskMemoryManager)
// If this task has been killed before we deserialized it, let's quit now. Otherwise,
@@ -254,7 +262,7 @@ private[spark] class Executor(
if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) {
throw new SparkException(errMsg)
} else {
- logError(errMsg)
+ logWarning(errMsg)
}
}
}
@@ -321,7 +329,7 @@ private[spark] class Executor(
logInfo(s"Executor killed $taskName (TID $taskId)")
execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
- case cDE: CommitDeniedException =>
+ case CausedBy(cDE: CommitDeniedException) =>
val reason = cDE.toTaskEndReason
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
@@ -506,3 +514,10 @@ private[spark] class Executor(
heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS)
}
}
+
+private[spark] object Executor {
+ // This is reserved for internal use by components that need to read task properties before a
+ // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be
+ // used instead.
+ val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties]
+}
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
index e07cb31cbe..7153323d01 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
@@ -25,6 +25,6 @@ import org.apache.spark.TaskState.TaskState
* A pluggable interface used by the Executor to send updates to the cluster scheduler.
*/
private[spark] trait ExecutorBackend {
- def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer)
+ def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala
index 6d30d3c76a..83e11c5e23 100644
--- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala
@@ -81,35 +81,9 @@ class InputMetrics private (
*/
def readMethod: DataReadMethod.Value = DataReadMethod.withName(_readMethod.localValue)
- // Once incBytesRead & intRecordsRead is ready to be removed from the public API
- // we can remove the internal versions and make the previous public API private.
- // This has been done to suppress warnings when building.
- @deprecated("incrementing input metrics is for internal use only", "2.0.0")
- def incBytesRead(v: Long): Unit = _bytesRead.add(v)
- private[spark] def incBytesReadInternal(v: Long): Unit = _bytesRead.add(v)
- @deprecated("incrementing input metrics is for internal use only", "2.0.0")
- def incRecordsRead(v: Long): Unit = _recordsRead.add(v)
- private[spark] def incRecordsReadInternal(v: Long): Unit = _recordsRead.add(v)
+ private[spark] def incBytesRead(v: Long): Unit = _bytesRead.add(v)
+ private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v)
private[spark] def setBytesRead(v: Long): Unit = _bytesRead.setValue(v)
- private[spark] def setReadMethod(v: DataReadMethod.Value): Unit =
- _readMethod.setValue(v.toString)
+ private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = _readMethod.setValue(v.toString)
}
-
-/**
- * Deprecated methods to preserve case class matching behavior before Spark 2.0.
- */
-object InputMetrics {
-
- @deprecated("matching on InputMetrics will not be supported in the future", "2.0.0")
- def apply(readMethod: DataReadMethod.Value): InputMetrics = {
- val im = new InputMetrics
- im.setReadMethod(readMethod)
- im
- }
-
- @deprecated("matching on InputMetrics will not be supported in the future", "2.0.0")
- def unapply(input: InputMetrics): Option[DataReadMethod.Value] = {
- Some(input.readMethod)
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala
index 0b37d559c7..93f953846f 100644
--- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala
@@ -52,18 +52,6 @@ class OutputMetrics private (
}
/**
- * Create a new [[OutputMetrics]] that is not associated with any particular task.
- *
- * This is only used for preserving matching behavior on [[OutputMetrics]], which used to be
- * a case class before Spark 2.0. Once we remove support for matching on [[OutputMetrics]]
- * we can remove this constructor as well.
- */
- private[executor] def this() {
- this(InternalAccumulator.createOutputAccums()
- .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]])
- }
-
- /**
* Total number of bytes written.
*/
def bytesWritten: Long = _bytesWritten.localValue
@@ -84,21 +72,3 @@ class OutputMetrics private (
_writeMethod.setValue(v.toString)
}
-
-/**
- * Deprecated methods to preserve case class matching behavior before Spark 2.0.
- */
-object OutputMetrics {
-
- @deprecated("matching on OutputMetrics will not be supported in the future", "2.0.0")
- def apply(writeMethod: DataWriteMethod.Value): OutputMetrics = {
- val om = new OutputMetrics
- om.setWriteMethod(writeMethod)
- om
- }
-
- @deprecated("matching on OutputMetrics will not be supported in the future", "2.0.0")
- def unapply(output: OutputMetrics): Option[DataWriteMethod.Value] = {
- Some(output.writeMethod)
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala
index 50bb645d97..71a24770b5 100644
--- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala
@@ -116,4 +116,25 @@ class ShuffleReadMetrics private (
private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v)
private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v)
+ /**
+ * Resets the value of the current metrics (`this`) and and merges all the independent
+ * [[ShuffleReadMetrics]] into `this`.
+ */
+ private[spark] def setMergeValues(metrics: Seq[ShuffleReadMetrics]): Unit = {
+ _remoteBlocksFetched.setValue(_remoteBlocksFetched.zero)
+ _localBlocksFetched.setValue(_localBlocksFetched.zero)
+ _remoteBytesRead.setValue(_remoteBytesRead.zero)
+ _localBytesRead.setValue(_localBytesRead.zero)
+ _fetchWaitTime.setValue(_fetchWaitTime.zero)
+ _recordsRead.setValue(_recordsRead.zero)
+ metrics.foreach { metric =>
+ _remoteBlocksFetched.add(metric.remoteBlocksFetched)
+ _localBlocksFetched.add(metric.localBlocksFetched)
+ _remoteBytesRead.add(metric.remoteBytesRead)
+ _localBytesRead.add(metric.localBytesRead)
+ _fetchWaitTime.add(metric.fetchWaitTime)
+ _recordsRead.add(metric.recordsRead)
+ }
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index 02219a84ab..bda2a91d9d 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -139,16 +139,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se
*/
def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue
- @deprecated("use updatedBlockStatuses instead", "2.0.0")
- def updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = {
- if (updatedBlockStatuses.nonEmpty) Some(updatedBlockStatuses) else None
- }
-
- @deprecated("setting updated blocks is not allowed", "2.0.0")
- def updatedBlocks_=(blocks: Option[Seq[(BlockId, BlockStatus)]]): Unit = {
- blocks.foreach(setUpdatedBlockStatuses)
- }
-
// Setters and increment-ers
private[spark] def setExecutorDeserializeTime(v: Long): Unit =
_executorDeserializeTime.setValue(v)
@@ -225,11 +215,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se
*/
def outputMetrics: Option[OutputMetrics] = _outputMetrics
- @deprecated("setting OutputMetrics is for internal use only", "2.0.0")
- def outputMetrics_=(om: Option[OutputMetrics]): Unit = {
- _outputMetrics = om
- }
-
/**
* Get or create a new [[OutputMetrics]] associated with this task.
*/
@@ -285,12 +270,7 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se
private[spark] def mergeShuffleReadMetrics(): Unit = synchronized {
if (tempShuffleReadMetrics.nonEmpty) {
val metrics = new ShuffleReadMetrics(initialAccumsMap)
- metrics.setRemoteBlocksFetched(tempShuffleReadMetrics.map(_.remoteBlocksFetched).sum)
- metrics.setLocalBlocksFetched(tempShuffleReadMetrics.map(_.localBlocksFetched).sum)
- metrics.setFetchWaitTime(tempShuffleReadMetrics.map(_.fetchWaitTime).sum)
- metrics.setRemoteBytesRead(tempShuffleReadMetrics.map(_.remoteBytesRead).sum)
- metrics.setLocalBytesRead(tempShuffleReadMetrics.map(_.localBytesRead).sum)
- metrics.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum)
+ metrics.setMergeValues(tempShuffleReadMetrics)
_shuffleReadMetrics = Some(metrics)
}
}
@@ -306,11 +286,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se
*/
def shuffleWriteMetrics: Option[ShuffleWriteMetrics] = _shuffleWriteMetrics
- @deprecated("setting ShuffleWriteMetrics is for internal use only", "2.0.0")
- def shuffleWriteMetrics_=(swm: Option[ShuffleWriteMetrics]): Unit = {
- _shuffleWriteMetrics = swm
- }
-
/**
* Get or create a new [[ShuffleWriteMetrics]] associated with this task.
*/
diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
index 770b43697a..5d50e3851a 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala
@@ -85,10 +85,12 @@ private[spark] class TypedConfigBuilder[T](
this(parent, converter, Option(_).map(_.toString).orNull)
}
+ /** Apply a transformation to the user-provided values of the config entry. */
def transform(fn: T => T): TypedConfigBuilder[T] = {
new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter)
}
+ /** Check that user-provided values for the config match a pre-defined set. */
def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = {
transform { v =>
if (!validValues.contains(v)) {
@@ -99,30 +101,38 @@ private[spark] class TypedConfigBuilder[T](
}
}
+ /** Turns the config entry into a sequence of values of the underlying type. */
def toSequence: TypedConfigBuilder[Seq[T]] = {
new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter))
}
- /** Creates a [[ConfigEntry]] that does not require a default value. */
- def optional: OptionalConfigEntry[T] = {
- new OptionalConfigEntry[T](parent.key, converter, stringConverter, parent._doc, parent._public)
+ /** Creates a [[ConfigEntry]] that does not have a default value. */
+ def createOptional: OptionalConfigEntry[T] = {
+ val entry = new OptionalConfigEntry[T](parent.key, converter, stringConverter, parent._doc,
+ parent._public)
+ parent._onCreate.foreach(_(entry))
+ entry
}
/** Creates a [[ConfigEntry]] that has a default value. */
- def withDefault(default: T): ConfigEntry[T] = {
+ def createWithDefault(default: T): ConfigEntry[T] = {
val transformedDefault = converter(stringConverter(default))
- new ConfigEntryWithDefault[T](parent.key, transformedDefault, converter, stringConverter,
- parent._doc, parent._public)
+ val entry = new ConfigEntryWithDefault[T](parent.key, transformedDefault, converter,
+ stringConverter, parent._doc, parent._public)
+ parent._onCreate.foreach(_(entry))
+ entry
}
/**
* Creates a [[ConfigEntry]] that has a default value. The default value is provided as a
* [[String]] and must be a valid value for the entry.
*/
- def withDefaultString(default: String): ConfigEntry[T] = {
+ def createWithDefaultString(default: String): ConfigEntry[T] = {
val typedDefault = converter(default)
- new ConfigEntryWithDefault[T](parent.key, typedDefault, converter, stringConverter, parent._doc,
- parent._public)
+ val entry = new ConfigEntryWithDefault[T](parent.key, typedDefault, converter, stringConverter,
+ parent._doc, parent._public)
+ parent._onCreate.foreach(_(entry))
+ entry
}
}
@@ -136,10 +146,11 @@ private[spark] case class ConfigBuilder(key: String) {
import ConfigHelpers._
- var _public = true
- var _doc = ""
+ private[config] var _public = true
+ private[config] var _doc = ""
+ private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None
- def internal: ConfigBuilder = {
+ def internal(): ConfigBuilder = {
_public = false
this
}
@@ -149,6 +160,15 @@ private[spark] case class ConfigBuilder(key: String) {
this
}
+ /**
+ * Registers a callback for when the config entry is finally instantiated. Currently used by
+ * SQLConf to keep track of SQL configuration entries.
+ */
+ def onCreate(callback: ConfigEntry[_] => Unit): ConfigBuilder = {
+ _onCreate = Option(callback)
+ this
+ }
+
def intConf: TypedConfigBuilder[Int] = {
new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int"))
}
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index f2f20b3207..94b50ee065 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -18,59 +18,75 @@
package org.apache.spark.internal
import org.apache.spark.launcher.SparkLauncher
+import org.apache.spark.network.util.ByteUnit
package object config {
private[spark] val DRIVER_CLASS_PATH =
- ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.optional
+ ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional
private[spark] val DRIVER_JAVA_OPTIONS =
- ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.optional
+ ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.createOptional
private[spark] val DRIVER_LIBRARY_PATH =
- ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.optional
+ ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional
private[spark] val DRIVER_USER_CLASS_PATH_FIRST =
- ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.withDefault(false)
+ ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false)
+
+ private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory")
+ .bytesConf(ByteUnit.MiB)
+ .createWithDefaultString("1g")
private[spark] val EXECUTOR_CLASS_PATH =
- ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.optional
+ ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional
private[spark] val EXECUTOR_JAVA_OPTIONS =
- ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.optional
+ ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.createOptional
private[spark] val EXECUTOR_LIBRARY_PATH =
- ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.optional
+ ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional
private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST =
- ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.withDefault(false)
+ ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false)
+
+ private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory")
+ .bytesConf(ByteUnit.MiB)
+ .createWithDefaultString("1g")
- private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal
- .booleanConf.withDefault(false)
+ private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal()
+ .booleanConf.createWithDefault(false)
- private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.withDefault(1)
+ private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1)
private[spark] val DYN_ALLOCATION_MIN_EXECUTORS =
- ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.withDefault(0)
+ ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.createWithDefault(0)
private[spark] val DYN_ALLOCATION_INITIAL_EXECUTORS =
ConfigBuilder("spark.dynamicAllocation.initialExecutors")
.fallbackConf(DYN_ALLOCATION_MIN_EXECUTORS)
private[spark] val DYN_ALLOCATION_MAX_EXECUTORS =
- ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.withDefault(Int.MaxValue)
+ ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.createWithDefault(Int.MaxValue)
private[spark] val SHUFFLE_SERVICE_ENABLED =
- ConfigBuilder("spark.shuffle.service.enabled").booleanConf.withDefault(false)
+ ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false)
private[spark] val KEYTAB = ConfigBuilder("spark.yarn.keytab")
.doc("Location of user's keytab.")
- .stringConf.optional
+ .stringConf.createOptional
private[spark] val PRINCIPAL = ConfigBuilder("spark.yarn.principal")
.doc("Name of the Kerberos principal.")
- .stringConf.optional
+ .stringConf.createOptional
- private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances").intConf.optional
+ private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances")
+ .intConf
+ .createOptional
+ private[spark] val PY_FILES = ConfigBuilder("spark.submit.pyFiles")
+ .internal()
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
}
diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
index a2add61617..31b9c5edf0 100644
--- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala
@@ -37,7 +37,6 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm
override def buildCommand(env: JMap[String, String]): JList[String] = {
val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator))
- cmd.add(s"-Xms${memoryMb}M")
cmd.add(s"-Xmx${memoryMb}M")
command.javaOpts.foreach(cmd.add)
CommandBuilderUtils.addPermGenSizeOpt(cmd)
diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
index 891facba33..607283a306 100644
--- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
+++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
@@ -33,11 +33,8 @@ object SparkHadoopMapRedUtil extends Logging {
* the driver in order to determine whether this attempt can commit (please see SPARK-4879 for
* details).
*
- * Output commit coordinator is only contacted when the following two configurations are both set
- * to `true`:
- *
- * - `spark.speculation`
- * - `spark.hadoop.outputCommitCoordination.enabled`
+ * Output commit coordinator is only used when `spark.hadoop.outputCommitCoordination.enabled`
+ * is set to true (which is the default).
*/
def commitTask(
committer: MapReduceOutputCommitter,
@@ -64,11 +61,10 @@ object SparkHadoopMapRedUtil extends Logging {
if (committer.needsTaskCommit(mrTaskContext)) {
val shouldCoordinateWithDriver: Boolean = {
val sparkConf = SparkEnv.get.conf
- // We only need to coordinate with the driver if there are multiple concurrent task
- // attempts, which should only occur if speculation is enabled
- val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false)
- // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs
- sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled)
+ // We only need to coordinate with the driver if there are concurrent task attempts.
+ // Note that this could happen even when speculation is not enabled (e.g. see SPARK-8029).
+ // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs.
+ sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", defaultValue = true)
}
if (shouldCoordinateWithDriver) {
diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
index 10656bc8c8..0210217e41 100644
--- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
@@ -23,6 +23,7 @@ import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.storage.BlockId
import org.apache.spark.storage.memory.MemoryStore
+import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.unsafe.memory.MemoryAllocator
@@ -190,6 +191,8 @@ private[spark] abstract class MemoryManager(
if (conf.getBoolean("spark.memory.offHeap.enabled", false)) {
require(conf.getSizeAsBytes("spark.memory.offHeap.size", 0) > 0,
"spark.memory.offHeap.size must be > 0 when spark.memory.offHeap.enabled == true")
+ require(Platform.unaligned(),
+ "No support for unaligned Unsafe. Set spark.memory.offHeap.enabled to false.")
MemoryMode.OFF_HEAP
} else {
MemoryMode.ON_HEAP
diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala
index a67e8da26b..0b552cabfc 100644
--- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala
+++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala
@@ -35,6 +35,11 @@ private[memory] class StorageMemoryPool(
memoryMode: MemoryMode
) extends MemoryPool(lock) with Logging {
+ private[this] val poolName: String = memoryMode match {
+ case MemoryMode.ON_HEAP => "on-heap storage"
+ case MemoryMode.OFF_HEAP => "off-heap storage"
+ }
+
@GuardedBy("lock")
private[this] var _memoryUsed: Long = 0L
@@ -60,7 +65,7 @@ private[memory] class StorageMemoryPool(
/**
* Acquire N bytes of memory to cache the given block, evicting existing ones if necessary.
- *
+ *
* @return whether all N bytes were successfully granted.
*/
def acquireMemory(blockId: BlockId, numBytes: Long): Boolean = lock.synchronized {
@@ -83,9 +88,8 @@ private[memory] class StorageMemoryPool(
assert(numBytesToAcquire >= 0)
assert(numBytesToFree >= 0)
assert(memoryUsed <= poolSize)
- // Once we support off-heap caching, this will need to change:
- if (numBytesToFree > 0 && memoryMode == MemoryMode.ON_HEAP) {
- memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree)
+ if (numBytesToFree > 0) {
+ memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, memoryMode)
}
// NOTE: If the memory store evicts blocks, then those evictions will synchronously call
// back into this StorageMemoryPool in order to free memory. Therefore, these variables
@@ -122,14 +126,8 @@ private[memory] class StorageMemoryPool(
val remainingSpaceToFree = spaceToFree - spaceFreedByReleasingUnusedMemory
if (remainingSpaceToFree > 0) {
// If reclaiming free memory did not adequately shrink the pool, begin evicting blocks:
- val spaceFreedByEviction = {
- // Once we support off-heap caching, this will need to change:
- if (memoryMode == MemoryMode.ON_HEAP) {
- memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree)
- } else {
- 0
- }
- }
+ val spaceFreedByEviction =
+ memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree, memoryMode)
// When a block is released, BlockManager.dropFromMemory() calls releaseMemory(), so we do
// not need to decrement _memoryUsed here. However, we do need to decrement the pool size.
decrementPoolSize(spaceFreedByEviction)
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 4da1017d28..0fed991049 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -196,10 +196,9 @@ private[spark] class MetricsSystem private (
sinks += sink.asInstanceOf[Sink]
}
} catch {
- case e: Exception => {
+ case e: Exception =>
logError("Sink class " + classPath + " cannot be instantiated")
throw e
- }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
index e43e3a2de2..09ce012e4e 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
@@ -36,7 +36,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo
* Initialize the transfer service by giving it the BlockDataManager that can be used to fetch
* local blocks or put local blocks.
*/
- def init(blockDataManager: BlockDataManager)
+ def init(blockDataManager: BlockDataManager): Unit
/**
* Tear down the transfer service.
diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
index 5f3d4532dd..33a3219607 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
@@ -39,7 +39,11 @@ import org.apache.spark.util.Utils
/**
* A BlockTransferService that uses Netty to fetch a set of blocks at at time.
*/
-class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManager, numCores: Int)
+private[spark] class NettyBlockTransferService(
+ conf: SparkConf,
+ securityManager: SecurityManager,
+ override val hostName: String,
+ numCores: Int)
extends BlockTransferService {
// TODO: Don't use Java serialization, use a more cross-version compatible serialization format.
@@ -65,13 +69,13 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage
clientFactory = transportContext.createClientFactory(clientBootstrap.toSeq.asJava)
server = createServer(serverBootstrap.toList)
appId = conf.getAppId
- logInfo("Server created on " + server.getPort)
+ logInfo(s"Server created on ${hostName}:${server.getPort}")
}
/** Creates and binds the TransportServer, possibly trying multiple ports. */
private def createServer(bootstraps: List[TransportServerBootstrap]): TransportServer = {
def startService(port: Int): (TransportServer, Int) = {
- val server = transportContext.createServer(port, bootstraps.asJava)
+ val server = transportContext.createServer(hostName, port, bootstraps.asJava)
(server, server.getPort)
}
@@ -109,8 +113,6 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage
}
}
- override def hostName: String = Utils.localHostName()
-
override def port: Int = server.getPort
override def uploadBlock(
diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
index 48b9434153..ab6aba6fc7 100644
--- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
+++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
@@ -21,5 +21,22 @@ package org.apache.spark.partial
* A Double value with error bars and associated confidence.
*/
class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) {
+
override def toString(): String = "[%.3f, %.3f]".format(low, high)
+
+ override def hashCode: Int =
+ this.mean.hashCode ^ this.confidence.hashCode ^ this.low.hashCode ^ this.high.hashCode
+
+ /**
+ * Note that consistent with Double, any NaN value will make equality false
+ */
+ override def equals(that: Any): Boolean =
+ that match {
+ case that: BoundedDouble =>
+ this.mean == that.mean &&
+ this.confidence == that.confidence &&
+ this.low == that.low &&
+ this.high == that.high
+ case _ => false
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
index 44295e5a1a..5fe3358316 100644
--- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
@@ -29,8 +29,9 @@ import org.apache.spark.util.StatCounter
private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[StatCounter, BoundedDouble] {
+ // modified in merge
var outputsMerged = 0
- var counter = new StatCounter
+ val counter = new StatCounter
override def merge(outputId: Int, taskResult: StatCounter) {
outputsMerged += 1
@@ -40,30 +41,39 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double)
override def currentResult(): BoundedDouble = {
if (outputsMerged == totalOutputs) {
new BoundedDouble(counter.sum, 1.0, counter.sum, counter.sum)
- } else if (outputsMerged == 0) {
+ } else if (outputsMerged == 0 || counter.count == 0) {
new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity)
} else {
val p = outputsMerged.toDouble / totalOutputs
val meanEstimate = counter.mean
- val meanVar = counter.sampleVariance / counter.count
val countEstimate = (counter.count + 1 - p) / p
- val countVar = (counter.count + 1) * (1 - p) / (p * p)
val sumEstimate = meanEstimate * countEstimate
- val sumVar = (meanEstimate * meanEstimate * countVar) +
- (countEstimate * countEstimate * meanVar) +
- (meanVar * countVar)
- val sumStdev = math.sqrt(sumVar)
- val confFactor = {
- if (counter.count > 100) {
+
+ val meanVar = counter.sampleVariance / counter.count
+
+ // branch at this point because counter.count == 1 implies counter.sampleVariance == Nan
+ // and we don't want to ever return a bound of NaN
+ if (meanVar.isNaN || counter.count == 1) {
+ new BoundedDouble(sumEstimate, confidence, Double.NegativeInfinity, Double.PositiveInfinity)
+ } else {
+ val countVar = (counter.count + 1) * (1 - p) / (p * p)
+ val sumVar = (meanEstimate * meanEstimate * countVar) +
+ (countEstimate * countEstimate * meanVar) +
+ (meanVar * countVar)
+ val sumStdev = math.sqrt(sumVar)
+ val confFactor = if (counter.count > 100) {
new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2)
} else {
+ // note that if this goes to 0, TDistribution will throw an exception.
+ // Hence special casing 1 above.
val degreesOfFreedom = (counter.count - 1).toInt
new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2)
}
+
+ val low = sumEstimate - confFactor * sumStdev
+ val high = sumEstimate + confFactor * sumStdev
+ new BoundedDouble(sumEstimate, confidence, low, high)
}
- val low = sumEstimate - confFactor * sumStdev
- val high = sumEstimate + confFactor * sumStdev
- new BoundedDouble(sumEstimate, confidence, low, high)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index 8358244987..63d1d1767a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -35,9 +35,9 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[Blo
override def getPartitions: Array[Partition] = {
assertValid()
- (0 until blockIds.length).map(i => {
+ (0 until blockIds.length).map { i =>
new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
- }).toArray
+ }.toArray
}
override def compute(split: Partition, context: TaskContext): Iterator[T] = {
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index e5ebc63082..7bc1eb0436 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -29,10 +29,12 @@ import org.apache.spark.serializer.Serializer
import org.apache.spark.util.collection.{CompactBuffer, ExternalAppendOnlyMap}
import org.apache.spark.util.Utils
-/** The references to rdd and splitIndex are transient because redundant information is stored
- * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from
- * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the
- * task closure. */
+/**
+ * The references to rdd and splitIndex are transient because redundant information is stored
+ * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from
+ * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the
+ * task closure.
+ */
private[spark] case class NarrowCoGroupSplitDep(
@transient rdd: RDD[_],
@transient splitIndex: Int,
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index 5e9230e733..368916a39e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -166,8 +166,8 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
val counters = new Array[Long](buckets.length - 1)
while (iter.hasNext) {
bucketFunction(iter.next()) match {
- case Some(x: Int) => {counters(x) += 1}
- case _ => {}
+ case Some(x: Int) => counters(x) += 1
+ case _ => // No-Op
}
}
Iterator(counters)
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 08db96edd6..35d190b464 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -213,15 +213,13 @@ class HadoopRDD[K, V](
logInfo("Input split: " + split.inputSplit)
val jobConf = getJobConf()
- // TODO: there is a lot of duplicate code between this and NewHadoopRDD and SqlNewHadoopRDD
-
val inputMetrics = context.taskMetrics().registerInputMetrics(DataReadMethod.Hadoop)
val existingBytesRead = inputMetrics.bytesRead
// Sets the thread local variable for the file's name
split.inputSplit.value match {
- case fs: FileSplit => SqlNewHadoopRDDState.setInputFileName(fs.getPath.toString)
- case _ => SqlNewHadoopRDDState.unsetInputFileName()
+ case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString)
+ case _ => InputFileNameHolder.unsetInputFileName()
}
// Find a function that will return the FileSystem bytes read by this thread. Do this before
@@ -261,7 +259,7 @@ class HadoopRDD[K, V](
finished = true
}
if (!finished) {
- inputMetrics.incRecordsReadInternal(1)
+ inputMetrics.incRecordsRead(1)
}
if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) {
updateBytesRead()
@@ -271,7 +269,7 @@ class HadoopRDD[K, V](
override def close() {
if (reader != null) {
- SqlNewHadoopRDDState.unsetInputFileName()
+ InputFileNameHolder.unsetInputFileName()
// Close the reader and release it. Note: it's very important that we don't close the
// reader more than once, since that exposes us to MAPREDUCE-5918 when running against
// Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic
@@ -293,7 +291,7 @@ class HadoopRDD[K, V](
// If we can't get the bytes read from the FS stats, fall back to the split size,
// which may be inaccurate.
try {
- inputMetrics.incBytesReadInternal(split.inputSplit.value.getLength)
+ inputMetrics.incBytesRead(split.inputSplit.value.getLength)
} catch {
case e: java.io.IOException =>
logWarning("Unable to get input size to set InputMetrics for task", e)
@@ -424,7 +422,7 @@ private[spark] object HadoopRDD extends Logging {
private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = {
val out = ListBuffer[String]()
- infos.foreach { loc => {
+ infos.foreach { loc =>
val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get.
getLocation.invoke(loc).asInstanceOf[String]
if (locationStr != "localhost") {
@@ -436,7 +434,7 @@ private[spark] object HadoopRDD extends Logging {
out += new HostTaskLocation(locationStr).toString
}
}
- }}
+ }
out.seq
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDDState.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala
index 3f15fff793..108e9d2558 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDDState.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala
@@ -20,10 +20,10 @@ package org.apache.spark.rdd
import org.apache.spark.unsafe.types.UTF8String
/**
- * State for SqlNewHadoopRDD objects. This is split this way because of the package splits.
- * TODO: Move/Combine this with org.apache.spark.sql.datasources.SqlNewHadoopRDD
+ * This holds file names of the current Spark task. This is used in HadoopRDD,
+ * FileScanRDD and InputFileName function in Spark SQL.
*/
-private[spark] object SqlNewHadoopRDDState {
+private[spark] object InputFileNameHolder {
/**
* The thread variable for the name of the current file being read. This is used by
* the InputFileName function in Spark SQL.
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index fb9606ae38..3ccd616cbf 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -189,7 +189,7 @@ class NewHadoopRDD[K, V](
}
havePair = false
if (!finished) {
- inputMetrics.incRecordsReadInternal(1)
+ inputMetrics.incRecordsRead(1)
}
if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) {
updateBytesRead()
@@ -220,7 +220,7 @@ class NewHadoopRDD[K, V](
// If we can't get the bytes read from the FS stats, fall back to the split size,
// which may be inaccurate.
try {
- inputMetrics.incBytesReadInternal(split.serializableHadoopSplit.value.getLength)
+ inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
} catch {
case e: java.io.IOException =>
logWarning("Unable to get input size to set InputMetrics for task", e)
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 363004e587..a5992022d0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -86,12 +86,11 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper)
val rddToFilter: RDD[P] = self.partitioner match {
- case Some(rp: RangePartitioner[K, V]) => {
+ case Some(rp: RangePartitioner[K, V]) =>
val partitionIndicies = (rp.getPartition(lower), rp.getPartition(upper)) match {
case (l, u) => Math.min(l, u) to Math.max(l, u)
}
PartitionPruningRDD.create(self, partitionIndicies.contains)
- }
case _ =>
self
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 296179b75b..085829af6e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -1111,9 +1111,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
maybeUpdateOutputMetrics(outputMetricsAndBytesWrittenCallback, recordsWritten)
recordsWritten += 1
}
- } {
- writer.close(hadoopContext)
- }
+ }(finallyBlock = writer.close(hadoopContext))
committer.commitTask(hadoopContext)
outputMetricsAndBytesWrittenCallback.foreach { case (om, callback) =>
om.setBytesWritten(callback())
@@ -1200,9 +1198,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
maybeUpdateOutputMetrics(outputMetricsAndBytesWrittenCallback, recordsWritten)
recordsWritten += 1
}
- } {
- writer.close()
- }
+ }(finallyBlock = writer.close())
writer.commit()
outputMetricsAndBytesWrittenCallback.foreach { case (om, callback) =>
om.setBytesWritten(callback())
diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index 582fa93afe..bb84e4af15 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -121,14 +121,14 @@ private object ParallelCollectionRDD {
// Sequences need to be sliced at the same set of index positions for operations
// like RDD.zip() to behave as expected
def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = {
- (0 until numSlices).iterator.map(i => {
+ (0 until numSlices).iterator.map { i =>
val start = ((i * length) / numSlices).toInt
val end = (((i + 1) * length) / numSlices).toInt
(start, end)
- })
+ }
}
seq match {
- case r: Range => {
+ case r: Range =>
positions(r.length, numSlices).zipWithIndex.map({ case ((start, end), index) =>
// If the range is inclusive, use inclusive range for the last slice
if (r.isInclusive && index == numSlices - 1) {
@@ -138,8 +138,7 @@ private object ParallelCollectionRDD {
new Range(r.start + start * r.step, r.start + end * r.step, r.step)
}
}).toSeq.asInstanceOf[Seq[Seq[T]]]
- }
- case nr: NumericRange[_] => {
+ case nr: NumericRange[_] =>
// For ranges of Long, Double, BigInteger, etc
val slices = new ArrayBuffer[Seq[T]](numSlices)
var r = nr
@@ -149,14 +148,12 @@ private object ParallelCollectionRDD {
r = r.drop(sliceSize)
}
slices
- }
- case _ => {
+ case _ =>
val array = seq.toArray // To prevent O(n^2) operations for List etc
positions(array.length, numSlices).map({
case (start, end) =>
array.slice(start, end).toSeq
}).toSeq
- }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
index 9e3880714a..0abba15bec 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
@@ -68,9 +68,9 @@ class PartitionerAwareUnionRDD[T: ClassTag](
override def getPartitions: Array[Partition] = {
val numPartitions = partitioner.get.numPartitions
- (0 until numPartitions).map(index => {
+ (0 until numPartitions).map { index =>
new PartitionerAwareUnionRDDPartition(rdds, index)
- }).toArray
+ }.toArray
}
// Get the location where most of the partitions of parent RDDs are located
@@ -78,11 +78,10 @@ class PartitionerAwareUnionRDD[T: ClassTag](
logDebug("Finding preferred location for " + this + ", partition " + s.index)
val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
val locations = rdds.zip(parentPartitions).flatMap {
- case (rdd, part) => {
+ case (rdd, part) =>
val parentLocations = currPrefLocs(rdd, part)
logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations)
parentLocations
- }
}
val location = if (locations.isEmpty) {
None
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index f96551c793..36ff3bcaae 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -255,8 +255,8 @@ abstract class RDD[T: ClassTag](
}
/**
- * Returns the number of partitions of this RDD.
- */
+ * Returns the number of partitions of this RDD.
+ */
@Since("1.6.0")
final def getNumPartitions: Int = partitions.length
@@ -333,10 +333,10 @@ abstract class RDD[T: ClassTag](
case Left(blockResult) =>
if (readCachedBlock) {
val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod)
- existingMetrics.incBytesReadInternal(blockResult.bytes)
+ existingMetrics.incBytesRead(blockResult.bytes)
new InterruptibleIterator[T](context, blockResult.data.asInstanceOf[Iterator[T]]) {
override def next(): T = {
- existingMetrics.incRecordsReadInternal(1)
+ existingMetrics.incRecordsRead(1)
delegate.next()
}
}
@@ -568,11 +568,7 @@ abstract class RDD[T: ClassTag](
* times (use `.distinct()` to eliminate them).
*/
def union(other: RDD[T]): RDD[T] = withScope {
- if (partitioner.isDefined && other.partitioner == partitioner) {
- new PartitionerAwareUnionRDD(sc, Array(this, other))
- } else {
- new UnionRDD(sc, Array(this, other))
- }
+ sc.union(this, other)
}
/**
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 5cdc91316b..c27aad268d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -950,13 +950,6 @@ class DAGScheduler(
// First figure out the indexes of partition ids to compute.
val partitionsToCompute: Seq[Int] = stage.findMissingPartitions()
- // Create internal accumulators if the stage has no accumulators initialized.
- // Reset internal accumulators only if this stage is not partially submitted
- // Otherwise, we may override existing accumulator values from some tasks
- if (stage.internalAccumulators.isEmpty || stage.numPartitions == partitionsToCompute.size) {
- stage.resetInternalAccumulators()
- }
-
// Use the scheduling pool, job group, description, etc. from an ActiveJob associated
// with this Stage
val properties = jobIdToActiveJob(jobId).properties
@@ -1036,7 +1029,7 @@ class DAGScheduler(
val locs = taskIdToLocations(id)
val part = stage.rdd.partitions(id)
new ShuffleMapTask(stage.id, stage.latestInfo.attemptId,
- taskBinary, part, locs, stage.internalAccumulators)
+ taskBinary, part, locs, stage.latestInfo.internalAccumulators, properties)
}
case stage: ResultStage =>
@@ -1046,7 +1039,7 @@ class DAGScheduler(
val part = stage.rdd.partitions(p)
val locs = taskIdToLocations(id)
new ResultTask(stage.id, stage.latestInfo.attemptId,
- taskBinary, part, locs, id, stage.internalAccumulators)
+ taskBinary, part, locs, id, properties, stage.latestInfo.internalAccumulators)
}
}
} catch {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 0640f26051..a6b032cc00 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -57,11 +57,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
// Since we are not doing canonicalization of path, this can be wrong : like relative vs
// absolute path .. which is fine, this is best case effort to remove duplicates - right ?
override def equals(other: Any): Boolean = other match {
- case that: InputFormatInfo => {
+ case that: InputFormatInfo =>
// not checking config - that should be fine, right ?
this.inputFormatClazz == that.inputFormatClazz &&
this.path == that.path
- }
case _ => false
}
@@ -86,10 +85,9 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
}
}
catch {
- case e: ClassNotFoundException => {
+ case e: ClassNotFoundException =>
throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz +
" cannot be found ?", e)
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
index 50c2b9acd6..e0f7c8f021 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
@@ -23,6 +23,6 @@ package org.apache.spark.scheduler
* job fails (and no further taskSucceeded events will happen).
*/
private[spark] trait JobListener {
- def taskSucceeded(index: Int, result: Any)
- def jobFailed(exception: Exception)
+ def taskSucceeded(index: Int, result: Any): Unit
+ def jobFailed(exception: Exception): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index cd2736e196..db6276f75d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler
import java.io._
import java.nio.ByteBuffer
+import java.util.Properties
import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
@@ -38,6 +39,7 @@ import org.apache.spark.rdd.RDD
* @param locs preferred task execution locations for locality scheduling
* @param outputId index of the task in this job (a job can launch tasks on only a subset of the
* input RDD's partitions).
+ * @param localProperties copy of thread-local properties set by the user on the driver side.
* @param _initialAccums initial set of accumulators to be used in this task for tracking
* internal metrics. Other accumulators will be registered later when
* they are deserialized on the executors.
@@ -49,8 +51,9 @@ private[spark] class ResultTask[T, U](
partition: Partition,
locs: Seq[TaskLocation],
val outputId: Int,
+ localProperties: Properties,
_initialAccums: Seq[Accumulator[_]] = InternalAccumulator.createAll())
- extends Task[U](stageId, stageAttemptId, partition.index, _initialAccums)
+ extends Task[U](stageId, stageAttemptId, partition.index, _initialAccums, localProperties)
with Serializable {
@transient private[this] val preferredLocs: Seq[TaskLocation] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
index 5baebe8c1f..100ed76ecb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -34,9 +34,9 @@ import org.apache.spark.util.Utils
private[spark] trait SchedulableBuilder {
def rootPool: Pool
- def buildPools()
+ def buildPools(): Unit
- def addTaskSetManager(manager: Schedulable, properties: Properties)
+ def addTaskSetManager(manager: Schedulable, properties: Properties): Unit
}
private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index e30964a01b..b7cab7013e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -18,6 +18,7 @@
package org.apache.spark.scheduler
import java.nio.ByteBuffer
+import java.util.Properties
import scala.language.existentials
@@ -42,6 +43,7 @@ import org.apache.spark.shuffle.ShuffleWriter
* @param _initialAccums initial set of accumulators to be used in this task for tracking
* internal metrics. Other accumulators will be registered later when
* they are deserialized on the executors.
+ * @param localProperties copy of thread-local properties set by the user on the driver side.
*/
private[spark] class ShuffleMapTask(
stageId: Int,
@@ -49,13 +51,14 @@ private[spark] class ShuffleMapTask(
taskBinary: Broadcast[Array[Byte]],
partition: Partition,
@transient private var locs: Seq[TaskLocation],
- _initialAccums: Seq[Accumulator[_]])
- extends Task[MapStatus](stageId, stageAttemptId, partition.index, _initialAccums)
+ _initialAccums: Seq[Accumulator[_]],
+ localProperties: Properties)
+ extends Task[MapStatus](stageId, stageAttemptId, partition.index, _initialAccums, localProperties)
with Logging {
/** A constructor used only in test suites. This does not require passing in an RDD. */
def this(partitionId: Int) {
- this(0, 0, null, new Partition { override def index: Int = 0 }, null, null)
+ this(0, 0, null, new Partition { override def index: Int = 0 }, null, null, new Properties)
}
@transient private val preferredLocs: Seq[TaskLocation] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 586173f180..080ea6c33a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -151,275 +151,152 @@ private[spark] trait SparkHistoryListenerFactory {
def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener]
}
+
/**
- * :: DeveloperApi ::
- * Interface for listening to events from the Spark scheduler. Note that this is an internal
- * interface which might change in different Spark releases. Java clients should extend
- * {@link JavaSparkListener}
+ * Interface for listening to events from the Spark scheduler. Most applications should probably
+ * extend SparkListener or SparkFirehoseListener directly, rather than implementing this class.
+ *
+ * Note that this is an internal interface which might change in different Spark releases.
*/
-@DeveloperApi
-trait SparkListener {
+private[spark] trait SparkListenerInterface {
+
/**
* Called when a stage completes successfully or fails, with information on the completed stage.
*/
- def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { }
+ def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit
/**
* Called when a stage is submitted
*/
- def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { }
+ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit
/**
* Called when a task starts
*/
- def onTaskStart(taskStart: SparkListenerTaskStart) { }
+ def onTaskStart(taskStart: SparkListenerTaskStart): Unit
/**
* Called when a task begins remotely fetching its result (will not be called for tasks that do
* not need to fetch the result remotely).
*/
- def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { }
+ def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit
/**
* Called when a task ends
*/
- def onTaskEnd(taskEnd: SparkListenerTaskEnd) { }
+ def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit
/**
* Called when a job starts
*/
- def onJobStart(jobStart: SparkListenerJobStart) { }
+ def onJobStart(jobStart: SparkListenerJobStart): Unit
/**
* Called when a job ends
*/
- def onJobEnd(jobEnd: SparkListenerJobEnd) { }
+ def onJobEnd(jobEnd: SparkListenerJobEnd): Unit
/**
* Called when environment properties have been updated
*/
- def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { }
+ def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit
/**
* Called when a new block manager has joined
*/
- def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { }
+ def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit
/**
* Called when an existing block manager has been removed
*/
- def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { }
+ def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit
/**
* Called when an RDD is manually unpersisted by the application
*/
- def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { }
+ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit
/**
* Called when the application starts
*/
- def onApplicationStart(applicationStart: SparkListenerApplicationStart) { }
+ def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit
/**
* Called when the application ends
*/
- def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { }
+ def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit
/**
* Called when the driver receives task metrics from an executor in a heartbeat.
*/
- def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { }
+ def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit
/**
* Called when the driver registers a new executor.
*/
- def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) { }
+ def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit
/**
* Called when the driver removes an executor.
*/
- def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { }
+ def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit
/**
* Called when the driver receives a block update info.
*/
- def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated) { }
+ def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit
/**
* Called when other events like SQL-specific events are posted.
*/
- def onOtherEvent(event: SparkListenerEvent) { }
+ def onOtherEvent(event: SparkListenerEvent): Unit
}
+
/**
* :: DeveloperApi ::
- * Simple SparkListener that logs a few summary statistics when each stage completes
+ * A default implementation for [[SparkListenerInterface]] that has no-op implementations for
+ * all callbacks.
+ *
+ * Note that this is an internal interface which might change in different Spark releases.
*/
@DeveloperApi
-class StatsReportListener extends SparkListener with Logging {
-
- import org.apache.spark.scheduler.StatsReportListener._
-
- private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]()
-
- override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
- val info = taskEnd.taskInfo
- val metrics = taskEnd.taskMetrics
- if (info != null && metrics != null) {
- taskInfoMetrics += ((info, metrics))
- }
- }
-
- override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
- implicit val sc = stageCompleted
- this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}")
- showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics)
-
- // Shuffle write
- showBytesDistribution("shuffle bytes written:",
- (_, metric) => metric.shuffleWriteMetrics.map(_.bytesWritten), taskInfoMetrics)
-
- // Fetch & I/O
- showMillisDistribution("fetch wait time:",
- (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics)
- showBytesDistribution("remote bytes read:",
- (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics)
- showBytesDistribution("task result size:",
- (_, metric) => Some(metric.resultSize), taskInfoMetrics)
-
- // Runtime breakdown
- val runtimePcts = taskInfoMetrics.map { case (info, metrics) =>
- RuntimePercentage(info.duration, metrics)
- }
- showDistribution("executor (non-fetch) time pct: ",
- Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%")
- showDistribution("fetch wait time pct: ",
- Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%")
- showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%")
- taskInfoMetrics.clear()
- }
-
- private def getStatusDetail(info: StageInfo): String = {
- val failureReason = info.failureReason.map("(" + _ + ")").getOrElse("")
- val timeTaken = info.submissionTime.map(
- x => info.completionTime.getOrElse(System.currentTimeMillis()) - x
- ).getOrElse("-")
-
- s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " +
- s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " +
- s"Took: $timeTaken msec"
- }
+abstract class SparkListener extends SparkListenerInterface {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { }
-}
+ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { }
-private[spark] object StatsReportListener extends Logging {
-
- // For profiling, the extremes are more interesting
- val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100)
- val probabilities = percentiles.map(_ / 100.0)
- val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
-
- def extractDoubleDistribution(
- taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)],
- getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = {
- Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) })
- }
-
- // Is there some way to setup the types that I can get rid of this completely?
- def extractLongDistribution(
- taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)],
- getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = {
- extractDoubleDistribution(
- taskInfoMetrics,
- (info, metric) => { getMetric(info, metric).map(_.toDouble) })
- }
-
- def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) {
- val stats = d.statCounter
- val quantiles = d.getQuantiles(probabilities).map(formatNumber)
- logInfo(heading + stats)
- logInfo(percentilesHeader)
- logInfo("\t" + quantiles.mkString("\t"))
- }
-
- def showDistribution(
- heading: String,
- dOpt: Option[Distribution],
- formatNumber: Double => String) {
- dOpt.foreach { d => showDistribution(heading, d, formatNumber)}
- }
-
- def showDistribution(heading: String, dOpt: Option[Distribution], format: String) {
- def f(d: Double): String = format.format(d)
- showDistribution(heading, dOpt, f _)
- }
-
- def showDistribution(
- heading: String,
- format: String,
- getMetric: (TaskInfo, TaskMetrics) => Option[Double],
- taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
- showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format)
- }
-
- def showBytesDistribution(
- heading: String,
- getMetric: (TaskInfo, TaskMetrics) => Option[Long],
- taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
- showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric))
- }
-
- def showBytesDistribution(heading: String, dOpt: Option[Distribution]) {
- dOpt.foreach { dist => showBytesDistribution(heading, dist) }
- }
-
- def showBytesDistribution(heading: String, dist: Distribution) {
- showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String)
- }
-
- def showMillisDistribution(heading: String, dOpt: Option[Distribution]) {
- showDistribution(heading, dOpt,
- (d => StatsReportListener.millisToString(d.toLong)): Double => String)
- }
-
- def showMillisDistribution(
- heading: String,
- getMetric: (TaskInfo, TaskMetrics) => Option[Long],
- taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
- showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric))
- }
-
- val seconds = 1000L
- val minutes = seconds * 60
- val hours = minutes * 60
+ override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { }
- /**
- * Reformat a time interval in milliseconds to a prettier format for output
- */
- def millisToString(ms: Long): String = {
- val (size, units) =
- if (ms > hours) {
- (ms.toDouble / hours, "hours")
- } else if (ms > minutes) {
- (ms.toDouble / minutes, "min")
- } else if (ms > seconds) {
- (ms.toDouble / seconds, "s")
- } else {
- (ms.toDouble, "ms")
- }
- "%.1f %s".format(size, units)
- }
-}
+ override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit = { }
+
+ override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { }
+
+ override def onJobStart(jobStart: SparkListenerJobStart): Unit = { }
+
+ override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { }
+
+ override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { }
+
+ override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { }
+
+ override def onBlockManagerRemoved(
+ blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { }
+
+ override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = { }
+
+ override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { }
+
+ override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { }
+
+ override def onExecutorMetricsUpdate(
+ executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { }
+
+ override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { }
+
+ override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { }
+
+ override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { }
-private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double)
-
-private object RuntimePercentage {
- def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = {
- val denom = totalTime.toDouble
- val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime)
- val fetch = fetchTime.map(_ / denom)
- val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom
- val other = 1.0 - (exec + fetch.getOrElse(0d))
- RuntimePercentage(exec, fetch, other)
- }
+ override def onOtherEvent(event: SparkListenerEvent): Unit = { }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index 94f0574f0e..471586ac08 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -22,9 +22,12 @@ import org.apache.spark.util.ListenerBus
/**
* A [[SparkListenerEvent]] bus that relays [[SparkListenerEvent]]s to its listeners
*/
-private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkListenerEvent] {
+private[spark] trait SparkListenerBus
+ extends ListenerBus[SparkListenerInterface, SparkListenerEvent] {
- protected override def doPostEvent(listener: SparkListener, event: SparkListenerEvent): Unit = {
+ protected override def doPostEvent(
+ listener: SparkListenerInterface,
+ event: SparkListenerEvent): Unit = {
event match {
case stageSubmitted: SparkListenerStageSubmitted =>
listener.onStageSubmitted(stageSubmitted)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
index 6e9337bb90..bc1431835e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
@@ -49,14 +49,13 @@ class SplitInfo(
// So unless there is identity equality between underlyingSplits, it will always fail even if it
// is pointing to same block.
override def equals(other: Any): Boolean = other match {
- case that: SplitInfo => {
+ case that: SplitInfo =>
this.hostLocation == that.hostLocation &&
this.inputFormatClazz == that.inputFormatClazz &&
this.path == that.path &&
this.length == that.length &&
// other split specific checks (like start for FileSplit)
this.underlyingSplit == that.underlyingSplit
- }
case _ => false
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index a40b700cdd..b6d4e39fe5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -75,22 +75,6 @@ private[scheduler] abstract class Stage(
val name: String = callSite.shortForm
val details: String = callSite.longForm
- private var _internalAccumulators: Seq[Accumulator[_]] = Seq.empty
-
- /** Internal accumulators shared across all tasks in this stage. */
- def internalAccumulators: Seq[Accumulator[_]] = _internalAccumulators
-
- /**
- * Re-initialize the internal accumulators associated with this stage.
- *
- * This is called every time the stage is submitted, *except* when a subset of tasks
- * belonging to this stage has already finished. Otherwise, reinitializing the internal
- * accumulators here again will override partial values from the finished tasks.
- */
- def resetInternalAccumulators(): Unit = {
- _internalAccumulators = InternalAccumulator.create(rdd.sparkContext)
- }
-
/**
* Pointer to the [StageInfo] object for the most recent attempt. This needs to be initialized
* here, before any attempts have actually been created, because the DAGScheduler uses this
@@ -127,7 +111,8 @@ private[scheduler] abstract class Stage(
numPartitionsToCompute: Int,
taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = {
_latestInfo = StageInfo.fromStage(
- this, nextAttemptId, Some(numPartitionsToCompute), taskLocalityPreferences)
+ this, nextAttemptId, Some(numPartitionsToCompute),
+ InternalAccumulator.createAll(rdd.sparkContext), taskLocalityPreferences)
nextAttemptId += 1
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index 24796c1430..0fd58c41cd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler
import scala.collection.mutable.HashMap
+import org.apache.spark.Accumulator
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.storage.RDDInfo
@@ -35,6 +36,7 @@ class StageInfo(
val rddInfos: Seq[RDDInfo],
val parentIds: Seq[Int],
val details: String,
+ val internalAccumulators: Seq[Accumulator[_]] = Seq.empty,
private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) {
/** When this stage was submitted from the DAGScheduler to a TaskScheduler. */
var submissionTime: Option[Long] = None
@@ -42,7 +44,11 @@ class StageInfo(
var completionTime: Option[Long] = None
/** If the stage failed, the reason why. */
var failureReason: Option[String] = None
- /** Terminal values of accumulables updated during this stage. */
+
+ /**
+ * Terminal values of accumulables updated during this stage, including all the user-defined
+ * accumulators.
+ */
val accumulables = HashMap[Long, AccumulableInfo]()
def stageFailed(reason: String) {
@@ -75,6 +81,7 @@ private[spark] object StageInfo {
stage: Stage,
attemptId: Int,
numTasks: Option[Int] = None,
+ internalAccumulators: Seq[Accumulator[_]] = Seq.empty,
taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty
): StageInfo = {
val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd)
@@ -87,6 +94,7 @@ private[spark] object StageInfo {
rddInfos,
stage.parents.map(_.id),
stage.details,
+ internalAccumulators,
taskLocalityPreferences)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
new file mode 100644
index 0000000000..309f4b806b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala
@@ -0,0 +1,199 @@
+/*
+ * 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.scheduler
+
+import scala.collection.mutable
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{Distribution, Utils}
+
+
+/**
+ * :: DeveloperApi ::
+ * Simple SparkListener that logs a few summary statistics when each stage completes.
+ */
+@DeveloperApi
+class StatsReportListener extends SparkListener with Logging {
+
+ import org.apache.spark.scheduler.StatsReportListener._
+
+ private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]()
+
+ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
+ val info = taskEnd.taskInfo
+ val metrics = taskEnd.taskMetrics
+ if (info != null && metrics != null) {
+ taskInfoMetrics += ((info, metrics))
+ }
+ }
+
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
+ implicit val sc = stageCompleted
+ this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}")
+ showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics)
+
+ // Shuffle write
+ showBytesDistribution("shuffle bytes written:",
+ (_, metric) => metric.shuffleWriteMetrics.map(_.bytesWritten), taskInfoMetrics)
+
+ // Fetch & I/O
+ showMillisDistribution("fetch wait time:",
+ (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics)
+ showBytesDistribution("remote bytes read:",
+ (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics)
+ showBytesDistribution("task result size:",
+ (_, metric) => Some(metric.resultSize), taskInfoMetrics)
+
+ // Runtime breakdown
+ val runtimePcts = taskInfoMetrics.map { case (info, metrics) =>
+ RuntimePercentage(info.duration, metrics)
+ }
+ showDistribution("executor (non-fetch) time pct: ",
+ Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%")
+ showDistribution("fetch wait time pct: ",
+ Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%")
+ showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%")
+ taskInfoMetrics.clear()
+ }
+
+ private def getStatusDetail(info: StageInfo): String = {
+ val failureReason = info.failureReason.map("(" + _ + ")").getOrElse("")
+ val timeTaken = info.submissionTime.map(
+ x => info.completionTime.getOrElse(System.currentTimeMillis()) - x
+ ).getOrElse("-")
+
+ s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " +
+ s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " +
+ s"Took: $timeTaken msec"
+ }
+
+}
+
+private[spark] object StatsReportListener extends Logging {
+
+ // For profiling, the extremes are more interesting
+ val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100)
+ val probabilities = percentiles.map(_ / 100.0)
+ val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
+
+ def extractDoubleDistribution(
+ taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)],
+ getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = {
+ Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) })
+ }
+
+ // Is there some way to setup the types that I can get rid of this completely?
+ def extractLongDistribution(
+ taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)],
+ getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = {
+ extractDoubleDistribution(
+ taskInfoMetrics,
+ (info, metric) => { getMetric(info, metric).map(_.toDouble) })
+ }
+
+ def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) {
+ val stats = d.statCounter
+ val quantiles = d.getQuantiles(probabilities).map(formatNumber)
+ logInfo(heading + stats)
+ logInfo(percentilesHeader)
+ logInfo("\t" + quantiles.mkString("\t"))
+ }
+
+ def showDistribution(
+ heading: String,
+ dOpt: Option[Distribution],
+ formatNumber: Double => String) {
+ dOpt.foreach { d => showDistribution(heading, d, formatNumber)}
+ }
+
+ def showDistribution(heading: String, dOpt: Option[Distribution], format: String) {
+ def f(d: Double): String = format.format(d)
+ showDistribution(heading, dOpt, f _)
+ }
+
+ def showDistribution(
+ heading: String,
+ format: String,
+ getMetric: (TaskInfo, TaskMetrics) => Option[Double],
+ taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
+ showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format)
+ }
+
+ def showBytesDistribution(
+ heading: String,
+ getMetric: (TaskInfo, TaskMetrics) => Option[Long],
+ taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
+ showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric))
+ }
+
+ def showBytesDistribution(heading: String, dOpt: Option[Distribution]) {
+ dOpt.foreach { dist => showBytesDistribution(heading, dist) }
+ }
+
+ def showBytesDistribution(heading: String, dist: Distribution) {
+ showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String)
+ }
+
+ def showMillisDistribution(heading: String, dOpt: Option[Distribution]) {
+ showDistribution(heading, dOpt,
+ (d => StatsReportListener.millisToString(d.toLong)): Double => String)
+ }
+
+ def showMillisDistribution(
+ heading: String,
+ getMetric: (TaskInfo, TaskMetrics) => Option[Long],
+ taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) {
+ showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric))
+ }
+
+ val seconds = 1000L
+ val minutes = seconds * 60
+ val hours = minutes * 60
+
+ /**
+ * Reformat a time interval in milliseconds to a prettier format for output
+ */
+ def millisToString(ms: Long): String = {
+ val (size, units) =
+ if (ms > hours) {
+ (ms.toDouble / hours, "hours")
+ } else if (ms > minutes) {
+ (ms.toDouble / minutes, "min")
+ } else if (ms > seconds) {
+ (ms.toDouble / seconds, "s")
+ } else {
+ (ms.toDouble, "ms")
+ }
+ "%.1f %s".format(size, units)
+ }
+}
+
+private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double)
+
+private object RuntimePercentage {
+ def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = {
+ val denom = totalTime.toDouble
+ val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime)
+ val fetch = fetchTime.map(_ / denom)
+ val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom
+ val other = 1.0 - (exec + fetch.getOrElse(0d))
+ RuntimePercentage(exec, fetch, other)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index d2b8ca90a9..1ff9d7795f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -19,12 +19,13 @@ package org.apache.spark.scheduler
import java.io.{DataInputStream, DataOutputStream}
import java.nio.ByteBuffer
+import java.util.Properties
import scala.collection.mutable.HashMap
import org.apache.spark.{Accumulator, SparkEnv, TaskContext, TaskContextImpl}
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.memory.TaskMemoryManager
+import org.apache.spark.memory.{MemoryMode, TaskMemoryManager}
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.serializer.SerializerInstance
import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils}
@@ -46,12 +47,14 @@ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Uti
* @param initialAccumulators initial set of accumulators to be used in this task for tracking
* internal metrics. Other accumulators will be registered later when
* they are deserialized on the executors.
+ * @param localProperties copy of thread-local properties set by the user on the driver side.
*/
private[spark] abstract class Task[T](
val stageId: Int,
val stageAttemptId: Int,
val partitionId: Int,
- val initialAccumulators: Seq[Accumulator[_]]) extends Serializable {
+ val initialAccumulators: Seq[Accumulator[_]],
+ @transient var localProperties: Properties) extends Serializable {
/**
* Called by [[org.apache.spark.executor.Executor]] to run this task.
@@ -71,6 +74,7 @@ private[spark] abstract class Task[T](
taskAttemptId,
attemptNumber,
taskMemoryManager,
+ localProperties,
metricsSystem,
initialAccumulators)
TaskContext.setTaskContext(context)
@@ -80,17 +84,24 @@ private[spark] abstract class Task[T](
}
try {
runTask(context)
- } catch { case e: Throwable =>
- // Catch all errors; run task failure callbacks, and rethrow the exception.
- context.markTaskFailed(e)
- throw e
+ } catch {
+ case e: Throwable =>
+ // Catch all errors; run task failure callbacks, and rethrow the exception.
+ try {
+ context.markTaskFailed(e)
+ } catch {
+ case t: Throwable =>
+ e.addSuppressed(t)
+ }
+ throw e
} finally {
// Call the task completion callbacks.
context.markTaskCompleted()
try {
Utils.tryLogNonFatalError {
// Release memory used by this thread for unrolling blocks
- SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask()
+ SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP)
+ SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.OFF_HEAP)
// Notify any tasks waiting for execution memory to be freed to wake up and try to
// acquire memory again. This makes impossible the scenario where a task sleeps forever
// because there are no other tasks left to notify it. Since this is safe to do but may
@@ -205,6 +216,11 @@ private[spark] object Task {
dataOut.writeLong(timestamp)
}
+ // Write the task properties separately so it is available before full task deserialization.
+ val propBytes = Utils.serialize(task.localProperties)
+ dataOut.writeInt(propBytes.length)
+ dataOut.write(propBytes)
+
// Write the task itself and finish
dataOut.flush()
val taskBytes = serializer.serialize(task)
@@ -220,7 +236,7 @@ private[spark] object Task {
* @return (taskFiles, taskJars, taskBytes)
*/
def deserializeWithDependencies(serializedTask: ByteBuffer)
- : (HashMap[String, Long], HashMap[String, Long], ByteBuffer) = {
+ : (HashMap[String, Long], HashMap[String, Long], Properties, ByteBuffer) = {
val in = new ByteBufferInputStream(serializedTask)
val dataIn = new DataInputStream(in)
@@ -239,8 +255,13 @@ private[spark] object Task {
taskJars(dataIn.readUTF()) = dataIn.readLong()
}
+ val propLength = dataIn.readInt()
+ val propBytes = new Array[Byte](propLength)
+ dataIn.readFully(propBytes, 0, propLength)
+ val taskProps = Utils.deserialize[Properties](propBytes)
+
// Create a sub-buffer for the rest of the data, which is the serialized Task object
val subBuffer = serializedTask.slice() // ByteBufferInputStream will have read just up to task
- (taskFiles, taskJars, subBuffer)
+ (taskFiles, taskJars, taskProps, subBuffer)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 873f1b56bd..ae7ef46abb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -133,7 +133,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul
// if we can't deserialize the reason.
logError(
"Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
- case ex: Exception => {}
+ case ex: Exception => // No-op
}
scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 8477a66b39..647d44a0f0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -51,7 +51,7 @@ private[spark] trait TaskScheduler {
def submitTasks(taskSet: TaskSet): Unit
// Cancel a stage.
- def cancelTasks(stageId: Int, interruptThread: Boolean)
+ def cancelTasks(stageId: Int, interruptThread: Boolean): Unit
// Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called.
def setDAGScheduler(dagScheduler: DAGScheduler): Unit
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index f7790fccc6..c3159188d9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -90,6 +90,8 @@ private[spark] class TaskSchedulerImpl(
// Number of tasks running on each executor
private val executorIdToTaskCount = new HashMap[String, Int]
+ def runningTasksByExecutors(): Map[String, Int] = executorIdToTaskCount.toMap
+
// The set of executors we have on each host; this is used to compute hostsAlive, which
// in turn is used to decide when we can attain data locality on a given host
protected val executorsByHost = new HashMap[String, HashSet[String]]
@@ -569,6 +571,11 @@ private[spark] class TaskSchedulerImpl(
return
}
while (!backend.isReady) {
+ // Might take a while for backend to be ready if it is waiting on resources.
+ if (sc.stopped.get) {
+ // For example: the master removes the application for some reason
+ throw new IllegalStateException("Spark context stopped while waiting for backend")
+ }
synchronized {
this.wait(100)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 15d3515a02..6e08cdd87a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -188,20 +188,18 @@ private[spark] class TaskSetManager(
loc match {
case e: ExecutorCacheTaskLocation =>
pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer) += index
- case e: HDFSCacheTaskLocation => {
+ case e: HDFSCacheTaskLocation =>
val exe = sched.getExecutorsAliveOnHost(loc.host)
exe match {
- case Some(set) => {
+ case Some(set) =>
for (e <- set) {
pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer) += index
}
logInfo(s"Pending task $index has a cached location at ${e.host} " +
", where there are executors " + set.mkString(","))
- }
case None => logDebug(s"Pending task $index has a cached location at ${e.host} " +
", but there are no executors alive there.")
}
- }
case _ =>
}
pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index
@@ -437,7 +435,7 @@ private[spark] class TaskSetManager(
}
dequeueTask(execId, host, allowedLocality) match {
- case Some((index, taskLocality, speculative)) => {
+ case Some((index, taskLocality, speculative)) =>
// Found a task; do some bookkeeping and return a task description
val task = tasks(index)
val taskId = sched.newTaskId()
@@ -486,7 +484,6 @@ private[spark] class TaskSetManager(
sched.dagScheduler.taskStarted(task, info)
return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId,
taskName, index, serializedTask))
- }
case _ =>
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
index 8d5c11dc36..46a829114e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
@@ -30,6 +30,8 @@ private[spark] object CoarseGrainedClusterMessages {
case object RetrieveSparkProps extends CoarseGrainedClusterMessage
+ case object RetrieveLastAllocatedExecutorId extends CoarseGrainedClusterMessage
+
// Driver to executors
case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index b7919efc4b..8896391f97 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger
+import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
@@ -43,24 +44,30 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
extends ExecutorAllocationClient with SchedulerBackend with Logging
{
// Use an atomic variable to track total number of cores in the cluster for simplicity and speed
- var totalCoreCount = new AtomicInteger(0)
+ protected val totalCoreCount = new AtomicInteger(0)
// Total number of executors that are currently registered
- var totalRegisteredExecutors = new AtomicInteger(0)
- val conf = scheduler.sc.conf
+ protected val totalRegisteredExecutors = new AtomicInteger(0)
+ protected val conf = scheduler.sc.conf
private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf)
// Submit tasks only after (registered resources / total expected resources)
// is equal to at least this value, that is double between 0 and 1.
- var minRegisteredRatio =
+ private val _minRegisteredRatio =
math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0))
// Submit tasks after maxRegisteredWaitingTime milliseconds
// if minRegisteredRatio has not yet been reached
- val maxRegisteredWaitingTimeMs =
+ private val maxRegisteredWaitingTimeMs =
conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s")
- val createTime = System.currentTimeMillis()
+ private val createTime = System.currentTimeMillis()
+ // Accessing `executorDataMap` in `DriverEndpoint.receive/receiveAndReply` doesn't need any
+ // protection. But accessing `executorDataMap` out of `DriverEndpoint.receive/receiveAndReply`
+ // must be protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should
+ // only be modified in `DriverEndpoint.receive/receiveAndReply` with protection by
+ // `CoarseGrainedSchedulerBackend.this`.
private val executorDataMap = new HashMap[String, ExecutorData]
// Number of executors requested from the cluster manager that have not registered yet
+ @GuardedBy("CoarseGrainedSchedulerBackend.this")
private var numPendingExecutors = 0
private val listenerBus = scheduler.sc.listenerBus
@@ -68,20 +75,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// Executors we have requested the cluster manager to kill that have not died yet; maps
// the executor ID to whether it was explicitly killed by the driver (and thus shouldn't
// be considered an app-related failure).
+ @GuardedBy("CoarseGrainedSchedulerBackend.this")
private val executorsPendingToRemove = new HashMap[String, Boolean]
// A map to store hostname with its possible task number running on it
+ @GuardedBy("CoarseGrainedSchedulerBackend.this")
protected var hostToLocalTaskCount: Map[String, Int] = Map.empty
// The number of pending tasks which is locality required
+ @GuardedBy("CoarseGrainedSchedulerBackend.this")
protected var localityAwareTasks = 0
- // Executors that have been lost, but for which we don't yet know the real exit reason.
- protected val executorsPendingLossReason = new HashSet[String]
+ // The num of current max ExecutorId used to re-register appMaster
+ @volatile protected var currentExecutorIdCounter = 0
class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)])
extends ThreadSafeRpcEndpoint with Logging {
+ // Executors that have been lost, but for which we don't yet know the real exit reason.
+ protected val executorsPendingLossReason = new HashSet[String]
+
// If this DriverEndpoint is changed to support multiple threads,
// then this may need to be changed so that we don't share the serializer
// instance across threads
@@ -137,7 +150,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
case RegisterExecutor(executorId, executorRef, cores, logUrls) =>
if (executorDataMap.contains(executorId)) {
- context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId))
+ executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId))
+ context.reply(true)
} else {
// If the executor's rpc env is not listening for incoming connections, `hostPort`
// will be null, and the client connection should be used to contact the executor.
@@ -156,13 +170,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// in this block are read when requesting executors
CoarseGrainedSchedulerBackend.this.synchronized {
executorDataMap.put(executorId, data)
+ if (currentExecutorIdCounter < executorId.toInt) {
+ currentExecutorIdCounter = executorId.toInt
+ }
if (numPendingExecutors > 0) {
numPendingExecutors -= 1
logDebug(s"Decremented number of pending executors ($numPendingExecutors left)")
}
}
+ executorRef.send(RegisteredExecutor(executorAddress.host))
// Note: some tests expect the reply to come after we put the executor in the map
- context.reply(RegisteredExecutor(executorAddress.host))
+ context.reply(true)
listenerBus.post(
SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data))
makeOffers()
@@ -255,7 +273,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
// Remove a disconnected slave from the cluster
- def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = {
+ private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = {
executorDataMap.get(executorId) match {
case Some(executorInfo) =>
// This must be synchronized because variables mutated
@@ -307,7 +325,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
var driverEndpoint: RpcEndpointRef = null
- val taskIdsOnSlave = new HashMap[String, HashSet[String]]
+
+ protected def minRegisteredRatio: Double = _minRegisteredRatio
override def start() {
val properties = new ArrayBuffer[(String, String)]
@@ -356,20 +375,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
/**
* Reset the state of CoarseGrainedSchedulerBackend to the initial state. Currently it will only
- * be called in the yarn-client mode when AM re-registers after a failure, also dynamic
- * allocation is enabled.
+ * be called in the yarn-client mode when AM re-registers after a failure.
* */
protected def reset(): Unit = synchronized {
- if (Utils.isDynamicAllocationEnabled(conf)) {
- numPendingExecutors = 0
- executorsPendingToRemove.clear()
-
- // Remove all the lingering executors that should be removed but not yet. The reason might be
- // because (1) disconnected event is not yet received; (2) executors die silently.
- executorDataMap.toMap.foreach { case (eid, _) =>
- driverEndpoint.askWithRetry[Boolean](
- RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")))
- }
+ numPendingExecutors = 0
+ executorsPendingToRemove.clear()
+
+ // Remove all the lingering executors that should be removed but not yet. The reason might be
+ // because (1) disconnected event is not yet received; (2) executors die silently.
+ executorDataMap.toMap.foreach { case (eid, _) =>
+ driverEndpoint.askWithRetry[Boolean](
+ RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")))
}
}
@@ -414,7 +430,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
/**
* Return the number of executors currently registered with this backend.
*/
- def numExistingExecutors: Int = executorDataMap.size
+ private def numExistingExecutors: Int = executorDataMap.size
+
+ override def getExecutorIds(): Seq[String] = {
+ executorDataMap.keySet.toSeq
+ }
/**
* Request an additional number of executors from the cluster manager.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 90b1813750..50b452c72f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -295,12 +295,12 @@ private[spark] class CoarseMesosSchedulerBackend(
}
/**
- * Launches executors on accepted offers, and declines unused offers. Executors are launched
- * round-robin on offers.
- *
- * @param d SchedulerDriver
- * @param offers Mesos offers that match attribute constraints
- */
+ * Launches executors on accepted offers, and declines unused offers. Executors are launched
+ * round-robin on offers.
+ *
+ * @param d SchedulerDriver
+ * @param offers Mesos offers that match attribute constraints
+ */
private def handleMatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = {
val tasks = buildMesosTasks(offers)
for (offer <- offers) {
@@ -336,12 +336,12 @@ private[spark] class CoarseMesosSchedulerBackend(
}
/**
- * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize
- * per-task memory and IO, tasks are round-robin assigned to offers.
- *
- * @param offers Mesos offers that match attribute constraints
- * @return A map from OfferID to a list of Mesos tasks to launch on that offer
- */
+ * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize
+ * per-task memory and IO, tasks are round-robin assigned to offers.
+ *
+ * @param offers Mesos offers that match attribute constraints
+ * @return A map from OfferID to a list of Mesos tasks to launch on that offer
+ */
private def buildMesosTasks(offers: Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = {
// offerID -> tasks
val tasks = new HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
index 3971e6c382..61ab3e87c5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala
@@ -121,11 +121,10 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine(
Some(Utils.deserialize[T](fileData))
} catch {
case e: NoNodeException => None
- case e: Exception => {
+ case e: Exception =>
logWarning("Exception while reading persisted file, deleting", e)
zk.delete().forPath(zkPath)
None
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
index 2df7b1120b..73bd4c58e1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
@@ -423,6 +423,12 @@ private[spark] class MesosClusterScheduler(
"--driver-cores", desc.cores.toString,
"--driver-memory", s"${desc.mem}M")
+ val replicatedOptionsBlacklist = Set(
+ "spark.jars", // Avoids duplicate classes in classpath
+ "spark.submit.deployMode", // this would be set to `cluster`, but we need client
+ "spark.master" // this contains the address of the dispatcher, not master
+ )
+
// Assume empty main class means we're running python
if (!desc.command.mainClass.equals("")) {
options ++= Seq("--class", desc.command.mainClass)
@@ -440,9 +446,29 @@ private[spark] class MesosClusterScheduler(
.mkString(",")
options ++= Seq("--py-files", formattedFiles)
}
+ desc.schedulerProperties
+ .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) }
+ .foreach { case (key, value) => options ++= Seq("--conf", s"$key=${shellEscape(value)}") }
options
}
+ /**
+ * Escape args for Unix-like shells, unless already quoted by the user.
+ * Based on: http://www.gnu.org/software/bash/manual/html_node/Double-Quotes.html
+ * and http://www.grymoire.com/Unix/Quote.html
+ * @param value argument
+ * @return escaped argument
+ */
+ private[scheduler] def shellEscape(value: String): String = {
+ val WrappedInQuotes = """^(".+"|'.+')$""".r
+ val ShellSpecialChars = (""".*([ '<>&|\?\*;!#\\(\)"$`]).*""").r
+ value match {
+ case WrappedInQuotes(c) => value // The user quoted his args, don't touch it!
+ case ShellSpecialChars(c) => "\"" + value.replaceAll("""(["`\$\\])""", """\\$1""") + "\""
+ case _: String => value // Don't touch harmless strings
+ }
+ }
+
private class ResourceOffer(
val offerId: OfferID,
val slaveId: SlaveID,
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
index 374c79a7e5..1b7ac172de 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala
@@ -55,11 +55,10 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
Some(vol.setContainerPath(container_path)
.setHostPath(host_path)
.setMode(Volume.Mode.RO))
- case spec => {
+ case spec =>
logWarning(s"Unable to parse volume specs: $volumes. "
+ "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"")
None
- }
}
}
.map { _.build() }
@@ -90,11 +89,10 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging {
Some(portmap.setHostPort(host_port.toInt)
.setContainerPort(container_port.toInt)
.setProtocol(protocol))
- case spec => {
+ case spec =>
logWarning(s"Unable to parse port mapping specs: $portmaps. "
+ "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"")
None
- }
}
}
.map { _.build() }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
index 9a12a61f2f..1e322ac679 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
@@ -124,11 +124,10 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
markErr()
}
} catch {
- case e: Exception => {
+ case e: Exception =>
logError("driver.run() failed", e)
error = Some(e)
markErr()
- }
}
}
}.start()
@@ -148,8 +147,8 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
}
/**
- * Signal that the scheduler has registered with Mesos.
- */
+ * Signal that the scheduler has registered with Mesos.
+ */
protected def markRegistered(): Unit = {
registerLatch.countDown()
}
@@ -184,7 +183,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
var remain = amountToUse
var requestedResources = new ArrayBuffer[Resource]
val remainingResources = resources.asScala.map {
- case r => {
+ case r =>
if (remain > 0 &&
r.getType == Value.Type.SCALAR &&
r.getScalar.getValue > 0.0 &&
@@ -196,7 +195,6 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
} else {
r
}
- }
}
// Filter any resource that has depleted.
@@ -228,7 +226,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
* @return
*/
protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = {
- offerAttributes.asScala.map(attr => {
+ offerAttributes.asScala.map { attr =>
val attrValue = attr.getType match {
case Value.Type.SCALAR => attr.getScalar
case Value.Type.RANGES => attr.getRanges
@@ -236,7 +234,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
case Value.Type.TEXT => attr.getText
}
(attr.getName, attrValue)
- }).toMap
+ }.toMap
}
@@ -283,11 +281,11 @@ private[mesos] trait MesosSchedulerUtils extends Logging {
* are separated by ':'. The ':' implies equality (for singular values) and "is one of" for
* multiple values (comma separated). For example:
* {{{
- * parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b")
+ * parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b")
* // would result in
* <code>
* Map(
- * "tachyon" -> Set("true"),
+ * "os" -> Set("centos7"),
* "zone": -> Set("us-east-1a", "us-east-1b")
* )
* }}}
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 3d090a4353..918ae376f6 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -357,7 +357,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ
* serialization.
*/
trait KryoRegistrator {
- def registerClasses(kryo: Kryo)
+ def registerClasses(kryo: Kryo): Unit
}
private[serializer] object KryoSerializer {
diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
index 5ead40e89e..cb95246d5b 100644
--- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
@@ -188,10 +188,9 @@ abstract class DeserializationStream {
try {
(readKey[Any](), readValue[Any]())
} catch {
- case eof: EOFException => {
+ case eof: EOFException =>
finished = true
null
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
index 27e5fa4c2b..745ef12691 100644
--- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
@@ -25,7 +25,7 @@ import scala.reflect.ClassTag
import org.apache.spark.SparkConf
import org.apache.spark.io.CompressionCodec
import org.apache.spark.storage._
-import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer}
+import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream}
/**
* Component which configures serialization and compression for various Spark components, including
@@ -128,17 +128,9 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar
/** Serializes into a chunked byte buffer. */
def dataSerialize[T: ClassTag](blockId: BlockId, values: Iterator[T]): ChunkedByteBuffer = {
- val byteArrayChunkOutputStream = new ByteArrayChunkOutputStream(1024 * 1024 * 4)
- dataSerializeStream(blockId, byteArrayChunkOutputStream, values)
- new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap))
- }
-
- /**
- * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of
- * the iterator is reached.
- */
- def dataDeserialize[T: ClassTag](blockId: BlockId, bytes: ChunkedByteBuffer): Iterator[T] = {
- dataDeserializeStream[T](blockId, bytes.toInputStream(dispose = true))
+ val bbos = new ChunkedByteBufferOutputStream(1024 * 1024 * 4, ByteBuffer.allocate)
+ dataSerializeStream(blockId, bbos, values)
+ bbos.toChunkedByteBuffer
}
/**
diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
index 637b2dfc19..876cdfaa87 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -69,10 +69,10 @@ private[spark] class BlockStoreShuffleReader[K, C](
// Update the context task metrics for each record read.
val readMetrics = context.taskMetrics.registerTempShuffleReadMetrics()
val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]](
- recordIter.map(record => {
+ recordIter.map { record =>
readMetrics.incRecordsRead(1)
record
- }),
+ },
context.taskMetrics().mergeShuffleReadMetrics())
// An interruptible iterator must be used here in order to support task cancellation
diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala
index 6cd7d69518..be1e84a2ba 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala
@@ -35,7 +35,7 @@ private[spark] trait ShuffleWriterGroup {
val writers: Array[DiskBlockObjectWriter]
/** @param success Indicates all writes were successful. If false, no blocks will be recorded. */
- def releaseWriters(success: Boolean)
+ def releaseWriters(success: Boolean): Unit
}
/**
diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
index 76fd249fbd..364fad664e 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala
@@ -54,9 +54,9 @@ private[spark] trait ShuffleManager {
context: TaskContext): ShuffleReader[K, C]
/**
- * Remove a shuffle's metadata from the ShuffleManager.
- * @return true if the metadata removed successfully, otherwise false.
- */
+ * Remove a shuffle's metadata from the ShuffleManager.
+ * @return true if the metadata removed successfully, otherwise false.
+ */
def unregisterShuffle(shuffleId: Int): Boolean
/**
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
index 9c92a50150..f8d6e9fbbb 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
@@ -147,7 +147,7 @@ private[v1] object AllStagesResource {
speculative = uiData.taskInfo.speculative,
accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo },
errorMessage = uiData.errorMessage,
- taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics }
+ taskMetrics = uiData.metrics.map { convertUiTaskMetrics }
)
}
@@ -155,7 +155,7 @@ private[v1] object AllStagesResource {
allTaskData: Iterable[TaskUIData],
quantiles: Array[Double]): TaskMetricDistributions = {
- val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq
+ val rawMetrics = allTaskData.flatMap{_.metrics}.toSeq
def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] =
Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles)
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
index 50b6ba67e9..ba9cd711f1 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
@@ -177,6 +177,12 @@ private[v1] class ApiRootResource extends UIRootFromServletContext {
@PathParam("attemptId") attemptId: String): EventLogDownloadResource = {
new EventLogDownloadResource(uiRoot, appId, Some(attemptId))
}
+
+ @Path("version")
+ def getVersion(): VersionResource = {
+ new VersionResource(uiRoot)
+ }
+
}
private[spark] object ApiRootResource {
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala
new file mode 100644
index 0000000000..673da1ce36
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.status.api.v1
+
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class VersionResource(ui: UIRoot) {
+
+ @GET
+ def getVersionInfo(): VersionInfo = new VersionInfo(
+ org.apache.spark.SPARK_VERSION
+ )
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index 909dd0c07e..ebbbf48148 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -38,7 +38,11 @@ class ApplicationAttemptInfo private[spark](
val lastUpdated: Date,
val duration: Long,
val sparkUser: String,
- val completed: Boolean = false)
+ val completed: Boolean = false) {
+ def getStartTimeEpoch: Long = startTime.getTime
+ def getEndTimeEpoch: Long = endTime.getTime
+ def getLastUpdatedEpoch: Long = lastUpdated.getTime
+}
class ExecutorStageSummary private[spark](
val taskTime : Long,
@@ -237,3 +241,6 @@ class AccumulableInfo private[spark](
val name: String,
val update: Option[String],
val value: String)
+
+class VersionInfo private[spark](
+ val spark: String)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 0c7763f236..35a6c63ad1 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -18,6 +18,7 @@
package org.apache.spark.storage
import java.io._
+import java.nio.ByteBuffer
import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.concurrent.{Await, ExecutionContext, Future}
@@ -39,6 +40,7 @@ import org.apache.spark.rpc.RpcEnv
import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.storage.memory._
+import org.apache.spark.unsafe.Platform
import org.apache.spark.util._
import org.apache.spark.util.io.ChunkedByteBuffer
@@ -372,8 +374,12 @@ private[spark] class BlockManager(
val onDisk = level.useDisk && diskStore.contains(blockId)
val deserialized = if (inMem) level.deserialized else false
val replication = if (inMem || onDisk) level.replication else 1
- val storageLevel =
- StorageLevel(onDisk, inMem, deserialized, replication)
+ val storageLevel = StorageLevel(
+ useDisk = onDisk,
+ useMemory = inMem,
+ useOffHeap = level.useOffHeap,
+ deserialized = deserialized,
+ replication = replication)
val memSize = if (inMem) memoryStore.getSize(blockId) else 0L
val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L
BlockStatus(storageLevel, memSize, diskSize)
@@ -407,8 +413,8 @@ private[spark] class BlockManager(
val iter: Iterator[Any] = if (level.deserialized) {
memoryStore.getValues(blockId).get
} else {
- serializerManager.dataDeserialize(
- blockId, memoryStore.getBytes(blockId).get)(info.classTag)
+ serializerManager.dataDeserializeStream(
+ blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag)
}
val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId))
Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
@@ -416,11 +422,15 @@ private[spark] class BlockManager(
val iterToReturn: Iterator[Any] = {
val diskBytes = diskStore.getBytes(blockId)
if (level.deserialized) {
- val diskValues = serializerManager.dataDeserialize(blockId, diskBytes)(info.classTag)
+ val diskValues = serializerManager.dataDeserializeStream(
+ blockId,
+ diskBytes.toInputStream(dispose = true))(info.classTag)
maybeCacheDiskValuesInMemory(info, blockId, level, diskValues)
} else {
- val bytes = maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes)
- serializerManager.dataDeserialize(blockId, bytes)(info.classTag)
+ val stream = maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes)
+ .map {_.toInputStream(dispose = false)}
+ .getOrElse { diskBytes.toInputStream(dispose = true) }
+ serializerManager.dataDeserializeStream(blockId, stream)(info.classTag)
}
}
val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId))
@@ -481,7 +491,8 @@ private[spark] class BlockManager(
if (level.useMemory && memoryStore.contains(blockId)) {
memoryStore.getBytes(blockId).get
} else if (level.useDisk && diskStore.contains(blockId)) {
- maybeCacheDiskBytesInMemory(info, blockId, level, diskStore.getBytes(blockId))
+ val diskBytes = diskStore.getBytes(blockId)
+ maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes).getOrElse(diskBytes)
} else {
releaseLock(blockId)
throw new SparkException(s"Block $blockId was not found even though it's read-locked")
@@ -496,8 +507,9 @@ private[spark] class BlockManager(
*/
private def getRemoteValues(blockId: BlockId): Option[BlockResult] = {
getRemoteBytes(blockId).map { data =>
- new BlockResult(
- serializerManager.dataDeserialize(blockId, data), DataReadMethod.Network, data.size)
+ val values =
+ serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))
+ new BlockResult(values, DataReadMethod.Network, data.size)
}
}
@@ -631,6 +643,14 @@ private[spark] class BlockManager(
level: StorageLevel,
classTag: ClassTag[T],
makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = {
+ // Attempt to read the block from local or remote storage. If it's present, then we don't need
+ // to go through the local-get-or-put path.
+ get(blockId) match {
+ case Some(block) =>
+ return Left(block)
+ case _ =>
+ // Need to compute the block.
+ }
// Initially we hold no locks on this block.
doPutIterator(blockId, makeIterator, level, classTag, keepReadLock = true) match {
case None =>
@@ -745,7 +765,8 @@ private[spark] class BlockManager(
// Put it in memory first, even if it also has useDisk set to true;
// We will drop it to disk later if the memory store can't hold it.
val putSucceeded = if (level.deserialized) {
- val values = serializerManager.dataDeserialize(blockId, bytes)(classTag)
+ val values =
+ serializerManager.dataDeserializeStream(blockId, bytes.toInputStream())(classTag)
memoryStore.putIteratorAsValues(blockId, values, classTag) match {
case Right(_) => true
case Left(iter) =>
@@ -755,7 +776,7 @@ private[spark] class BlockManager(
false
}
} else {
- memoryStore.putBytes(blockId, size, () => bytes)
+ memoryStore.putBytes(blockId, size, level.memoryMode, () => bytes)
}
if (!putSucceeded && level.useDisk) {
logWarning(s"Persisting block $blockId to disk instead.")
@@ -893,7 +914,7 @@ private[spark] class BlockManager(
}
}
} else { // !level.deserialized
- memoryStore.putIteratorAsBytes(blockId, iterator(), classTag) match {
+ memoryStore.putIteratorAsBytes(blockId, iterator(), classTag, level.memoryMode) match {
case Right(s) =>
size = s
case Left(partiallySerializedValues) =>
@@ -951,14 +972,16 @@ private[spark] class BlockManager(
* Attempts to cache spilled bytes read from disk into the MemoryStore in order to speed up
* subsequent reads. This method requires the caller to hold a read lock on the block.
*
- * @return a copy of the bytes. The original bytes passed this method should no longer
- * be used after this method returns.
+ * @return a copy of the bytes from the memory store if the put succeeded, otherwise None.
+ * If this returns bytes from the memory store then the original disk store bytes will
+ * automatically be disposed and the caller should not continue to use them. Otherwise,
+ * if this returns None then the original disk store bytes will be unaffected.
*/
private def maybeCacheDiskBytesInMemory(
blockInfo: BlockInfo,
blockId: BlockId,
level: StorageLevel,
- diskBytes: ChunkedByteBuffer): ChunkedByteBuffer = {
+ diskBytes: ChunkedByteBuffer): Option[ChunkedByteBuffer] = {
require(!level.deserialized)
if (level.useMemory) {
// Synchronize on blockInfo to guard against a race condition where two readers both try to
@@ -966,25 +989,29 @@ private[spark] class BlockManager(
blockInfo.synchronized {
if (memoryStore.contains(blockId)) {
diskBytes.dispose()
- memoryStore.getBytes(blockId).get
+ Some(memoryStore.getBytes(blockId).get)
} else {
- val putSucceeded = memoryStore.putBytes(blockId, diskBytes.size, () => {
+ val allocator = level.memoryMode match {
+ case MemoryMode.ON_HEAP => ByteBuffer.allocate _
+ case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
+ }
+ val putSucceeded = memoryStore.putBytes(blockId, diskBytes.size, level.memoryMode, () => {
// https://issues.apache.org/jira/browse/SPARK-6076
// If the file size is bigger than the free memory, OOM will happen. So if we
// cannot put it into MemoryStore, copyForMemory should not be created. That's why
// this action is put into a `() => ChunkedByteBuffer` and created lazily.
- diskBytes.copy()
+ diskBytes.copy(allocator)
})
if (putSucceeded) {
diskBytes.dispose()
- memoryStore.getBytes(blockId).get
+ Some(memoryStore.getBytes(blockId).get)
} else {
- diskBytes
+ None
}
}
}
} else {
- diskBytes
+ None
}
}
@@ -1055,7 +1082,12 @@ private[spark] class BlockManager(
val peersForReplication = new ArrayBuffer[BlockManagerId]
val peersReplicatedTo = new ArrayBuffer[BlockManagerId]
val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId]
- val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
+ val tLevel = StorageLevel(
+ useDisk = level.useDisk,
+ useMemory = level.useMemory,
+ useOffHeap = level.useOffHeap,
+ deserialized = level.deserialized,
+ replication = 1)
val startTime = System.currentTimeMillis
val random = new Random(blockId.hashCode)
@@ -1264,9 +1296,12 @@ private[spark] class BlockManager(
"the disk, memory, or external block store")
}
blockInfoManager.removeBlock(blockId)
+ val removeBlockStatus = getCurrentBlockStatus(blockId, info)
if (tellMaster && info.tellMaster) {
- val status = getCurrentBlockStatus(blockId, info)
- reportBlockStatus(blockId, info, status)
+ reportBlockStatus(blockId, info, removeBlockStatus)
+ }
+ Option(TaskContext.get()).foreach { c =>
+ c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, removeBlockStatus)))
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
index d2a5c69e15..8fa1215011 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
@@ -453,7 +453,7 @@ private[spark] class BlockManagerInfo(
}
if (storageLevel.isValid) {
- /* isValid means it is either stored in-memory, on-disk or on-externalBlockStore.
+ /* isValid means it is either stored in-memory or on-disk.
* The memSize here indicates the data size in or dropped from memory,
* externalBlockStoreSize here indicates the data size in or dropped from externalBlockStore,
* and the diskSize here indicates the data size in or dropped to disk.
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index 25edb9f1e4..4ec5b4bbb0 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -143,13 +143,12 @@ final class ShuffleBlockFetcherIterator(
while (iter.hasNext) {
val result = iter.next()
result match {
- case SuccessFetchResult(_, address, _, buf, _) => {
+ case SuccessFetchResult(_, address, _, buf, _) =>
if (address != blockManager.blockManagerId) {
shuffleMetrics.incRemoteBytesRead(buf.size)
shuffleMetrics.incRemoteBlocksFetched(1)
}
buf.release()
- }
case _ =>
}
}
@@ -313,7 +312,7 @@ final class ShuffleBlockFetcherIterator(
shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait)
result match {
- case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) => {
+ case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) =>
if (address != blockManager.blockManagerId) {
shuffleMetrics.incRemoteBytesRead(buf.size)
shuffleMetrics.incRemoteBlocksFetched(1)
@@ -323,7 +322,6 @@ final class ShuffleBlockFetcherIterator(
reqsInFlight -= 1
logDebug("Number of requests in flight " + reqsInFlight)
}
- }
case _ =>
}
// Send fetch requests up to maxBytesInFlight
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index 7d23295e25..216ec07934 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -60,10 +60,7 @@ class StorageLevel private(
assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes")
if (useOffHeap) {
- require(!useDisk, "Off-heap storage level does not support using disk")
- require(!useMemory, "Off-heap storage level does not support using heap memory")
require(!deserialized, "Off-heap storage level does not support deserialized storage")
- require(replication == 1, "Off-heap storage level does not support multiple replication")
}
private[spark] def memoryMode: MemoryMode = {
@@ -86,7 +83,7 @@ class StorageLevel private(
false
}
- def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0)
+ def isValid: Boolean = (useMemory || useDisk) && (replication > 0)
def toInt: Int = {
var ret = 0
@@ -123,7 +120,8 @@ class StorageLevel private(
private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this)
override def toString: String = {
- s"StorageLevel($useDisk, $useMemory, $useOffHeap, $deserialized, $replication)"
+ s"StorageLevel(disk=$useDisk, memory=$useMemory, offheap=$useOffHeap, " +
+ s"deserialized=$deserialized, replication=$replication)"
}
override def hashCode(): Int = toInt * 41 + replication
@@ -131,8 +129,9 @@ class StorageLevel private(
def description: String = {
var result = ""
result += (if (useDisk) "Disk " else "")
- result += (if (useMemory) "Memory " else "")
- result += (if (useOffHeap) "ExternalBlockStore " else "")
+ if (useMemory) {
+ result += (if (useOffHeap) "Memory (off heap) " else "Memory ")
+ }
result += (if (deserialized) "Deserialized " else "Serialized ")
result += s"${replication}x Replicated"
result
@@ -156,9 +155,7 @@ object StorageLevel {
val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2)
val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false)
val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2)
-
- // Redirect to MEMORY_ONLY_SER for now.
- val OFF_HEAP = MEMORY_ONLY_SER
+ val OFF_HEAP = new StorageLevel(true, true, true, false, 1)
/**
* :: DeveloperApi ::
@@ -183,7 +180,7 @@ object StorageLevel {
/**
* :: DeveloperApi ::
- * Create a new StorageLevel object without setting useOffHeap.
+ * Create a new StorageLevel object.
*/
@DeveloperApi
def apply(
@@ -198,7 +195,7 @@ object StorageLevel {
/**
* :: DeveloperApi ::
- * Create a new StorageLevel object.
+ * Create a new StorageLevel object without setting useOffHeap.
*/
@DeveloperApi
def apply(
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
index f552b498a7..3008520f61 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
@@ -66,17 +66,6 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener {
}
}
- override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
- val info = taskEnd.taskInfo
- val metrics = taskEnd.taskMetrics
- if (info != null && metrics != null) {
- val updatedBlocks = metrics.updatedBlockStatuses
- if (updatedBlocks.length > 0) {
- updateStorageStatus(info.executorId, updatedBlocks)
- }
- }
- }
-
override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized {
updateStorageStatus(unpersistRDD.rddId)
}
@@ -102,4 +91,14 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener {
}
}
}
+
+ override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = {
+ val executorId = blockUpdated.blockUpdatedInfo.blockManagerId.executorId
+ val blockId = blockUpdated.blockUpdatedInfo.blockId
+ val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel
+ val memSize = blockUpdated.blockUpdatedInfo.memSize
+ val diskSize = blockUpdated.blockUpdatedInfo.diskSize
+ val blockStatus = BlockStatus(storageLevel, memSize, diskSize)
+ updateStorageStatus(executorId, Seq((blockId, blockStatus)))
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index 199a5fc270..fb9941bbd9 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -175,7 +175,10 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) {
def memRemaining: Long = maxMem - memUsed
/** Return the memory used by this block manager. */
- def memUsed: Long = _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum
+ def memUsed: Long = _nonRddStorageInfo._1 + cacheSize
+
+ /** Return the memory used by caching RDDs */
+ def cacheSize: Long = _rddBlocks.keys.toSeq.map(memUsedByRdd).sum
/** Return the disk space used by this block manager. */
def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum
diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
index 3ca41f32c1..99be4de065 100644
--- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
@@ -32,20 +32,25 @@ import org.apache.spark.internal.Logging
import org.apache.spark.memory.{MemoryManager, MemoryMode}
import org.apache.spark.serializer.{SerializationStream, SerializerManager}
import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel}
+import org.apache.spark.unsafe.Platform
import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils}
import org.apache.spark.util.collection.SizeTrackingVector
-import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer}
+import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream}
private sealed trait MemoryEntry[T] {
def size: Long
+ def memoryMode: MemoryMode
def classTag: ClassTag[T]
}
private case class DeserializedMemoryEntry[T](
value: Array[T],
size: Long,
- classTag: ClassTag[T]) extends MemoryEntry[T]
+ classTag: ClassTag[T]) extends MemoryEntry[T] {
+ val memoryMode: MemoryMode = MemoryMode.ON_HEAP
+}
private case class SerializedMemoryEntry[T](
buffer: ChunkedByteBuffer,
+ memoryMode: MemoryMode,
classTag: ClassTag[T]) extends MemoryEntry[T] {
def size: Long = buffer.size
}
@@ -86,7 +91,10 @@ private[spark] class MemoryStore(
// A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes)
// All accesses of this map are assumed to have manually synchronized on `memoryManager`
- private val unrollMemoryMap = mutable.HashMap[Long, Long]()
+ private val onHeapUnrollMemoryMap = mutable.HashMap[Long, Long]()
+ // Note: off-heap unroll memory is only used in putIteratorAsBytes() because off-heap caching
+ // always stores serialized values.
+ private val offHeapUnrollMemoryMap = mutable.HashMap[Long, Long]()
// Initial memory to request before unrolling any block
private val unrollMemoryThreshold: Long =
@@ -131,13 +139,14 @@ private[spark] class MemoryStore(
def putBytes[T: ClassTag](
blockId: BlockId,
size: Long,
+ memoryMode: MemoryMode,
_bytes: () => ChunkedByteBuffer): Boolean = {
require(!contains(blockId), s"Block $blockId is already present in the MemoryStore")
- if (memoryManager.acquireStorageMemory(blockId, size, MemoryMode.ON_HEAP)) {
+ if (memoryManager.acquireStorageMemory(blockId, size, memoryMode)) {
// We acquired enough memory for the block, so go ahead and put it
val bytes = _bytes()
assert(bytes.size == size)
- val entry = new SerializedMemoryEntry[T](bytes, implicitly[ClassTag[T]])
+ val entry = new SerializedMemoryEntry[T](bytes, memoryMode, implicitly[ClassTag[T]])
entries.synchronized {
entries.put(blockId, entry)
}
@@ -190,7 +199,8 @@ private[spark] class MemoryStore(
var vector = new SizeTrackingVector[T]()(classTag)
// Request enough memory to begin unrolling
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold)
+ keepUnrolling =
+ reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, MemoryMode.ON_HEAP)
if (!keepUnrolling) {
logWarning(s"Failed to reserve initial memory threshold of " +
@@ -207,7 +217,8 @@ private[spark] class MemoryStore(
val currentSize = vector.estimateSize()
if (currentSize >= memoryThreshold) {
val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest)
+ keepUnrolling =
+ reserveUnrollMemoryForThisTask(blockId, amountToRequest, MemoryMode.ON_HEAP)
if (keepUnrolling) {
unrollMemoryUsedByThisBlock += amountToRequest
}
@@ -228,7 +239,7 @@ private[spark] class MemoryStore(
def transferUnrollToStorage(amount: Long): Unit = {
// Synchronize so that transfer is atomic
memoryManager.synchronized {
- releaseUnrollMemoryForThisTask(amount)
+ releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, amount)
val success = memoryManager.acquireStorageMemory(blockId, amount, MemoryMode.ON_HEAP)
assert(success, "transferring unroll memory to storage memory failed")
}
@@ -247,7 +258,7 @@ private[spark] class MemoryStore(
// If this task attempt already owns more unroll memory than is necessary to store the
// block, then release the extra memory that will not be used.
val excessUnrollMemory = unrollMemoryUsedByThisBlock - size
- releaseUnrollMemoryForThisTask(excessUnrollMemory)
+ releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, excessUnrollMemory)
transferUnrollToStorage(size)
true
}
@@ -295,10 +306,16 @@ private[spark] class MemoryStore(
private[storage] def putIteratorAsBytes[T](
blockId: BlockId,
values: Iterator[T],
- classTag: ClassTag[T]): Either[PartiallySerializedBlock[T], Long] = {
+ classTag: ClassTag[T],
+ memoryMode: MemoryMode): Either[PartiallySerializedBlock[T], Long] = {
require(!contains(blockId), s"Block $blockId is already present in the MemoryStore")
+ val allocator = memoryMode match {
+ case MemoryMode.ON_HEAP => ByteBuffer.allocate _
+ case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
+ }
+
// Whether there is still enough memory for us to continue unrolling this block
var keepUnrolling = true
// Initial per-task memory to request for unrolling blocks (bytes).
@@ -307,15 +324,15 @@ private[spark] class MemoryStore(
var unrollMemoryUsedByThisBlock = 0L
// Underlying buffer for unrolling the block
val redirectableStream = new RedirectableOutputStream
- val byteArrayChunkOutputStream = new ByteArrayChunkOutputStream(initialMemoryThreshold.toInt)
- redirectableStream.setOutputStream(byteArrayChunkOutputStream)
+ val bbos = new ChunkedByteBufferOutputStream(initialMemoryThreshold.toInt, allocator)
+ redirectableStream.setOutputStream(bbos)
val serializationStream: SerializationStream = {
val ser = serializerManager.getSerializer(classTag).newInstance()
ser.serializeStream(serializerManager.wrapForCompression(blockId, redirectableStream))
}
// Request enough memory to begin unrolling
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold)
+ keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, memoryMode)
if (!keepUnrolling) {
logWarning(s"Failed to reserve initial memory threshold of " +
@@ -325,9 +342,9 @@ private[spark] class MemoryStore(
}
def reserveAdditionalMemoryIfNecessary(): Unit = {
- if (byteArrayChunkOutputStream.size > unrollMemoryUsedByThisBlock) {
- val amountToRequest = byteArrayChunkOutputStream.size - unrollMemoryUsedByThisBlock
- keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest)
+ if (bbos.size > unrollMemoryUsedByThisBlock) {
+ val amountToRequest = bbos.size - unrollMemoryUsedByThisBlock
+ keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
if (keepUnrolling) {
unrollMemoryUsedByThisBlock += amountToRequest
}
@@ -349,12 +366,11 @@ private[spark] class MemoryStore(
}
if (keepUnrolling) {
- val entry = SerializedMemoryEntry[T](
- new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap)), classTag)
+ val entry = SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag)
// Synchronize so that transfer is atomic
memoryManager.synchronized {
- releaseUnrollMemoryForThisTask(unrollMemoryUsedByThisBlock)
- val success = memoryManager.acquireStorageMemory(blockId, entry.size, MemoryMode.ON_HEAP)
+ releaseUnrollMemoryForThisTask(memoryMode, unrollMemoryUsedByThisBlock)
+ val success = memoryManager.acquireStorageMemory(blockId, entry.size, memoryMode)
assert(success, "transferring unroll memory to storage memory failed")
}
entries.synchronized {
@@ -365,7 +381,7 @@ private[spark] class MemoryStore(
Right(entry.size)
} else {
// We ran out of space while unrolling the values for this block
- logUnrollFailureMessage(blockId, byteArrayChunkOutputStream.size)
+ logUnrollFailureMessage(blockId, bbos.size)
Left(
new PartiallySerializedBlock(
this,
@@ -374,7 +390,8 @@ private[spark] class MemoryStore(
serializationStream,
redirectableStream,
unrollMemoryUsedByThisBlock,
- new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap)),
+ memoryMode,
+ bbos.toChunkedByteBuffer,
values,
classTag))
}
@@ -386,7 +403,7 @@ private[spark] class MemoryStore(
case null => None
case e: DeserializedMemoryEntry[_] =>
throw new IllegalArgumentException("should only call getBytes on serialized blocks")
- case SerializedMemoryEntry(bytes, _) => Some(bytes)
+ case SerializedMemoryEntry(bytes, _, _) => Some(bytes)
}
}
@@ -407,8 +424,12 @@ private[spark] class MemoryStore(
entries.remove(blockId)
}
if (entry != null) {
- memoryManager.releaseStorageMemory(entry.size, MemoryMode.ON_HEAP)
- logInfo(s"Block $blockId of size ${entry.size} dropped " +
+ entry match {
+ case SerializedMemoryEntry(buffer, _, _) => buffer.dispose()
+ case _ =>
+ }
+ memoryManager.releaseStorageMemory(entry.size, entry.memoryMode)
+ logDebug(s"Block $blockId of size ${entry.size} dropped " +
s"from memory (free ${maxMemory - blocksMemoryUsed})")
true
} else {
@@ -420,7 +441,8 @@ private[spark] class MemoryStore(
entries.synchronized {
entries.clear()
}
- unrollMemoryMap.clear()
+ onHeapUnrollMemoryMap.clear()
+ offHeapUnrollMemoryMap.clear()
memoryManager.releaseAllStorageMemory()
logInfo("MemoryStore cleared")
}
@@ -433,23 +455,27 @@ private[spark] class MemoryStore(
}
/**
- * Try to evict blocks to free up a given amount of space to store a particular block.
- * Can fail if either the block is bigger than our memory or it would require replacing
- * another block from the same RDD (which leads to a wasteful cyclic replacement pattern for
- * RDDs that don't fit into memory that we want to avoid).
- *
- * @param blockId the ID of the block we are freeing space for, if any
- * @param space the size of this block
- * @return the amount of memory (in bytes) freed by eviction
- */
- private[spark] def evictBlocksToFreeSpace(blockId: Option[BlockId], space: Long): Long = {
+ * Try to evict blocks to free up a given amount of space to store a particular block.
+ * Can fail if either the block is bigger than our memory or it would require replacing
+ * another block from the same RDD (which leads to a wasteful cyclic replacement pattern for
+ * RDDs that don't fit into memory that we want to avoid).
+ *
+ * @param blockId the ID of the block we are freeing space for, if any
+ * @param space the size of this block
+ * @param memoryMode the type of memory to free (on- or off-heap)
+ * @return the amount of memory (in bytes) freed by eviction
+ */
+ private[spark] def evictBlocksToFreeSpace(
+ blockId: Option[BlockId],
+ space: Long,
+ memoryMode: MemoryMode): Long = {
assert(space > 0)
memoryManager.synchronized {
var freedMemory = 0L
val rddToAdd = blockId.flatMap(getRddId)
val selectedBlocks = new ArrayBuffer[BlockId]
- def blockIsEvictable(blockId: BlockId): Boolean = {
- rddToAdd.isEmpty || rddToAdd != getRddId(blockId)
+ def blockIsEvictable(blockId: BlockId, entry: MemoryEntry[_]): Boolean = {
+ entry.memoryMode == memoryMode && (rddToAdd.isEmpty || rddToAdd != getRddId(blockId))
}
// This is synchronized to ensure that the set of entries is not changed
// (because of getValue or getBytes) while traversing the iterator, as that
@@ -459,7 +485,8 @@ private[spark] class MemoryStore(
while (freedMemory < space && iterator.hasNext) {
val pair = iterator.next()
val blockId = pair.getKey
- if (blockIsEvictable(blockId)) {
+ val entry = pair.getValue
+ if (blockIsEvictable(blockId, entry)) {
// We don't want to evict blocks which are currently being read, so we need to obtain
// an exclusive write lock on blocks which are candidates for eviction. We perform a
// non-blocking "tryLock" here in order to ignore blocks which are locked for reading:
@@ -474,7 +501,7 @@ private[spark] class MemoryStore(
def dropBlock[T](blockId: BlockId, entry: MemoryEntry[T]): Unit = {
val data = entry match {
case DeserializedMemoryEntry(values, _, _) => Left(values)
- case SerializedMemoryEntry(buffer, _) => Right(buffer)
+ case SerializedMemoryEntry(buffer, _, _) => Right(buffer)
}
val newEffectiveStorageLevel =
blockEvictionHandler.dropFromMemory(blockId, () => data)(entry.classTag)
@@ -530,11 +557,18 @@ private[spark] class MemoryStore(
*
* @return whether the request is granted.
*/
- def reserveUnrollMemoryForThisTask(blockId: BlockId, memory: Long): Boolean = {
+ def reserveUnrollMemoryForThisTask(
+ blockId: BlockId,
+ memory: Long,
+ memoryMode: MemoryMode): Boolean = {
memoryManager.synchronized {
- val success = memoryManager.acquireUnrollMemory(blockId, memory, MemoryMode.ON_HEAP)
+ val success = memoryManager.acquireUnrollMemory(blockId, memory, memoryMode)
if (success) {
val taskAttemptId = currentTaskAttemptId()
+ val unrollMemoryMap = memoryMode match {
+ case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap
+ case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap
+ }
unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory
}
success
@@ -545,9 +579,13 @@ private[spark] class MemoryStore(
* Release memory used by this task for unrolling blocks.
* If the amount is not specified, remove the current task's allocation altogether.
*/
- def releaseUnrollMemoryForThisTask(memory: Long = Long.MaxValue): Unit = {
+ def releaseUnrollMemoryForThisTask(memoryMode: MemoryMode, memory: Long = Long.MaxValue): Unit = {
val taskAttemptId = currentTaskAttemptId()
memoryManager.synchronized {
+ val unrollMemoryMap = memoryMode match {
+ case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap
+ case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap
+ }
if (unrollMemoryMap.contains(taskAttemptId)) {
val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId))
if (memoryToRelease > 0) {
@@ -555,7 +593,7 @@ private[spark] class MemoryStore(
if (unrollMemoryMap(taskAttemptId) == 0) {
unrollMemoryMap.remove(taskAttemptId)
}
- memoryManager.releaseUnrollMemory(memoryToRelease, MemoryMode.ON_HEAP)
+ memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode)
}
}
}
@@ -565,20 +603,23 @@ private[spark] class MemoryStore(
* Return the amount of memory currently occupied for unrolling blocks across all tasks.
*/
def currentUnrollMemory: Long = memoryManager.synchronized {
- unrollMemoryMap.values.sum
+ onHeapUnrollMemoryMap.values.sum + offHeapUnrollMemoryMap.values.sum
}
/**
* Return the amount of memory currently occupied for unrolling blocks by this task.
*/
def currentUnrollMemoryForThisTask: Long = memoryManager.synchronized {
- unrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L)
+ onHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) +
+ offHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L)
}
/**
* Return the number of tasks currently unrolling blocks.
*/
- private def numTasksUnrolling: Int = memoryManager.synchronized { unrollMemoryMap.keys.size }
+ private def numTasksUnrolling: Int = memoryManager.synchronized {
+ (onHeapUnrollMemoryMap.keys ++ offHeapUnrollMemoryMap.keys).toSet.size
+ }
/**
* Log information about current memory usage.
@@ -627,7 +668,7 @@ private[storage] class PartiallyUnrolledIterator[T](
private[this] var iter: Iterator[T] = {
val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, {
unrolledIteratorIsConsumed = true
- memoryStore.releaseUnrollMemoryForThisTask(unrollMemory)
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory)
})
completionIterator ++ rest
}
@@ -640,7 +681,7 @@ private[storage] class PartiallyUnrolledIterator[T](
*/
def close(): Unit = {
if (!unrolledIteratorIsConsumed) {
- memoryStore.releaseUnrollMemoryForThisTask(unrollMemory)
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory)
unrolledIteratorIsConsumed = true
}
iter = null
@@ -669,6 +710,7 @@ private class RedirectableOutputStream extends OutputStream {
* @param serializationStream a serialization stream which writes to [[redirectableOutputStream]].
* @param redirectableOutputStream an OutputStream which can be redirected to a different sink.
* @param unrollMemory the amount of unroll memory used by the values in `unrolled`.
+ * @param memoryMode whether the unroll memory is on- or off-heap
* @param unrolled a byte buffer containing the partially-serialized values.
* @param rest the rest of the original iterator passed to
* [[MemoryStore.putIteratorAsValues()]].
@@ -681,18 +723,36 @@ private[storage] class PartiallySerializedBlock[T](
serializationStream: SerializationStream,
redirectableOutputStream: RedirectableOutputStream,
unrollMemory: Long,
+ memoryMode: MemoryMode,
unrolled: ChunkedByteBuffer,
rest: Iterator[T],
classTag: ClassTag[T]) {
+ // If the task does not fully consume `valuesIterator` or otherwise fails to consume or dispose of
+ // this PartiallySerializedBlock then we risk leaking of direct buffers, so we use a task
+ // completion listener here in order to ensure that `unrolled.dispose()` is called at least once.
+ // The dispose() method is idempotent, so it's safe to call it unconditionally.
+ Option(TaskContext.get()).foreach { taskContext =>
+ taskContext.addTaskCompletionListener { _ =>
+ // When a task completes, its unroll memory will automatically be freed. Thus we do not call
+ // releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing.
+ unrolled.dispose()
+ }
+ }
+
/**
* Called to dispose of this block and free its memory.
*/
def discard(): Unit = {
try {
+ // We want to close the output stream in order to free any resources associated with the
+ // serializer itself (such as Kryo's internal buffers). close() might cause data to be
+ // written, so redirect the output stream to discard that data.
+ redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream())
serializationStream.close()
} finally {
- memoryStore.releaseUnrollMemoryForThisTask(unrollMemory)
+ unrolled.dispose()
+ memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory)
}
}
@@ -701,12 +761,14 @@ private[storage] class PartiallySerializedBlock[T](
* and then serializing the values from the original input iterator.
*/
def finishWritingToStream(os: OutputStream): Unit = {
- ByteStreams.copy(unrolled.toInputStream(), os)
+ // `unrolled`'s underlying buffers will be freed once this input stream is fully read:
+ ByteStreams.copy(unrolled.toInputStream(dispose = true), os)
+ memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory)
redirectableOutputStream.setOutputStream(os)
while (rest.hasNext) {
serializationStream.writeObject(rest.next())(classTag)
}
- discard()
+ serializationStream.close()
}
/**
@@ -717,10 +779,13 @@ private[storage] class PartiallySerializedBlock[T](
* `close()` on it to free its resources.
*/
def valuesIterator: PartiallyUnrolledIterator[T] = {
+ // `unrolled`'s underlying buffers will be freed once this input stream is fully read:
+ val unrolledIter = serializerManager.dataDeserializeStream(
+ blockId, unrolled.toInputStream(dispose = true))(classTag)
new PartiallyUnrolledIterator(
memoryStore,
unrollMemory,
- unrolled = serializerManager.dataDeserialize(blockId, unrolled)(classTag),
+ unrolled = CompletionIterator[T, Iterator[T]](unrolledIter, discard()),
rest = rest)
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index c3c59f857d..119165f724 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -30,6 +30,7 @@ import org.eclipse.jetty.server.handler._
import org.eclipse.jetty.server.nio.SelectChannelConnector
import org.eclipse.jetty.server.ssl.SslSelectChannelConnector
import org.eclipse.jetty.servlet._
+import org.eclipse.jetty.util.component.LifeCycle
import org.eclipse.jetty.util.thread.QueuedThreadPool
import org.json4s.JValue
import org.json4s.jackson.JsonMethods.{pretty, render}
@@ -350,4 +351,15 @@ private[spark] object JettyUtils extends Logging {
private[spark] case class ServerInfo(
server: Server,
boundPort: Int,
- rootHandler: ContextHandlerCollection)
+ rootHandler: ContextHandlerCollection) {
+
+ def stop(): Unit = {
+ server.stop()
+ // Stop the ThreadPool if it supports stop() method (through LifeCycle).
+ // It is needed because stopping the Server won't stop the ThreadPool it uses.
+ val threadPool = server.getThreadPool
+ if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) {
+ threadPool.asInstanceOf[LifeCycle].stop
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 6057522509..39155ff264 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -80,6 +80,10 @@ private[spark] class SparkUI private (
}
initialize()
+ def getSparkUser: String = {
+ environmentListener.systemProperties.toMap.get("user.name").getOrElse("<unknown>")
+ }
+
def getAppName: String = appName
def setAppId(id: String): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index 250b7f2e5f..2b0bc32cf6 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -129,7 +129,7 @@ private[spark] abstract class WebUI(
}
/** Initialize all components of the server. */
- def initialize()
+ def initialize(): Unit
/** Bind to the HTTP server behind this web interface. */
def bind() {
@@ -153,7 +153,7 @@ private[spark] abstract class WebUI(
def stop() {
assert(serverInfo.isDefined,
"Attempted to stop %s before binding to a server!".format(className))
- serverInfo.get.server.stop()
+ serverInfo.get.stop()
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
index cc476d61b5..a0ef80d9bd 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
@@ -38,7 +38,7 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage
val content = maybeThreadDump.map { threadDump =>
val dumpRows = threadDump.sortWith {
- case (threadTrace1, threadTrace2) => {
+ case (threadTrace1, threadTrace2) =>
val v1 = if (threadTrace1.threadName.contains("Executor task launch")) 1 else 0
val v2 = if (threadTrace2.threadName.contains("Executor task launch")) 1 else 0
if (v1 == v2) {
@@ -46,7 +46,6 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage
} else {
v1 > v2
}
- }
}.map { thread =>
val threadId = thread.threadId
<tr id={s"thread_${threadId}_tr"} class="accordion-heading"
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
index 788f35ec77..3fd0efd3a1 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
@@ -70,7 +70,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar
executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap
executorToTotalCores(eid) = executorAdded.executorInfo.totalCores
executorToTasksMax(eid) = executorToTotalCores(eid) / conf.getInt("spark.task.cpus", 1)
- executorIdToData(eid) = ExecutorUIData(executorAdded.time)
+ executorIdToData(eid) = new ExecutorUIData(executorAdded.time)
}
override def onExecutorRemoved(
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index d1c8b3089a..07484c9550 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -148,7 +148,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
| 'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' +
| '${
if (event.finishReason.isDefined) {
- s"""<br>Reason: ${event.finishReason.get}"""
+ s"""<br>Reason: ${event.finishReason.get.replace("\n", " ")}"""
} else {
""
}
@@ -297,6 +297,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
<div>
<ul class="unstyled">
<li>
+ <strong>User:</strong>
+ {parent.getSparkUser}
+ </li>
+ <li>
<strong>Total Uptime:</strong>
{
if (endTime < 0 && parent.sc.isDefined) {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
index 1304efd8f2..f609fb4cd2 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
@@ -42,13 +42,13 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
var hasShuffleWrite = false
var hasShuffleRead = false
var hasBytesSpilled = false
- stageData.foreach(data => {
+ stageData.foreach { data =>
hasInput = data.hasInput
hasOutput = data.hasOutput
hasShuffleRead = data.hasShuffleRead
hasShuffleWrite = data.hasShuffleWrite
hasBytesSpilled = data.hasBytesSpilled
- })
+ }
<table class={UIUtils.TABLE_CLASS_STRIPED_SORTABLE}>
<thead>
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 654d988807..bd4797ae8e 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -122,7 +122,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
| 'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' +
| '${
if (event.finishReason.isDefined) {
- s"""<br>Reason: ${event.finishReason.get}"""
+ s"""<br>Reason: ${event.finishReason.get.replace("\n", " ")}"""
} else {
""
}
@@ -203,7 +203,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
// This could be empty if the JobProgressListener hasn't received information about the
// stage or if the stage information has been garbage collected
listener.stageIdToInfo.getOrElse(stageId,
- new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown"))
+ new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown", Seq.empty))
}
val activeStages = Buffer[StageInfo]()
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index ed3ab66e3b..13f5f84d06 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -396,13 +396,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
None
}
taskMetrics.foreach { m =>
- val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics)
+ val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics)
updateAggregateMetrics(stageData, info.executorId, m, oldMetrics)
}
val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info))
taskData.taskInfo = info
- taskData.taskMetrics = taskMetrics
+ taskData.metrics = taskMetrics
taskData.errorMessage = errorMessage
for (
@@ -506,9 +506,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
val metrics = TaskMetrics.fromAccumulatorUpdates(accumUpdates)
taskData.foreach { t =>
if (!t.taskInfo.finished) {
- updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.taskMetrics)
+ updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics)
// Overwrite task metrics
- t.taskMetrics = Some(metrics)
+ t.metrics = Some(metrics)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index 0d0e9b00d3..7b00b558d5 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -31,6 +31,8 @@ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") {
def isFairScheduler: Boolean =
jobProgresslistener.schedulingMode == Some(SchedulingMode.FAIR)
+ def getSparkUser: String = parent.getSparkUser
+
attachPage(new AllJobsPage(this))
attachPage(new JobPage(this))
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 689ab7dd5e..8a44bbd9fc 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -330,7 +330,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
else taskTable.dataSource.slicedTaskIds
// Excludes tasks which failed and have incomplete metrics
- val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined)
+ val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined)
val summaryTable: Option[Seq[Node]] =
if (validTasks.size == 0) {
@@ -348,8 +348,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
getDistributionQuantiles(data).map(d => <td>{Utils.bytesToString(d.toLong)}</td>)
}
- val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.executorDeserializeTime.toDouble
+ val deserializationTimes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.executorDeserializeTime.toDouble
}
val deserializationQuantiles =
<td>
@@ -359,13 +359,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</span>
</td> +: getFormattedTimeQuantiles(deserializationTimes)
- val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.executorRunTime.toDouble
+ val serviceTimes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.executorRunTime.toDouble
}
val serviceQuantiles = <td>Duration</td> +: getFormattedTimeQuantiles(serviceTimes)
- val gcTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.jvmGCTime.toDouble
+ val gcTimes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.jvmGCTime.toDouble
}
val gcQuantiles =
<td>
@@ -374,8 +374,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</span>
</td> +: getFormattedTimeQuantiles(gcTimes)
- val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.resultSerializationTime.toDouble
+ val serializationTimes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.resultSerializationTime.toDouble
}
val serializationQuantiles =
<td>
@@ -385,8 +385,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</span>
</td> +: getFormattedTimeQuantiles(serializationTimes)
- val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) =>
- getGettingResultTime(info, currentTime).toDouble
+ val gettingResultTimes = validTasks.map { taskUIData: TaskUIData =>
+ getGettingResultTime(taskUIData.taskInfo, currentTime).toDouble
}
val gettingResultQuantiles =
<td>
@@ -397,8 +397,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</td> +:
getFormattedTimeQuantiles(gettingResultTimes)
- val peakExecutionMemory = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.peakExecutionMemory.toDouble
+ val peakExecutionMemory = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.peakExecutionMemory.toDouble
}
val peakExecutionMemoryQuantiles = {
<td>
@@ -412,8 +412,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
// The scheduler delay includes the network delay to send the task to the worker
// machine and to send back the result (but not the time to fetch the task result,
// if it needed to be fetched from the block manager on the worker).
- val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) =>
- getSchedulerDelay(info, metrics.get, currentTime).toDouble
+ val schedulerDelays = validTasks.map { taskUIData: TaskUIData =>
+ getSchedulerDelay(taskUIData.taskInfo, taskUIData.metrics.get, currentTime).toDouble
}
val schedulerDelayTitle = <td><span data-toggle="tooltip"
title={ToolTips.SCHEDULER_DELAY} data-placement="right">Scheduler Delay</span></td>
@@ -427,30 +427,30 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
)
}
- val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble
+ val inputSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble
}
- val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble
+ val inputRecords = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble
}
val inputQuantiles = <td>Input Size / Records</td> +:
getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords)
- val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble
+ val outputSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble
}
- val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble
+ val outputRecords = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble
}
val outputQuantiles = <td>Output Size / Records</td> +:
getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords)
- val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble
+ val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble
}
val shuffleReadBlockedQuantiles =
<td>
@@ -461,11 +461,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</td> +:
getFormattedTimeQuantiles(shuffleReadBlockedTimes)
- val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble
+ val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble
}
- val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble
+ val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble
}
val shuffleReadTotalQuantiles =
<td>
@@ -476,8 +476,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</td> +:
getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords)
- val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble
+ val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble
}
val shuffleReadRemoteQuantiles =
<td>
@@ -488,25 +488,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
</td> +:
getFormattedSizeQuantiles(shuffleReadRemoteSizes)
- val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L).toDouble
+ val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L).toDouble
}
- val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L).toDouble
+ val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L).toDouble
}
val shuffleWriteQuantiles = <td>Shuffle Write Size / Records</td> +:
getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords)
- val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.memoryBytesSpilled.toDouble
+ val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.memoryBytesSpilled.toDouble
}
val memoryBytesSpilledQuantiles = <td>Shuffle spill (memory)</td> +:
getFormattedSizeQuantiles(memoryBytesSpilledSizes)
- val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) =>
- metrics.get.diskBytesSpilled.toDouble
+ val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData =>
+ taskUIData.metrics.get.diskBytesSpilled.toDouble
}
val diskBytesSpilledQuantiles = <td>Shuffle spill (disk)</td> +:
getFormattedSizeQuantiles(diskBytesSpilledSizes)
@@ -601,7 +601,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
def toProportion(time: Long) = time.toDouble / totalExecutionTime * 100
- val metricsOpt = taskUIData.taskMetrics
+ val metricsOpt = taskUIData.metrics
val shuffleReadTime =
metricsOpt.flatMap(_.shuffleReadMetrics.map(_.fetchWaitTime)).getOrElse(0L)
val shuffleReadTimeProportion = toProportion(shuffleReadTime)
@@ -868,7 +868,8 @@ private[ui] class TaskDataSource(
def slicedTaskIds: Set[Long] = _slicedTaskIds
private def taskRow(taskData: TaskUIData): TaskTableRowData = {
- val TaskUIData(info, metrics, errorMessage) = taskData
+ val info = taskData.taskInfo
+ val metrics = taskData.metrics
val duration = if (info.status == "RUNNING") info.timeRunning(currentTime)
else metrics.map(_.executorRunTime).getOrElse(1L)
val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration)
@@ -1014,7 +1015,7 @@ private[ui] class TaskDataSource(
shuffleRead,
shuffleWrite,
bytesSpilled,
- errorMessage.getOrElse(""))
+ taskData.errorMessage.getOrElse(""))
}
/**
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
index 78165d7b74..b454ef1b20 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
@@ -105,12 +105,12 @@ private[spark] object UIData {
/**
* These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation.
*/
- case class TaskUIData(
+ class TaskUIData(
var taskInfo: TaskInfo,
- var taskMetrics: Option[TaskMetrics] = None,
+ var metrics: Option[TaskMetrics] = None,
var errorMessage: Option[String] = None)
- case class ExecutorUIData(
+ class ExecutorUIData(
val startTime: Long,
var finishTime: Option[Long] = None,
var finishReason: Option[String] = None)
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
index 8f75b586e1..50095831b4 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
@@ -57,17 +57,6 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc
StorageUtils.updateRddInfo(rddInfosToUpdate, activeStorageStatusList)
}
- /**
- * Assumes the storage status list is fully up-to-date. This implies the corresponding
- * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener.
- */
- override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
- val metrics = taskEnd.taskMetrics
- if (metrics != null && metrics.updatedBlockStatuses.nonEmpty) {
- updateRDDInfo(metrics.updatedBlockStatuses)
- }
- }
-
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
val rddInfos = stageSubmitted.stageInfo.rddInfos
rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) }
@@ -84,4 +73,14 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc
override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized {
_rddInfoMap.remove(unpersistRDD.rddId)
}
+
+ override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = {
+ super.onBlockUpdated(blockUpdated)
+ val blockId = blockUpdated.blockUpdatedInfo.blockId
+ val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel
+ val memSize = blockUpdated.blockUpdatedInfo.memSize
+ val diskSize = blockUpdated.blockUpdatedInfo.diskSize
+ val blockStatus = BlockStatus(storageLevel, memSize, diskSize)
+ updateRDDInfo(Seq((blockId, blockStatus)))
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/util/CausedBy.scala b/core/src/main/scala/org/apache/spark/util/CausedBy.scala
new file mode 100644
index 0000000000..73df446d98
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/CausedBy.scala
@@ -0,0 +1,36 @@
+/*
+ * 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.util
+
+/**
+ * Extractor Object for pulling out the root cause of an error.
+ * If the error contains no cause, it will return the error itself.
+ *
+ * Usage:
+ * try {
+ * ...
+ * } catch {
+ * case CausedBy(ex: CommitDeniedException) => ...
+ * }
+ */
+private[spark] object CausedBy {
+
+ def unapply(e: Throwable): Option[Throwable] = {
+ Option(e.getCause).flatMap(cause => unapply(cause)).orElse(Some(e))
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index f4772a9803..489688cb08 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -19,7 +19,8 @@ package org.apache.spark.util
import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
-import scala.collection.mutable.{Map, Set}
+import scala.collection.mutable.{Map, Set, Stack}
+import scala.language.existentials
import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor, Type}
import org.apache.xbean.asm5.Opcodes._
@@ -77,15 +78,14 @@ private[spark] object ClosureCleaner extends Logging {
*/
private def getInnerClosureClasses(obj: AnyRef): List[Class[_]] = {
val seen = Set[Class[_]](obj.getClass)
- var stack = List[Class[_]](obj.getClass)
+ val stack = Stack[Class[_]](obj.getClass)
while (!stack.isEmpty) {
- val cr = getClassReader(stack.head)
- stack = stack.tail
+ val cr = getClassReader(stack.pop())
val set = Set[Class[_]]()
cr.accept(new InnerClosureFinder(set), 0)
for (cls <- set -- seen) {
seen += cls
- stack = cls :: stack
+ stack.push(cls)
}
}
(seen - obj.getClass).toList
@@ -218,16 +218,24 @@ private[spark] object ClosureCleaner extends Logging {
// Note that all outer objects but the outermost one (first one in this list) must be closures
var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse
var parent: AnyRef = null
- if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) {
- // The closure is ultimately nested inside a class; keep the object of that
- // class without cloning it since we don't want to clone the user's objects.
- // Note that we still need to keep around the outermost object itself because
- // we need it to clone its child closure later (see below).
- logDebug(s" + outermost object is not a closure, so do not clone it: ${outerPairs.head}")
- parent = outerPairs.head._2 // e.g. SparkContext
- outerPairs = outerPairs.tail
- } else if (outerPairs.size > 0) {
- logDebug(s" + outermost object is a closure, so we just keep it: ${outerPairs.head}")
+ if (outerPairs.size > 0) {
+ val (outermostClass, outermostObject) = outerPairs.head
+ if (isClosure(outermostClass)) {
+ logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}")
+ } else if (outermostClass.getName.startsWith("$line")) {
+ // SPARK-14558: if the outermost object is a REPL line object, we should clone and clean it
+ // as it may carray a lot of unnecessary information, e.g. hadoop conf, spark conf, etc.
+ logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}")
+ } else {
+ // The closure is ultimately nested inside a class; keep the object of that
+ // class without cloning it since we don't want to clone the user's objects.
+ // Note that we still need to keep around the outermost object itself because
+ // we need it to clone its child closure later (see below).
+ logDebug(" + outermost object is not a closure or REPL line object, so do not clone it: " +
+ outerPairs.head)
+ parent = outermostObject // e.g. SparkContext
+ outerPairs = outerPairs.tail
+ }
} else {
logDebug(" + there are no enclosing objects!")
}
diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala
index 153025cef2..3ea9139e11 100644
--- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala
+++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala
@@ -47,13 +47,12 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging {
try {
onReceive(event)
} catch {
- case NonFatal(e) => {
+ case NonFatal(e) =>
try {
onError(e)
} catch {
case NonFatal(e) => logError("Unexpected error in " + name, e)
}
- }
}
}
} catch {
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 09d955300a..558767e36f 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -578,7 +578,9 @@ private[spark] object JsonProtocol {
// The "Stage Infos" field was added in Spark 1.2.0
val stageInfos = Utils.jsonOption(json \ "Stage Infos")
.map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse {
- stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown"))
+ stageIds.map { id =>
+ new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", Seq.empty)
+ }
}
SparkListenerJobStart(jobId, submissionTime, stageInfos, properties)
}
@@ -686,7 +688,7 @@ private[spark] object JsonProtocol {
}
val stageInfo = new StageInfo(
- stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details)
+ stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details, Seq.empty)
stageInfo.submissionTime = submissionTime
stageInfo.completionTime = completionTime
stageInfo.failureReason = failureReason
@@ -811,8 +813,8 @@ private[spark] object JsonProtocol {
Utils.jsonOption(json \ "Input Metrics").foreach { inJson =>
val readMethod = DataReadMethod.withName((inJson \ "Data Read Method").extract[String])
val inputMetrics = metrics.registerInputMetrics(readMethod)
- inputMetrics.incBytesReadInternal((inJson \ "Bytes Read").extract[Long])
- inputMetrics.incRecordsReadInternal((inJson \ "Records Read").extractOpt[Long].getOrElse(0L))
+ inputMetrics.incBytesRead((inJson \ "Bytes Read").extract[Long])
+ inputMetrics.incRecordsRead((inJson \ "Records Read").extractOpt[Long].getOrElse(0L))
}
// Updated blocks
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index 3f627a0145..6861a75612 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -151,13 +151,12 @@ object SizeEstimator extends Logging {
// TODO: We could use reflection on the VMOption returned ?
getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true")
} catch {
- case e: Exception => {
+ case e: Exception =>
// Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB
val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024)
val guessInWords = if (guess) "yes" else "not"
logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords)
return guess
- }
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
new file mode 100644
index 0000000000..4dcf95177a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
@@ -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.util
+
+import javax.annotation.concurrent.GuardedBy
+
+/**
+ * A special Thread that provides "runUninterruptibly" to allow running codes without being
+ * interrupted by `Thread.interrupt()`. If `Thread.interrupt()` is called during runUninterruptibly
+ * is running, it won't set the interrupted status. Instead, setting the interrupted status will be
+ * deferred until it's returning from "runUninterruptibly".
+ *
+ * Note: "runUninterruptibly" should be called only in `this` thread.
+ */
+private[spark] class UninterruptibleThread(name: String) extends Thread(name) {
+
+ /** A monitor to protect "uninterruptible" and "interrupted" */
+ private val uninterruptibleLock = new Object
+
+ /**
+ * Indicates if `this` thread are in the uninterruptible status. If so, interrupting
+ * "this" will be deferred until `this` enters into the interruptible status.
+ */
+ @GuardedBy("uninterruptibleLock")
+ private var uninterruptible = false
+
+ /**
+ * Indicates if we should interrupt `this` when we are leaving the uninterruptible zone.
+ */
+ @GuardedBy("uninterruptibleLock")
+ private var shouldInterruptThread = false
+
+ /**
+ * Run `f` uninterruptibly in `this` thread. The thread won't be interrupted before returning
+ * from `f`.
+ *
+ * If this method finds that `interrupt` is called before calling `f` and it's not inside another
+ * `runUninterruptibly`, it will throw `InterruptedException`.
+ *
+ * Note: this method should be called only in `this` thread.
+ */
+ def runUninterruptibly[T](f: => T): T = {
+ if (Thread.currentThread() != this) {
+ throw new IllegalStateException(s"Call runUninterruptibly in a wrong thread. " +
+ s"Expected: $this but was ${Thread.currentThread()}")
+ }
+
+ if (uninterruptibleLock.synchronized { uninterruptible }) {
+ // We are already in the uninterruptible status. So just run "f" and return
+ return f
+ }
+
+ uninterruptibleLock.synchronized {
+ // Clear the interrupted status if it's set.
+ if (Thread.interrupted() || shouldInterruptThread) {
+ shouldInterruptThread = false
+ // Since it's interrupted, we don't need to run `f` which may be a long computation.
+ // Throw InterruptedException as we don't have a T to return.
+ throw new InterruptedException()
+ }
+ uninterruptible = true
+ }
+ try {
+ f
+ } finally {
+ uninterruptibleLock.synchronized {
+ uninterruptible = false
+ if (shouldInterruptThread) {
+ // Recover the interrupted status
+ super.interrupt()
+ shouldInterruptThread = false
+ }
+ }
+ }
+ }
+
+ /**
+ * Tests whether `interrupt()` has been called.
+ */
+ override def isInterrupted: Boolean = {
+ super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread }
+ }
+
+ /**
+ * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be
+ * interrupted until it enters into the interruptible status.
+ */
+ override def interrupt(): Unit = {
+ uninterruptibleLock.synchronized {
+ if (uninterruptible) {
+ shouldInterruptThread = true
+ } else {
+ super.interrupt()
+ }
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 73768ff4c8..78e164cff7 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -256,10 +256,11 @@ private[spark] object Utils extends Logging {
dir
}
- /** Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream
- * copying is disabled by default unless explicitly set transferToEnabled as true,
- * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false].
- */
+ /**
+ * Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream
+ * copying is disabled by default unless explicitly set transferToEnabled as true,
+ * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false].
+ */
def copyStream(in: InputStream,
out: OutputStream,
closeStreams: Boolean = false,
@@ -1120,9 +1121,9 @@ private[spark] object Utils extends Logging {
extraEnvironment: Map[String, String] = Map.empty,
redirectStderr: Boolean = true): String = {
val process = executeCommand(command, workingDir, extraEnvironment, redirectStderr)
- val output = new StringBuffer
+ val output = new StringBuilder
val threadName = "read stdout for " + command(0)
- def appendToOutput(s: String): Unit = output.append(s)
+ def appendToOutput(s: String): Unit = output.append(s).append("\n")
val stdoutThread = processStreamByLine(threadName, process.getInputStream, appendToOutput)
val exitCode = process.waitFor()
stdoutThread.join() // Wait for it to finish reading output
@@ -1259,26 +1260,35 @@ private[spark] object Utils extends Logging {
}
/**
- * Execute a block of code, call the failure callbacks before finally block if there is any
- * exceptions happen. But if exceptions happen in the finally block, do not suppress the original
- * exception.
+ * Execute a block of code and call the failure callbacks in the catch block. If exceptions occur
+ * in either the catch or the finally block, they are appended to the list of suppressed
+ * exceptions in original exception which is then rethrown.
*
- * This is primarily an issue with `finally { out.close() }` blocks, where
- * close needs to be called to clean up `out`, but if an exception happened
- * in `out.write`, it's likely `out` may be corrupted and `out.close` will
+ * This is primarily an issue with `catch { abort() }` or `finally { out.close() }` blocks,
+ * where the abort/close needs to be called to clean up `out`, but if an exception happened
+ * in `out.write`, it's likely `out` may be corrupted and `abort` or `out.close` will
* fail as well. This would then suppress the original/likely more meaningful
* exception from the original `out.write` call.
*/
- def tryWithSafeFinallyAndFailureCallbacks[T](block: => T)(finallyBlock: => Unit): T = {
+ def tryWithSafeFinallyAndFailureCallbacks[T](block: => T)
+ (catchBlock: => Unit = (), finallyBlock: => Unit = ()): T = {
var originalThrowable: Throwable = null
try {
block
} catch {
- case t: Throwable =>
+ case cause: Throwable =>
// Purposefully not using NonFatal, because even fatal exceptions
// we don't want to have our finallyBlock suppress
- originalThrowable = t
- TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(t)
+ originalThrowable = cause
+ try {
+ logError("Aborting task", originalThrowable)
+ TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(originalThrowable)
+ catchBlock
+ } catch {
+ case t: Throwable =>
+ originalThrowable.addSuppressed(t)
+ logWarning(s"Suppressing exception in catch: " + t.getMessage, t)
+ }
throw originalThrowable
} finally {
try {
@@ -1564,9 +1574,11 @@ private[spark] object Utils extends Logging {
else -1
}
- /** Returns the system properties map that is thread-safe to iterator over. It gets the
- * properties which have been set explicitly, as well as those for which only a default value
- * has been defined. */
+ /**
+ * Returns the system properties map that is thread-safe to iterator over. It gets the
+ * properties which have been set explicitly, as well as those for which only a default value
+ * has been defined.
+ */
def getSystemProperties: Map[String, String] = {
System.getProperties.stringPropertyNames().asScala
.map(key => (key, System.getProperty(key))).toMap
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index c643c4b63c..fb4706e78d 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -41,6 +41,8 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
require(chunks.forall(_.limit() > 0), "chunks must be non-empty")
require(chunks.forall(_.position() == 0), "chunks' positions must be 0")
+ private[this] var disposed: Boolean = false
+
/**
* This size of this buffer, in bytes.
*/
@@ -117,11 +119,12 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
/**
* Make a copy of this ChunkedByteBuffer, copying all of the backing data into new buffers.
* The new buffer will share no resources with the original buffer.
+ *
+ * @param allocator a method for allocating byte buffers
*/
- def copy(): ChunkedByteBuffer = {
+ def copy(allocator: Int => ByteBuffer): ChunkedByteBuffer = {
val copiedChunks = getChunks().map { chunk =>
- // TODO: accept an allocator in this copy method to integrate with mem. accounting systems
- val newChunk = ByteBuffer.allocate(chunk.limit())
+ val newChunk = allocator(chunk.limit())
newChunk.put(chunk)
newChunk.flip()
newChunk
@@ -136,7 +139,10 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
* unfortunately no standard API to do this.
*/
def dispose(): Unit = {
- chunks.foreach(StorageUtils.dispose)
+ if (!disposed) {
+ chunks.foreach(StorageUtils.dispose)
+ disposed = true
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala
index 16fe3be303..67b50d1e70 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala
@@ -18,19 +18,25 @@
package org.apache.spark.util.io
import java.io.OutputStream
+import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.storage.StorageUtils
/**
* An OutputStream that writes to fixed-size chunks of byte arrays.
*
* @param chunkSize size of each chunk, in bytes.
*/
-private[spark]
-class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
+private[spark] class ChunkedByteBufferOutputStream(
+ chunkSize: Int,
+ allocator: Int => ByteBuffer)
+ extends OutputStream {
- private[this] val chunks = new ArrayBuffer[Array[Byte]]
+ private[this] var toChunkedByteBufferWasCalled = false
+
+ private val chunks = new ArrayBuffer[ByteBuffer]
/** Index of the last chunk. Starting with -1 when the chunks array is empty. */
private[this] var lastChunkIndex = -1
@@ -48,7 +54,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
override def write(b: Int): Unit = {
allocateNewChunkIfNeeded()
- chunks(lastChunkIndex)(position) = b.toByte
+ chunks(lastChunkIndex).put(b.toByte)
position += 1
_size += 1
}
@@ -58,7 +64,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
while (written < len) {
allocateNewChunkIfNeeded()
val thisBatch = math.min(chunkSize - position, len - written)
- System.arraycopy(bytes, written + off, chunks(lastChunkIndex), position, thisBatch)
+ chunks(lastChunkIndex).put(bytes, written + off, thisBatch)
written += thisBatch
position += thisBatch
}
@@ -67,33 +73,41 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream {
@inline
private def allocateNewChunkIfNeeded(): Unit = {
+ require(!toChunkedByteBufferWasCalled, "cannot write after toChunkedByteBuffer() is called")
if (position == chunkSize) {
- chunks += new Array[Byte](chunkSize)
+ chunks += allocator(chunkSize)
lastChunkIndex += 1
position = 0
}
}
- def toArrays: Array[Array[Byte]] = {
+ def toChunkedByteBuffer: ChunkedByteBuffer = {
+ require(!toChunkedByteBufferWasCalled, "toChunkedByteBuffer() can only be called once")
+ toChunkedByteBufferWasCalled = true
if (lastChunkIndex == -1) {
- new Array[Array[Byte]](0)
+ new ChunkedByteBuffer(Array.empty[ByteBuffer])
} else {
// Copy the first n-1 chunks to the output, and then create an array that fits the last chunk.
// An alternative would have been returning an array of ByteBuffers, with the last buffer
// bounded to only the last chunk's position. However, given our use case in Spark (to put
// the chunks in block manager), only limiting the view bound of the buffer would still
// require the block manager to store the whole chunk.
- val ret = new Array[Array[Byte]](chunks.size)
+ val ret = new Array[ByteBuffer](chunks.size)
for (i <- 0 until chunks.size - 1) {
ret(i) = chunks(i)
+ ret(i).flip()
}
if (position == chunkSize) {
ret(lastChunkIndex) = chunks(lastChunkIndex)
+ ret(lastChunkIndex).flip()
} else {
- ret(lastChunkIndex) = new Array[Byte](position)
- System.arraycopy(chunks(lastChunkIndex), 0, ret(lastChunkIndex), 0, position)
+ ret(lastChunkIndex) = allocator(position)
+ chunks(lastChunkIndex).flip()
+ ret(lastChunkIndex).put(chunks(lastChunkIndex))
+ ret(lastChunkIndex).flip()
+ StorageUtils.dispose(chunks(lastChunkIndex))
}
- ret
+ new ChunkedByteBuffer(ret)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
index b34880d3a7..6e80db2f51 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
@@ -32,10 +32,10 @@ private[spark] trait RollingPolicy {
def shouldRollover(bytesToBeWritten: Long): Boolean
/** Notify that rollover has occurred */
- def rolledOver()
+ def rolledOver(): Unit
/** Notify that bytes have been written */
- def bytesWritten(bytes: Long)
+ def bytesWritten(bytes: Long): Unit
/** Get the desired name of the rollover file */
def generateRolledOverFileSuffix(): String
diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
index 70f3dd62b9..41f28f6e51 100644
--- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala
@@ -26,5 +26,5 @@ import org.apache.spark.annotation.DeveloperApi
@DeveloperApi
trait Pseudorandom {
/** Set random seed. */
- def setSeed(seed: Long)
+ def setSeed(seed: Long): Unit
}
diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
index 3c61528ab5..8c67364ef1 100644
--- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
@@ -39,7 +39,14 @@ import org.apache.spark.annotation.DeveloperApi
trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable {
/** take a random sample */
- def sample(items: Iterator[T]): Iterator[U]
+ def sample(items: Iterator[T]): Iterator[U] =
+ items.filter(_ => sample > 0).asInstanceOf[Iterator[U]]
+
+ /**
+ * Whether to sample the next item or not.
+ * Return how many times the next item will be sampled. Return 0 if it is not sampled.
+ */
+ def sample(): Int
/** return a copy of the RandomSampler object */
override def clone: RandomSampler[T, U] =
@@ -107,21 +114,13 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals
override def setSeed(seed: Long): Unit = rng.setSeed(seed)
- override def sample(items: Iterator[T]): Iterator[T] = {
+ override def sample(): Int = {
if (ub - lb <= 0.0) {
- if (complement) items else Iterator.empty
+ if (complement) 1 else 0
} else {
- if (complement) {
- items.filter { item => {
- val x = rng.nextDouble()
- (x < lb) || (x >= ub)
- }}
- } else {
- items.filter { item => {
- val x = rng.nextDouble()
- (x >= lb) && (x < ub)
- }}
- }
+ val x = rng.nextDouble()
+ val n = if ((x >= lb) && (x < ub)) 1 else 0
+ if (complement) 1 - n else n
}
}
@@ -155,15 +154,22 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T
override def setSeed(seed: Long): Unit = rng.setSeed(seed)
- override def sample(items: Iterator[T]): Iterator[T] = {
+ private lazy val gapSampling: GapSampling =
+ new GapSampling(fraction, rng, RandomSampler.rngEpsilon)
+
+ override def sample(): Int = {
if (fraction <= 0.0) {
- Iterator.empty
+ 0
} else if (fraction >= 1.0) {
- items
+ 1
} else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) {
- new GapSamplingIterator(items, fraction, rng, RandomSampler.rngEpsilon)
+ gapSampling.sample()
} else {
- items.filter { _ => rng.nextDouble() <= fraction }
+ if (rng.nextDouble() <= fraction) {
+ 1
+ } else {
+ 0
+ }
}
}
@@ -180,7 +186,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T
* @tparam T item type
*/
@DeveloperApi
-class PoissonSampler[T: ClassTag](
+class PoissonSampler[T](
fraction: Double,
useGapSamplingIfPossible: Boolean) extends RandomSampler[T, T] {
@@ -201,15 +207,29 @@ class PoissonSampler[T: ClassTag](
rngGap.setSeed(seed)
}
- override def sample(items: Iterator[T]): Iterator[T] = {
+ private lazy val gapSamplingReplacement =
+ new GapSamplingReplacement(fraction, rngGap, RandomSampler.rngEpsilon)
+
+ override def sample(): Int = {
if (fraction <= 0.0) {
- Iterator.empty
+ 0
} else if (useGapSamplingIfPossible &&
fraction <= RandomSampler.defaultMaxGapSamplingFraction) {
- new GapSamplingReplacementIterator(items, fraction, rngGap, RandomSampler.rngEpsilon)
+ gapSamplingReplacement.sample()
} else {
+ rng.sample()
+ }
+ }
+
+ override def sample(items: Iterator[T]): Iterator[T] = {
+ if (fraction <= 0.0) {
+ Iterator.empty
+ } else {
+ val useGapSampling = useGapSamplingIfPossible &&
+ fraction <= RandomSampler.defaultMaxGapSamplingFraction
+
items.flatMap { item =>
- val count = rng.sample()
+ val count = if (useGapSampling) gapSamplingReplacement.sample() else rng.sample()
if (count == 0) Iterator.empty else Iterator.fill(count)(item)
}
}
@@ -220,50 +240,36 @@ class PoissonSampler[T: ClassTag](
private[spark]
-class GapSamplingIterator[T: ClassTag](
- var data: Iterator[T],
+class GapSampling(
f: Double,
rng: Random = RandomSampler.newDefaultRNG,
- epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] {
+ epsilon: Double = RandomSampler.rngEpsilon) extends Serializable {
require(f > 0.0 && f < 1.0, s"Sampling fraction ($f) must reside on open interval (0, 1)")
require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0")
- /** implement efficient linear-sequence drop until Scala includes fix for jira SI-8835. */
- private val iterDrop: Int => Unit = {
- val arrayClass = Array.empty[T].iterator.getClass
- val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass
- data.getClass match {
- case `arrayClass` =>
- (n: Int) => { data = data.drop(n) }
- case `arrayBufferClass` =>
- (n: Int) => { data = data.drop(n) }
- case _ =>
- (n: Int) => {
- var j = 0
- while (j < n && data.hasNext) {
- data.next()
- j += 1
- }
- }
- }
- }
-
- override def hasNext: Boolean = data.hasNext
+ private val lnq = math.log1p(-f)
- override def next(): T = {
- val r = data.next()
- advance()
- r
+ /** Return 1 if the next item should be sampled. Otherwise, return 0. */
+ def sample(): Int = {
+ if (countForDropping > 0) {
+ countForDropping -= 1
+ 0
+ } else {
+ advance()
+ 1
+ }
}
- private val lnq = math.log1p(-f)
+ private var countForDropping: Int = 0
- /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */
+ /**
+ * Decide the number of elements that won't be sampled,
+ * according to geometric dist P(k) = (f)(1-f)^k.
+ */
private def advance(): Unit = {
val u = math.max(rng.nextDouble(), epsilon)
- val k = (math.log(u) / lnq).toInt
- iterDrop(k)
+ countForDropping = (math.log(u) / lnq).toInt
}
/** advance to first sample as part of object construction. */
@@ -273,73 +279,24 @@ class GapSamplingIterator[T: ClassTag](
// work reliably.
}
+
private[spark]
-class GapSamplingReplacementIterator[T: ClassTag](
- var data: Iterator[T],
- f: Double,
- rng: Random = RandomSampler.newDefaultRNG,
- epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] {
+class GapSamplingReplacement(
+ val f: Double,
+ val rng: Random = RandomSampler.newDefaultRNG,
+ epsilon: Double = RandomSampler.rngEpsilon) extends Serializable {
require(f > 0.0, s"Sampling fraction ($f) must be > 0")
require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0")
- /** implement efficient linear-sequence drop until scala includes fix for jira SI-8835. */
- private val iterDrop: Int => Unit = {
- val arrayClass = Array.empty[T].iterator.getClass
- val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass
- data.getClass match {
- case `arrayClass` =>
- (n: Int) => { data = data.drop(n) }
- case `arrayBufferClass` =>
- (n: Int) => { data = data.drop(n) }
- case _ =>
- (n: Int) => {
- var j = 0
- while (j < n && data.hasNext) {
- data.next()
- j += 1
- }
- }
- }
- }
-
- /** current sampling value, and its replication factor, as we are sampling with replacement. */
- private var v: T = _
- private var rep: Int = 0
-
- override def hasNext: Boolean = data.hasNext || rep > 0
-
- override def next(): T = {
- val r = v
- rep -= 1
- if (rep <= 0) advance()
- r
- }
-
- /**
- * Skip elements with replication factor zero (i.e. elements that won't be sampled).
- * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is
- * q is the probability of Poisson(0; f)
- */
- private def advance(): Unit = {
- val u = math.max(rng.nextDouble(), epsilon)
- val k = (math.log(u) / (-f)).toInt
- iterDrop(k)
- // set the value and replication factor for the next value
- if (data.hasNext) {
- v = data.next()
- rep = poissonGE1
- }
- }
-
- private val q = math.exp(-f)
+ protected val q = math.exp(-f)
/**
* Sample from Poisson distribution, conditioned such that the sampled value is >= 1.
* This is an adaptation from the algorithm for Generating Poisson distributed random variables:
* http://en.wikipedia.org/wiki/Poisson_distribution
*/
- private def poissonGE1: Int = {
+ protected def poissonGE1: Int = {
// simulate that the standard poisson sampling
// gave us at least one iteration, for a sample of >= 1
var pp = q + ((1.0 - q) * rng.nextDouble())
@@ -353,6 +310,28 @@ class GapSamplingReplacementIterator[T: ClassTag](
}
r
}
+ private var countForDropping: Int = 0
+
+ def sample(): Int = {
+ if (countForDropping > 0) {
+ countForDropping -= 1
+ 0
+ } else {
+ val r = poissonGE1
+ advance()
+ r
+ }
+ }
+
+ /**
+ * Skip elements with replication factor zero (i.e. elements that won't be sampled).
+ * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is
+ * q is the probability of Poisson(0; f)
+ */
+ private def advance(): Unit = {
+ val u = math.max(rng.nextDouble(), epsilon)
+ countForDropping = (math.log(u) / (-f)).toInt
+ }
/** advance to first sample as part of object construction. */
advance()
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index 44733dcdaf..30750b1bf1 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -170,11 +170,11 @@ public class UnsafeShuffleWriterSuite {
private UnsafeShuffleWriter<Object, Object> createWriter(
boolean transferToEnabled) throws IOException {
conf.set("spark.file.transferTo", String.valueOf(transferToEnabled));
- return new UnsafeShuffleWriter<Object, Object>(
+ return new UnsafeShuffleWriter<>(
blockManager,
shuffleBlockResolver,
taskMemoryManager,
- new SerializedShuffleHandle<Object, Object>(0, 1, shuffleDep),
+ new SerializedShuffleHandle<>(0, 1, shuffleDep),
0, // map id
taskContext,
conf
diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
index 449fb45c30..84b82f5a47 100644
--- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
+++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
@@ -182,7 +182,7 @@ public abstract class AbstractBytesToBytesMapSuite {
public void emptyMap() {
BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 64, PAGE_SIZE_BYTES);
try {
- Assert.assertEquals(0, map.numElements());
+ Assert.assertEquals(0, map.numKeys());
final int keyLengthInWords = 10;
final int keyLengthInBytes = keyLengthInWords * 8;
final byte[] key = getRandomByteArray(keyLengthInWords);
@@ -204,7 +204,7 @@ public abstract class AbstractBytesToBytesMapSuite {
final BytesToBytesMap.Location loc =
map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes);
Assert.assertFalse(loc.isDefined());
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
keyData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
@@ -232,7 +232,7 @@ public abstract class AbstractBytesToBytesMapSuite {
getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes));
try {
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
keyData,
Platform.BYTE_ARRAY_OFFSET,
recordLengthBytes,
@@ -260,7 +260,7 @@ public abstract class AbstractBytesToBytesMapSuite {
Assert.assertFalse(loc.isDefined());
// Ensure that we store some zero-length keys
if (i % 5 == 0) {
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
null,
Platform.LONG_ARRAY_OFFSET,
0,
@@ -269,7 +269,7 @@ public abstract class AbstractBytesToBytesMapSuite {
8
));
} else {
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
value,
Platform.LONG_ARRAY_OFFSET,
8,
@@ -349,7 +349,7 @@ public abstract class AbstractBytesToBytesMapSuite {
KEY_LENGTH
);
Assert.assertFalse(loc.isDefined());
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
key,
Platform.LONG_ARRAY_OFFSET,
KEY_LENGTH,
@@ -417,7 +417,7 @@ public abstract class AbstractBytesToBytesMapSuite {
key.length
);
Assert.assertFalse(loc.isDefined());
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length,
@@ -471,7 +471,7 @@ public abstract class AbstractBytesToBytesMapSuite {
key.length
);
Assert.assertFalse(loc.isDefined());
- Assert.assertTrue(loc.putNewKey(
+ Assert.assertTrue(loc.append(
key,
Platform.BYTE_ARRAY_OFFSET,
key.length,
@@ -514,7 +514,7 @@ public abstract class AbstractBytesToBytesMapSuite {
final BytesToBytesMap.Location loc =
map.lookup(emptyArray, Platform.LONG_ARRAY_OFFSET, 0);
Assert.assertFalse(loc.isDefined());
- Assert.assertFalse(loc.putNewKey(
+ Assert.assertFalse(loc.append(
emptyArray, Platform.LONG_ARRAY_OFFSET, 0, emptyArray, Platform.LONG_ARRAY_OFFSET, 0));
} finally {
map.free();
@@ -535,7 +535,7 @@ public abstract class AbstractBytesToBytesMapSuite {
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
success =
- loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
if (!success) {
break;
}
@@ -556,7 +556,7 @@ public abstract class AbstractBytesToBytesMapSuite {
for (i = 0; i < 1024; i++) {
final long[] arr = new long[]{i};
final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
- loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
}
BytesToBytesMap.MapIterator iter = map.iterator();
for (i = 0; i < 100; i++) {
@@ -587,6 +587,44 @@ public abstract class AbstractBytesToBytesMapSuite {
}
@Test
+ public void multipleValuesForSameKey() {
+ BytesToBytesMap map =
+ new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false);
+ try {
+ int i;
+ for (i = 0; i < 1024; i++) {
+ final long[] arr = new long[]{i};
+ map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8)
+ .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ }
+ assert map.numKeys() == 1024;
+ assert map.numValues() == 1024;
+ for (i = 0; i < 1024; i++) {
+ final long[] arr = new long[]{i};
+ map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8)
+ .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ }
+ assert map.numKeys() == 1024;
+ assert map.numValues() == 2048;
+ for (i = 0; i < 1024; i++) {
+ final long[] arr = new long[]{i};
+ final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
+ assert loc.isDefined();
+ assert loc.nextValue();
+ assert !loc.nextValue();
+ }
+ BytesToBytesMap.MapIterator iter = map.iterator();
+ for (i = 0; i < 2048; i++) {
+ assert iter.hasNext();
+ final BytesToBytesMap.Location loc = iter.next();
+ assert loc.isDefined();
+ }
+ } finally {
+ map.free();
+ }
+ }
+
+ @Test
public void initialCapacityBoundsChecking() {
try {
new BytesToBytesMap(taskMemoryManager, 0, PAGE_SIZE_BYTES);
@@ -608,7 +646,7 @@ public abstract class AbstractBytesToBytesMapSuite {
@Test
public void testPeakMemoryUsed() {
- final long recordLengthBytes = 24;
+ final long recordLengthBytes = 32;
final long pageSizeBytes = 256 + 8; // 8 bytes for end-of-page marker
final long numRecordsPerPage = (pageSizeBytes - 8) / recordLengthBytes;
final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 1024, pageSizeBytes);
@@ -622,7 +660,7 @@ public abstract class AbstractBytesToBytesMapSuite {
try {
for (long i = 0; i < numRecordsPerPage * 10; i++) {
final long[] value = new long[]{i};
- map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).putNewKey(
+ map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append(
value,
Platform.LONG_ARRAY_OFFSET,
8,
diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
index 5bbb4ceb97..1a13233133 100644
--- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1430917381534",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917391398,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:11.398GMT",
"lastUpdated" : "",
@@ -14,6 +17,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917380950,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:00.950GMT",
"lastUpdated" : "",
@@ -22,6 +28,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1430917380880,
+ "endTimeEpoch" : 1430917380890,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.880GMT",
"endTime" : "2015-05-06T13:03:00.890GMT",
"lastUpdated" : "",
@@ -34,6 +43,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1426633910242,
+ "endTimeEpoch" : 1426633945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-17T23:11:50.242GMT",
"endTime" : "2015-03-17T23:12:25.177GMT",
"lastUpdated" : "",
@@ -42,6 +54,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1426533910242,
+ "endTimeEpoch" : 1426533945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-16T19:25:10.242GMT",
"endTime" : "2015-03-16T19:25:45.177GMT",
"lastUpdated" : "",
@@ -53,6 +68,9 @@
"id" : "local-1425081759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1425081758277,
+ "endTimeEpoch" : 1425081766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-28T00:02:38.277GMT",
"endTime" : "2015-02-28T00:02:46.912GMT",
"lastUpdated" : "",
@@ -64,6 +82,9 @@
"id" : "local-1422981780767",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981779720,
+ "endTimeEpoch" : 1422981788731,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:59.720GMT",
"endTime" : "2015-02-03T16:43:08.731GMT",
"lastUpdated" : "",
@@ -75,6 +96,9 @@
"id" : "local-1422981759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981758277,
+ "endTimeEpoch" : 1422981766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:38.277GMT",
"endTime" : "2015-02-03T16:42:46.912GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
index 5bbb4ceb97..1a13233133 100644
--- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1430917381534",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917391398,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:11.398GMT",
"lastUpdated" : "",
@@ -14,6 +17,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917380950,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:00.950GMT",
"lastUpdated" : "",
@@ -22,6 +28,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1430917380880,
+ "endTimeEpoch" : 1430917380890,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.880GMT",
"endTime" : "2015-05-06T13:03:00.890GMT",
"lastUpdated" : "",
@@ -34,6 +43,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1426633910242,
+ "endTimeEpoch" : 1426633945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-17T23:11:50.242GMT",
"endTime" : "2015-03-17T23:12:25.177GMT",
"lastUpdated" : "",
@@ -42,6 +54,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1426533910242,
+ "endTimeEpoch" : 1426533945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-16T19:25:10.242GMT",
"endTime" : "2015-03-16T19:25:45.177GMT",
"lastUpdated" : "",
@@ -53,6 +68,9 @@
"id" : "local-1425081759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1425081758277,
+ "endTimeEpoch" : 1425081766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-28T00:02:38.277GMT",
"endTime" : "2015-02-28T00:02:46.912GMT",
"lastUpdated" : "",
@@ -64,6 +82,9 @@
"id" : "local-1422981780767",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981779720,
+ "endTimeEpoch" : 1422981788731,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:59.720GMT",
"endTime" : "2015-02-03T16:43:08.731GMT",
"lastUpdated" : "",
@@ -75,6 +96,9 @@
"id" : "local-1422981759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981758277,
+ "endTimeEpoch" : 1422981766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:38.277GMT",
"endTime" : "2015-02-03T16:42:46.912GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json
index 4a88eeee74..efc865919b 100644
--- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json
@@ -2,8 +2,8 @@
"id" : "<driver>",
"hostPort" : "localhost:57971",
"isActive" : true,
- "rddBlocks" : 8,
- "memoryUsed" : 28000128,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
"diskUsed" : 0,
"totalCores" : 0,
"maxTasks" : 0,
diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json
index 3f80a529a0..eacf04b901 100644
--- a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1422981759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981758277,
+ "endTimeEpoch" : 1422981766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:38.277GMT",
"endTime" : "2015-02-03T16:42:46.912GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json
index 508bdc17ef..adad25bf17 100644
--- a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1422981780767",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981779720,
+ "endTimeEpoch" : 1422981788731,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:59.720GMT",
"endTime" : "2015-02-03T16:43:08.731GMT",
"lastUpdated" : "",
@@ -13,6 +16,9 @@
"id" : "local-1422981759269",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981758277,
+ "endTimeEpoch" : 1422981766912,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:38.277GMT",
"endTime" : "2015-02-03T16:42:46.912GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
index 5dca7d73de..a658909088 100644
--- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1430917381534",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917391398,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:11.398GMT",
"lastUpdated" : "",
@@ -14,6 +17,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1430917380893,
+ "endTimeEpoch" : 1430917380950,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.893GMT",
"endTime" : "2015-05-06T13:03:00.950GMT",
"lastUpdated" : "",
@@ -22,6 +28,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1430917380880,
+ "endTimeEpoch" : 1430917380890,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-05-06T13:03:00.880GMT",
"endTime" : "2015-05-06T13:03:00.890GMT",
"lastUpdated" : "",
@@ -34,6 +43,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1426633910242,
+ "endTimeEpoch" : 1426633945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-17T23:11:50.242GMT",
"endTime" : "2015-03-17T23:12:25.177GMT",
"lastUpdated" : "",
@@ -42,6 +54,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1426533910242,
+ "endTimeEpoch" : 1426533945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-16T19:25:10.242GMT",
"endTime" : "2015-03-16T19:25:45.177GMT",
"lastUpdated" : "",
@@ -54,6 +69,9 @@
"name": "Spark shell",
"attempts": [
{
+ "startTimeEpoch" : 1425081758277,
+ "endTimeEpoch" : 1425081766912,
+ "lastUpdatedEpoch" : 0,
"startTime": "2015-02-28T00:02:38.277GMT",
"endTime": "2015-02-28T00:02:46.912GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json
index cca32c7910..0217facad9 100644
--- a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json
@@ -2,6 +2,9 @@
"id" : "local-1422981780767",
"name" : "Spark shell",
"attempts" : [ {
+ "startTimeEpoch" : 1422981779720,
+ "endTimeEpoch" : 1422981788731,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-02-03T16:42:59.720GMT",
"endTime" : "2015-02-03T16:43:08.731GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json
index 1ea1779e83..b20a26648e 100644
--- a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json
@@ -3,6 +3,9 @@
"name" : "Spark shell",
"attempts" : [ {
"attemptId" : "2",
+ "startTimeEpoch" : 1426633910242,
+ "endTimeEpoch" : 1426633945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-17T23:11:50.242GMT",
"endTime" : "2015-03-17T23:12:25.177GMT",
"lastUpdated" : "",
@@ -11,6 +14,9 @@
"completed" : true
}, {
"attemptId" : "1",
+ "startTimeEpoch" : 1426533910242,
+ "endTimeEpoch" : 1426533945177,
+ "lastUpdatedEpoch" : 0,
"startTime" : "2015-03-16T19:25:10.242GMT",
"endTime" : "2015-03-16T19:25:45.177GMT",
"lastUpdated" : "",
diff --git a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json
index f79a31022d..8878e547a7 100644
--- a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json
@@ -1,9 +1 @@
-[ {
- "id" : 0,
- "name" : "0",
- "numPartitions" : 8,
- "numCachedPartitions" : 8,
- "storageLevel" : "Memory Deserialized 1x Replicated",
- "memoryUsed" : 28000128,
- "diskUsed" : 0
-} ] \ No newline at end of file
+[ ] \ No newline at end of file
diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
index a54d27de91..fb9d9851cb 100644
--- a/core/src/test/resources/log4j.properties
+++ b/core/src/test/resources/log4j.properties
@@ -33,5 +33,4 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%t: %m%n
# Ignore messages below warning level from Jetty, because it's a bit verbose
-log4j.logger.org.spark-project.jetty=WARN
-org.spark-project.jetty.LEVEL=WARN
+log4j.logger.org.spark_project.jetty=WARN
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index ec192a8543..37879d11ca 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark
+import java.util.Properties
import java.util.concurrent.Semaphore
import javax.annotation.concurrent.GuardedBy
@@ -292,7 +293,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex
dummyTask, mutable.HashMap(), mutable.HashMap(), serInstance)
// Now we're on the executors.
// Deserialize the task and assert that its accumulators are zero'ed out.
- val (_, _, taskBytes) = Task.deserializeWithDependencies(taskSer)
+ val (_, _, _, taskBytes) = Task.deserializeWithDependencies(taskSer)
val taskDeser = serInstance.deserialize[DummyTask](
taskBytes, Thread.currentThread.getContextClassLoader)
// Assert that executors see only zeros
@@ -403,6 +404,6 @@ private class SaveInfoListener extends SparkListener {
private[spark] class DummyTask(
val internalAccums: Seq[Accumulator[_]],
val externalAccums: Seq[Accumulator[_]])
- extends Task[Int](0, 0, 0, internalAccums) {
+ extends Task[Int](0, 0, 0, internalAccums, new Properties) {
override def runTask(c: TaskContext): Int = 1
}
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 3dded4d486..2110d3d770 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -198,8 +198,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
blockManager.master.getLocations(blockId).foreach { cmId =>
val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId,
blockId.toString)
- val deserialized = serializerManager.dataDeserialize[Int](blockId,
- new ChunkedByteBuffer(bytes.nioByteBuffer())).toList
+ val deserialized = serializerManager.dataDeserializeStream[Int](blockId,
+ new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream()).toList
assert(deserialized === (1 to 100).toList)
}
}
@@ -320,7 +320,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
Thread.sleep(200)
}
} catch {
- case _: Throwable => { Thread.sleep(10) }
+ case _: Throwable => Thread.sleep(10)
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 80a1de6065..ee6b991461 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -928,8 +928,8 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester {
numTasks: Int,
taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty
): StageInfo = {
- new StageInfo(
- stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", taskLocalityPreferences)
+ new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details",
+ Seq.empty, taskLocalityPreferences)
}
private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = {
diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
index 3777d77f8f..713d5e58b4 100644
--- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala
@@ -174,9 +174,9 @@ class HeartbeatReceiverSuite
val dummyExecutorEndpoint2 = new FakeExecutorEndpoint(rpcEnv)
val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1)
val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2)
- fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse](
+ fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean](
RegisterExecutor(executorId1, dummyExecutorEndpointRef1, 0, Map.empty))
- fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse](
+ fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean](
RegisterExecutor(executorId2, dummyExecutorEndpointRef2, 0, Map.empty))
heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet)
addExecutorAndVerify(executorId1)
@@ -255,7 +255,12 @@ class HeartbeatReceiverSuite
/**
* Dummy RPC endpoint to simulate executors.
*/
-private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint
+private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint {
+
+ override def receive: PartialFunction[Any, Unit] = {
+ case _ =>
+ }
+}
/**
* Dummy scheduler backend to simulate executor allocation requests to the cluster manager.
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index 6ffa1c8ac1..cd7d2e1570 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark
+import java.util.Properties
import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService}
import org.scalatest.Matchers
@@ -335,16 +336,16 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
// first attempt -- its successful
val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0,
- new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, metricsSystem,
- InternalAccumulator.create(sc)))
+ new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem,
+ InternalAccumulator.createAll(sc)))
val data1 = (1 to 10).map { x => x -> x}
// second attempt -- also successful. We'll write out different data,
// just to simulate the fact that the records may get written differently
// depending on what gets spilled, what gets combined, etc.
val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0,
- new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, metricsSystem,
- InternalAccumulator.create(sc)))
+ new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem,
+ InternalAccumulator.createAll(sc)))
val data2 = (11 to 20).map { x => x -> x}
// interleave writes of both attempts -- we want to test that both attempts can occur
@@ -372,8 +373,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
}
val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1,
- new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem,
- InternalAccumulator.create(sc)))
+ new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem,
+ InternalAccumulator.createAll(sc)))
val readData = reader.read().toIndexedSeq
assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq)
diff --git a/core/src/test/scala/org/apache/spark/Smuggle.scala b/core/src/test/scala/org/apache/spark/Smuggle.scala
index 9f0a1b4c25..9d9217ea1b 100644
--- a/core/src/test/scala/org/apache/spark/Smuggle.scala
+++ b/core/src/test/scala/org/apache/spark/Smuggle.scala
@@ -24,16 +24,16 @@ import scala.collection.mutable
import scala.language.implicitConversions
/**
- * Utility wrapper to "smuggle" objects into tasks while bypassing serialization.
- * This is intended for testing purposes, primarily to make locks, semaphores, and
- * other constructs that would not survive serialization available from within tasks.
- * A Smuggle reference is itself serializable, but after being serialized and
- * deserialized, it still refers to the same underlying "smuggled" object, as long
- * as it was deserialized within the same JVM. This can be useful for tests that
- * depend on the timing of task completion to be deterministic, since one can "smuggle"
- * a lock or semaphore into the task, and then the task can block until the test gives
- * the go-ahead to proceed via the lock.
- */
+ * Utility wrapper to "smuggle" objects into tasks while bypassing serialization.
+ * This is intended for testing purposes, primarily to make locks, semaphores, and
+ * other constructs that would not survive serialization available from within tasks.
+ * A Smuggle reference is itself serializable, but after being serialized and
+ * deserialized, it still refers to the same underlying "smuggled" object, as long
+ * as it was deserialized within the same JVM. This can be useful for tests that
+ * depend on the timing of task completion to be deterministic, since one can "smuggle"
+ * a lock or semaphore into the task, and then the task can block until the test gives
+ * the go-ahead to proceed via the lock.
+ */
class Smuggle[T] private(val key: Symbol) extends Serializable {
def smuggledObject: T = Smuggle.get(key)
}
@@ -41,13 +41,13 @@ class Smuggle[T] private(val key: Symbol) extends Serializable {
object Smuggle {
/**
- * Wraps the specified object to be smuggled into a serialized task without
- * being serialized itself.
- *
- * @param smuggledObject
- * @tparam T
- * @return Smuggle wrapper around smuggledObject.
- */
+ * Wraps the specified object to be smuggled into a serialized task without
+ * being serialized itself.
+ *
+ * @param smuggledObject
+ * @tparam T
+ * @return Smuggle wrapper around smuggledObject.
+ */
def apply[T](smuggledObject: T): Smuggle[T] = {
val key = Symbol(UUID.randomUUID().toString)
lock.writeLock().lock()
@@ -72,12 +72,12 @@ object Smuggle {
}
/**
- * Implicit conversion of a Smuggle wrapper to the object being smuggled.
- *
- * @param smuggle the wrapper to unpack.
- * @tparam T
- * @return the smuggled object represented by the wrapper.
- */
+ * Implicit conversion of a Smuggle wrapper to the object being smuggled.
+ *
+ * @param smuggle the wrapper to unpack.
+ * @tparam T
+ * @return the smuggled object represented by the wrapper.
+ */
implicit def unpackSmuggledObject[T](smuggle : Smuggle[T]): T = smuggle.smuggledObject
}
diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
index 3706455c3f..8feb3dee05 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
@@ -82,20 +82,18 @@ package object testPackage extends Assertions {
val curCallSite = sc.getCallSite().shortForm // note: 2 lines after definition of "rdd"
val rddCreationLine = rddCreationSite match {
- case CALL_SITE_REGEX(func, file, line) => {
+ case CALL_SITE_REGEX(func, file, line) =>
assert(func === "makeRDD")
assert(file === "SparkContextInfoSuite.scala")
line.toInt
- }
case _ => fail("Did not match expected call site format")
}
curCallSite match {
- case CALL_SITE_REGEX(func, file, line) => {
+ case CALL_SITE_REGEX(func, file, line) =>
assert(func === "getCallSite") // this is correct because we called it from outside of Spark
assert(file === "SparkContextInfoSuite.scala")
assert(line.toInt === rddCreationLine.toInt + 2)
- }
case _ => fail("Did not match expected call site format")
}
}
diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
index f7a13ab399..09e21646ee 100644
--- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
@@ -35,7 +35,7 @@ class UnpersistSuite extends SparkFunSuite with LocalSparkContext {
Thread.sleep(200)
}
} catch {
- case _: Throwable => { Thread.sleep(10) }
+ case _: Throwable => Thread.sleep(10)
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 96cb4fd0eb..2718976992 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -199,21 +199,21 @@ class SparkSubmitSuite
val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
val childArgsStr = childArgs.mkString(" ")
childArgsStr should include ("--class org.SomeClass")
- childArgsStr should include ("--executor-memory 5g")
- childArgsStr should include ("--driver-memory 4g")
- childArgsStr should include ("--executor-cores 5")
childArgsStr should include ("--arg arg1 --arg arg2")
- childArgsStr should include ("--queue thequeue")
childArgsStr should include regex ("--jar .*thejar.jar")
- childArgsStr should include regex ("--addJars .*one.jar,.*two.jar,.*three.jar")
- childArgsStr should include regex ("--files .*file1.txt,.*file2.txt")
- childArgsStr should include regex ("--archives .*archive1.txt,.*archive2.txt")
mainClass should be ("org.apache.spark.deploy.yarn.Client")
classpath should have length (0)
+
+ sysProps("spark.executor.memory") should be ("5g")
+ sysProps("spark.driver.memory") should be ("4g")
+ sysProps("spark.executor.cores") should be ("5")
+ sysProps("spark.yarn.queue") should be ("thequeue")
+ sysProps("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar")
+ sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
+ sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
sysProps("spark.app.name") should be ("beauty")
sysProps("spark.ui.enabled") should be ("false")
sysProps("SPARK_SUBMIT") should be ("true")
- sysProps.keys should not contain ("spark.jars")
}
test("handles YARN client mode") {
@@ -249,7 +249,8 @@ class SparkSubmitSuite
sysProps("spark.executor.instances") should be ("6")
sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
- sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar")
+ sysProps("spark.yarn.dist.jars") should include
+ regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar")
sysProps("SPARK_SUBMIT") should be ("true")
sysProps("spark.ui.enabled") should be ("false")
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
index d2e24912b5..3d39bd4a74 100644
--- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
@@ -561,7 +561,7 @@ class StandaloneDynamicAllocationSuite
when(endpointRef.address).thenReturn(mockAddress)
val message = RegisterExecutor(id, endpointRef, 10, Map.empty)
val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend]
- backend.driverEndpoint.askWithRetry[CoarseGrainedClusterMessage](message)
+ backend.driverEndpoint.askWithRetry[Boolean](message)
}
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 5822261d8d..2a013aca7b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -140,8 +140,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
"stage task list from multi-attempt app json(2)" ->
"applications/local-1426533911241/2/stages/0/0/taskList",
- "rdd list storage json" -> "applications/local-1422981780767/storage/rdd",
- "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0"
+ "rdd list storage json" -> "applications/local-1422981780767/storage/rdd"
+ // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845
+ // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0"
)
// run a bunch of characterization tests -- just verify the behavior is the same as what is saved
@@ -161,7 +162,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
val json = if (jsonOrg.indexOf("lastUpdated") >= 0) {
val subStrings = jsonOrg.split(",")
for (i <- subStrings.indices) {
- if (subStrings(i).indexOf("lastUpdated") >= 0) {
+ if (subStrings(i).indexOf("lastUpdatedEpoch") >= 0) {
+ subStrings(i) = subStrings(i).replaceAll("(\\d+)", "0")
+ } else if (subStrings(i).indexOf("lastUpdated") >= 0) {
subStrings(i) = "\"lastUpdated\":\"\""
}
}
diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala
index 088b05403c..d91f50f18f 100644
--- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala
@@ -285,8 +285,8 @@ class TaskMetricsSuite extends SparkFunSuite {
// set and increment values
in.setBytesRead(1L)
in.setBytesRead(2L)
- in.incRecordsReadInternal(1L)
- in.incRecordsReadInternal(2L)
+ in.incRecordsRead(1L)
+ in.incRecordsRead(2L)
in.setReadMethod(DataReadMethod.Disk)
// assert new values exist
assertValEquals(_.bytesRead, BYTES_READ, 2L)
diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
index 0644148eae..337fd7e85e 100644
--- a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
+++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala
@@ -26,7 +26,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: int") {
val conf = new SparkConf()
- val iConf = ConfigBuilder("spark.int").intConf.withDefault(1)
+ val iConf = ConfigBuilder("spark.int").intConf.createWithDefault(1)
assert(conf.get(iConf) === 1)
conf.set(iConf, 2)
assert(conf.get(iConf) === 2)
@@ -34,21 +34,21 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: long") {
val conf = new SparkConf()
- val lConf = ConfigBuilder("spark.long").longConf.withDefault(0L)
+ val lConf = ConfigBuilder("spark.long").longConf.createWithDefault(0L)
conf.set(lConf, 1234L)
assert(conf.get(lConf) === 1234L)
}
test("conf entry: double") {
val conf = new SparkConf()
- val dConf = ConfigBuilder("spark.double").doubleConf.withDefault(0.0)
+ val dConf = ConfigBuilder("spark.double").doubleConf.createWithDefault(0.0)
conf.set(dConf, 20.0)
assert(conf.get(dConf) === 20.0)
}
test("conf entry: boolean") {
val conf = new SparkConf()
- val bConf = ConfigBuilder("spark.boolean").booleanConf.withDefault(false)
+ val bConf = ConfigBuilder("spark.boolean").booleanConf.createWithDefault(false)
assert(!conf.get(bConf))
conf.set(bConf, true)
assert(conf.get(bConf))
@@ -56,7 +56,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: optional") {
val conf = new SparkConf()
- val optionalConf = ConfigBuilder("spark.optional").intConf.optional
+ val optionalConf = ConfigBuilder("spark.optional").intConf.createOptional
assert(conf.get(optionalConf) === None)
conf.set(optionalConf, 1)
assert(conf.get(optionalConf) === Some(1))
@@ -64,7 +64,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: fallback") {
val conf = new SparkConf()
- val parentConf = ConfigBuilder("spark.int").intConf.withDefault(1)
+ val parentConf = ConfigBuilder("spark.int").intConf.createWithDefault(1)
val confWithFallback = ConfigBuilder("spark.fallback").fallbackConf(parentConf)
assert(conf.get(confWithFallback) === 1)
conf.set(confWithFallback, 2)
@@ -74,7 +74,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: time") {
val conf = new SparkConf()
- val time = ConfigBuilder("spark.time").timeConf(TimeUnit.SECONDS).withDefaultString("1h")
+ val time = ConfigBuilder("spark.time").timeConf(TimeUnit.SECONDS).createWithDefaultString("1h")
assert(conf.get(time) === 3600L)
conf.set(time.key, "1m")
assert(conf.get(time) === 60L)
@@ -82,7 +82,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: bytes") {
val conf = new SparkConf()
- val bytes = ConfigBuilder("spark.bytes").bytesConf(ByteUnit.KiB).withDefaultString("1m")
+ val bytes = ConfigBuilder("spark.bytes").bytesConf(ByteUnit.KiB).createWithDefaultString("1m")
assert(conf.get(bytes) === 1024L)
conf.set(bytes.key, "1k")
assert(conf.get(bytes) === 1L)
@@ -90,7 +90,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: string seq") {
val conf = new SparkConf()
- val seq = ConfigBuilder("spark.seq").stringConf.toSequence.withDefault(Seq())
+ val seq = ConfigBuilder("spark.seq").stringConf.toSequence.createWithDefault(Seq())
conf.set(seq.key, "1,,2, 3 , , 4")
assert(conf.get(seq) === Seq("1", "2", "3", "4"))
conf.set(seq, Seq("1", "2"))
@@ -99,7 +99,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: int seq") {
val conf = new SparkConf()
- val seq = ConfigBuilder("spark.seq").intConf.toSequence.withDefault(Seq())
+ val seq = ConfigBuilder("spark.seq").intConf.toSequence.createWithDefault(Seq())
conf.set(seq.key, "1,,2, 3 , , 4")
assert(conf.get(seq) === Seq(1, 2, 3, 4))
conf.set(seq, Seq(1, 2))
@@ -111,7 +111,7 @@ class ConfigEntrySuite extends SparkFunSuite {
val transformationConf = ConfigBuilder("spark.transformation")
.stringConf
.transform(_.toLowerCase())
- .withDefault("FOO")
+ .createWithDefault("FOO")
assert(conf.get(transformationConf) === "foo")
conf.set(transformationConf, "BAR")
@@ -123,7 +123,7 @@ class ConfigEntrySuite extends SparkFunSuite {
val enum = ConfigBuilder("spark.enum")
.stringConf
.checkValues(Set("a", "b", "c"))
- .withDefault("a")
+ .createWithDefault("a")
assert(conf.get(enum) === "a")
conf.set(enum, "b")
@@ -138,7 +138,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("conf entry: conversion error") {
val conf = new SparkConf()
- val conversionTest = ConfigBuilder("spark.conversionTest").doubleConf.optional
+ val conversionTest = ConfigBuilder("spark.conversionTest").doubleConf.createOptional
conf.set(conversionTest.key, "abc")
val conversionError = intercept[IllegalArgumentException] {
conf.get(conversionTest)
@@ -148,7 +148,7 @@ class ConfigEntrySuite extends SparkFunSuite {
test("default value handling is null-safe") {
val conf = new SparkConf()
- val stringConf = ConfigBuilder("spark.string").stringConf.withDefault(null)
+ val stringConf = ConfigBuilder("spark.string").stringConf.createWithDefault(null)
assert(conf.get(stringConf) === null)
}
diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
index aab70e7431..f205d4f0d6 100644
--- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala
@@ -52,7 +52,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite {
test("copy() does not affect original buffer's position") {
val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8)))
- chunkedByteBuffer.copy()
+ chunkedByteBuffer.copy(ByteBuffer.allocate)
assert(chunkedByteBuffer.getChunks().head.position() === 0)
}
diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
index aaca653c58..99d5b496bc 100644
--- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
@@ -71,24 +71,25 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft
*/
protected def makeMemoryStore(mm: MemoryManager): MemoryStore = {
val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS)
- when(ms.evictBlocksToFreeSpace(any(), anyLong())).thenAnswer(evictBlocksToFreeSpaceAnswer(mm))
+ when(ms.evictBlocksToFreeSpace(any(), anyLong(), any()))
+ .thenAnswer(evictBlocksToFreeSpaceAnswer(mm))
mm.setMemoryStore(ms)
ms
}
/**
- * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory.
- *
- * This is a significant simplification of the real method, which actually drops existing
- * blocks based on the size of each block. Instead, here we simply release as many bytes
- * as needed to ensure the requested amount of free space. This allows us to set up the
- * test without relying on the [[org.apache.spark.storage.BlockManager]], which brings in
- * many other dependencies.
- *
- * Every call to this method will set a global variable, [[evictBlocksToFreeSpaceCalled]], that
- * records the number of bytes this is called with. This variable is expected to be cleared
- * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]].
- */
+ * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory.
+ *
+ * This is a significant simplification of the real method, which actually drops existing
+ * blocks based on the size of each block. Instead, here we simply release as many bytes
+ * as needed to ensure the requested amount of free space. This allows us to set up the
+ * test without relying on the [[org.apache.spark.storage.BlockManager]], which brings in
+ * many other dependencies.
+ *
+ * Every call to this method will set a global variable, [[evictBlocksToFreeSpaceCalled]], that
+ * records the number of bytes this is called with. This variable is expected to be cleared
+ * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]].
+ */
private def evictBlocksToFreeSpaceAnswer(mm: MemoryManager): Answer[Long] = {
new Answer[Long] {
override def answer(invocation: InvocationOnMock): Long = {
diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
index 2b5e4b80e9..362cd861cc 100644
--- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
+++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala
@@ -17,6 +17,8 @@
package org.apache.spark.memory
+import java.util.Properties
+
import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl}
/**
@@ -31,6 +33,7 @@ object MemoryTestingUtils {
taskAttemptId = 0,
attemptNumber = 0,
taskMemoryManager = taskMemoryManager,
+ localProperties = new Properties,
metricsSystem = env.metricsSystem)
}
}
diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala
index 6da18cfd49..ed15e77ff1 100644
--- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala
+++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala
@@ -108,11 +108,11 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi
when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer)
val securityManager0 = new SecurityManager(conf0)
- val exec0 = new NettyBlockTransferService(conf0, securityManager0, numCores = 1)
+ val exec0 = new NettyBlockTransferService(conf0, securityManager0, "localhost", numCores = 1)
exec0.init(blockManager)
val securityManager1 = new SecurityManager(conf1)
- val exec1 = new NettyBlockTransferService(conf1, securityManager1, numCores = 1)
+ val exec1 = new NettyBlockTransferService(conf1, securityManager1, "localhost", numCores = 1)
exec1.init(blockManager)
val result = fetchBlock(exec0, exec1, "1", blockId) match {
diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala
index cc1a9e0287..f3c156e4f7 100644
--- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala
+++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala
@@ -80,7 +80,7 @@ class NettyBlockTransferServiceSuite
.set("spark.blockManager.port", port.toString)
val securityManager = new SecurityManager(conf)
val blockDataManager = mock(classOf[BlockDataManager])
- val service = new NettyBlockTransferService(conf, securityManager, numCores = 1)
+ val service = new NettyBlockTransferService(conf, securityManager, "localhost", numCores = 1)
service.init(blockDataManager)
service
}
diff --git a/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala
new file mode 100644
index 0000000000..a79f5b4d74
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.partial
+
+import org.apache.spark._
+import org.apache.spark.util.StatCounter
+
+class SumEvaluatorSuite extends SparkFunSuite with SharedSparkContext {
+
+ test("correct handling of count 1") {
+
+ // setup
+ val counter = new StatCounter(List(2.0))
+ // count of 10 because it's larger than 1,
+ // and 0.95 because that's the default
+ val evaluator = new SumEvaluator(10, 0.95)
+ // arbitrarily assign id 1
+ evaluator.merge(1, counter)
+
+ // execute
+ val res = evaluator.currentResult()
+ // 38.0 - 7.1E-15 because that's how the maths shakes out
+ val targetMean = 38.0 - 7.1E-15
+
+ // Sanity check that equality works on BoundedDouble
+ assert(new BoundedDouble(2.0, 0.95, 1.1, 1.2) == new BoundedDouble(2.0, 0.95, 1.1, 1.2))
+
+ // actual test
+ assert(res ==
+ new BoundedDouble(targetMean, 0.950, Double.NegativeInfinity, Double.PositiveInfinity))
+ }
+
+ test("correct handling of count 0") {
+
+ // setup
+ val counter = new StatCounter(List())
+ // count of 10 because it's larger than 0,
+ // and 0.95 because that's the default
+ val evaluator = new SumEvaluator(10, 0.95)
+ // arbitrarily assign id 1
+ evaluator.merge(1, counter)
+
+ // execute
+ val res = evaluator.currentResult()
+ // assert
+ assert(res == new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity))
+ }
+
+ test("correct handling of NaN") {
+
+ // setup
+ val counter = new StatCounter(List(1, Double.NaN, 2))
+ // count of 10 because it's larger than 0,
+ // and 0.95 because that's the default
+ val evaluator = new SumEvaluator(10, 0.95)
+ // arbitrarily assign id 1
+ evaluator.merge(1, counter)
+
+ // execute
+ val res = evaluator.currentResult()
+ // assert - note semantics of == in face of NaN
+ assert(res.mean.isNaN)
+ assert(res.confidence == 0.95)
+ assert(res.low == Double.NegativeInfinity)
+ assert(res.high == Double.PositiveInfinity)
+ }
+
+ test("correct handling of > 1 values") {
+
+ // setup
+ val counter = new StatCounter(List(1, 3, 2))
+ // count of 10 because it's larger than 0,
+ // and 0.95 because that's the default
+ val evaluator = new SumEvaluator(10, 0.95)
+ // arbitrarily assign id 1
+ evaluator.merge(1, counter)
+
+ // execute
+ val res = evaluator.currentResult()
+
+ // These vals because that's how the maths shakes out
+ val targetMean = 78.0
+ val targetLow = -117.617 + 2.732357258139473E-5
+ val targetHigh = 273.617 - 2.7323572624027292E-5
+ val target = new BoundedDouble(targetMean, 0.95, targetLow, targetHigh)
+
+
+ // check that values are within expected tolerance of expectation
+ assert(res == target)
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
index 132a5fa9a8..cb0de1c6be 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala
@@ -29,6 +29,8 @@ class MockSampler extends RandomSampler[Long, Long] {
s = seed
}
+ override def sample(): Int = 1
+
override def sample(items: Iterator[Long]): Iterator[Long] = {
Iterator(s)
}
diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
index 43e61241b6..cebac2097f 100644
--- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
@@ -127,9 +127,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
override val rpcEnv = env
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case msg: String => {
+ case msg: String =>
context.reply(msg)
- }
}
})
val reply = rpcEndpointRef.askWithRetry[String]("hello")
@@ -141,9 +140,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
override val rpcEnv = env
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case msg: String => {
+ case msg: String =>
context.reply(msg)
- }
}
})
@@ -164,10 +162,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
override val rpcEnv = env
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case msg: String => {
+ case msg: String =>
Thread.sleep(100)
context.reply(msg)
- }
}
})
@@ -317,10 +314,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
override val rpcEnv = env
override def receive: PartialFunction[Any, Unit] = {
- case m => {
+ case m =>
self
callSelfSuccessfully = true
- }
}
})
@@ -682,9 +678,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
override val rpcEnv = localEnv
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
- case msg: String => {
+ case msg: String =>
context.reply(msg)
- }
}
})
val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication")
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 2293c11dad..fd96fb04f8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -1144,7 +1144,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou
// SPARK-9809 -- this stage is submitted without a task for each partition (because some of
// the shuffle map output is still available from stage 0); make sure we've still got internal
// accumulators setup
- assert(scheduler.stageIdToStage(2).internalAccumulators.nonEmpty)
+ assert(scheduler.stageIdToStage(2).latestInfo.internalAccumulators.nonEmpty)
completeShuffleMapStageSuccessfully(2, 0, 2)
completeNextResultStageWithSuccess(3, 1, idx => idx + 1234)
assert(results === Map(0 -> 1234, 1 -> 1235))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
index f7e16af9d3..e3e6df6831 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
@@ -17,12 +17,14 @@
package org.apache.spark.scheduler
+import java.util.Properties
+
import org.apache.spark.TaskContext
class FakeTask(
stageId: Int,
prefLocs: Seq[TaskLocation] = Nil)
- extends Task[Int](stageId, 0, 0, Seq.empty) {
+ extends Task[Int](stageId, 0, 0, Seq.empty, new Properties) {
override def runTask(context: TaskContext): Int = 0
override def preferredLocations: Seq[TaskLocation] = prefLocs
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala
index 1dca4bd89f..76a7087645 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala
@@ -18,6 +18,7 @@
package org.apache.spark.scheduler
import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
+import java.util.Properties
import org.apache.spark.TaskContext
@@ -25,7 +26,7 @@ import org.apache.spark.TaskContext
* A Task implementation that fails to serialize.
*/
private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int)
- extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) {
+ extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) {
override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte]
override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]()
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
index 9f41aca8a1..601f1c378c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala
@@ -38,7 +38,7 @@ class OutputCommitCoordinatorIntegrationSuite
super.beforeAll()
val conf = new SparkConf()
.set("master", "local[2,4]")
- .set("spark.speculation", "true")
+ .set("spark.hadoop.outputCommitCoordination.enabled", "true")
.set("spark.hadoop.mapred.output.committer.class",
classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName)
sc = new SparkContext("local[2, 4]", "test", conf)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
index c461da65bd..8e509de767 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -77,7 +77,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
val conf = new SparkConf()
.setMaster("local[4]")
.setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName)
- .set("spark.speculation", "true")
+ .set("spark.hadoop.outputCommitCoordination.enabled", "true")
sc = new SparkContext(conf) {
override private[spark] def createSparkEnv(
conf: SparkConf,
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 58d217ffef..b854d742b5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConverters._
import org.scalatest.Matchers
-import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite}
+import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.util.{ResetSystemProperties, RpcUtils}
@@ -377,13 +377,18 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
}
test("registering listeners via spark.extraListeners") {
+ val listeners = Seq(
+ classOf[ListenerThatAcceptsSparkConf],
+ classOf[FirehoseListenerThatAcceptsSparkConf],
+ classOf[BasicJobCounter])
val conf = new SparkConf().setMaster("local").setAppName("test")
- .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," +
- classOf[BasicJobCounter].getName)
+ .set("spark.extraListeners", listeners.map(_.getName).mkString(","))
sc = new SparkContext(conf)
sc.listenerBus.listeners.asScala.count(_.isInstanceOf[BasicJobCounter]) should be (1)
sc.listenerBus.listeners.asScala
.count(_.isInstanceOf[ListenerThatAcceptsSparkConf]) should be (1)
+ sc.listenerBus.listeners.asScala
+ .count(_.isInstanceOf[FirehoseListenerThatAcceptsSparkConf]) should be (1)
}
/**
@@ -476,3 +481,11 @@ private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListene
var count = 0
override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1
}
+
+private class FirehoseListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkFirehoseListener {
+ var count = 0
+ override def onEvent(event: SparkListenerEvent): Unit = event match {
+ case job: SparkListenerJobEnd => count += 1
+ case _ =>
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 5df541e5a5..5ca0c6419d 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -17,12 +17,14 @@
package org.apache.spark.scheduler
+import java.util.Properties
+
import org.mockito.Matchers.any
import org.mockito.Mockito._
import org.scalatest.BeforeAndAfter
import org.apache.spark._
-import org.apache.spark.executor.TaskMetricsSuite
+import org.apache.spark.executor.{Executor, TaskMetricsSuite}
import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.metrics.source.JvmSource
import org.apache.spark.network.util.JavaUtils
@@ -59,7 +61,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
val closureSerializer = SparkEnv.get.closureSerializer.newInstance()
val func = (c: TaskContext, i: Iterator[String]) => i.next()
val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func))))
- val task = new ResultTask[String, String](0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0)
+ val task = new ResultTask[String, String](
+ 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties)
intercept[RuntimeException] {
task.run(0, 0, null)
}
@@ -80,7 +83,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
val closureSerializer = SparkEnv.get.closureSerializer.newInstance()
val func = (c: TaskContext, i: Iterator[String]) => i.next()
val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func))))
- val task = new ResultTask[String, String](0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0)
+ val task = new ResultTask[String, String](
+ 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties)
intercept[RuntimeException] {
task.run(0, 0, null)
}
@@ -171,9 +175,10 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
val initialAccums = InternalAccumulator.createAll()
// Create a dummy task. We won't end up running this; we just want to collect
// accumulator updates from it.
- val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]]) {
+ val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]], new Properties) {
context = new TaskContextImpl(0, 0, 0L, 0,
new TaskMemoryManager(SparkEnv.get.memoryManager, 0L),
+ new Properties,
SparkEnv.get.metricsSystem,
initialAccums)
context.taskMetrics.registerAccumulator(acc1)
@@ -190,6 +195,17 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
TaskMetricsSuite.assertUpdatesEquals(accumUpdates3, accumUpdates4)
}
+ test("localProperties are propagated to executors correctly") {
+ sc = new SparkContext("local", "test")
+ sc.setLocalProperty("testPropKey", "testPropValue")
+ val res = sc.parallelize(Array(1), 1).map(i => i).map(i => {
+ val inTask = TaskContext.get().getLocalProperty("testPropKey")
+ val inDeser = Executor.taskDeserializationProps.get().getProperty("testPropKey")
+ s"$inTask,$inDeser"
+ }).collect()
+ assert(res === Array("testPropValue,testPropValue"))
+ }
+
}
private object TaskContextSuite {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 167d3fd2e4..ade8e84d84 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.scheduler
-import java.util.Random
+import java.util.{Properties, Random}
import scala.collection.Map
import scala.collection.mutable
@@ -138,7 +138,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex
/**
* A Task implementation that results in a large serialized task.
*/
-class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) {
+class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) {
val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024)
val random = new Random(0)
random.nextBytes(randomBuffer)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
index dbef6868f2..a32423dc4f 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala
@@ -136,4 +136,40 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi
capture.capture()
)
}
+
+ test("escapes commandline args for the shell") {
+ val conf = new SparkConf()
+ conf.setMaster("mesos://localhost:5050")
+ conf.setAppName("spark mesos")
+ val scheduler = new MesosClusterScheduler(
+ new BlackHoleMesosClusterPersistenceEngineFactory, conf) {
+ override def start(): Unit = { ready = true }
+ }
+ val escape = scheduler.shellEscape _
+ def wrapped(str: String): String = "\"" + str + "\""
+
+ // Wrapped in quotes
+ assert(escape("'should be left untouched'") === "'should be left untouched'")
+ assert(escape("\"should be left untouched\"") === "\"should be left untouched\"")
+
+ // Harmless
+ assert(escape("") === "")
+ assert(escape("harmless") === "harmless")
+ assert(escape("har-m.l3ss") === "har-m.l3ss")
+
+ // Special Chars escape
+ assert(escape("should escape this \" quote") === wrapped("should escape this \\\" quote"))
+ assert(escape("shouldescape\"quote") === wrapped("shouldescape\\\"quote"))
+ assert(escape("should escape this $ dollar") === wrapped("should escape this \\$ dollar"))
+ assert(escape("should escape this ` backtick") === wrapped("should escape this \\` backtick"))
+ assert(escape("""should escape this \ backslash""")
+ === wrapped("""should escape this \\ backslash"""))
+ assert(escape("""\"?""") === wrapped("""\\\"?"""))
+
+
+ // Special Chars no escape only wrap
+ List(" ", "'", "<", ">", "&", "|", "?", "*", ";", "!", "#", "(", ")").foreach(char => {
+ assert(escape(s"onlywrap${char}this") === wrapped(s"onlywrap${char}this"))
+ })
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
index 85437b2f80..ceb3a52983 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
@@ -59,10 +59,10 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS
test("parse a non-empty constraint string correctly") {
val expectedMap = Map(
- "tachyon" -> Set("true"),
+ "os" -> Set("centos7"),
"zone" -> Set("us-east-1a", "us-east-1b")
)
- utils.parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") should be (expectedMap)
+ utils.parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") should be (expectedMap)
}
test("parse an empty constraint string correctly") {
@@ -71,35 +71,35 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS
test("throw an exception when the input is malformed") {
an[IllegalArgumentException] should be thrownBy
- utils.parseConstraintString("tachyon;zone:us-east")
+ utils.parseConstraintString("os;zone:us-east")
}
test("empty values for attributes' constraints matches all values") {
- val constraintsStr = "tachyon:"
+ val constraintsStr = "os:"
val parsedConstraints = utils.parseConstraintString(constraintsStr)
- parsedConstraints shouldBe Map("tachyon" -> Set())
+ parsedConstraints shouldBe Map("os" -> Set())
val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build()
- val noTachyonOffer = Map("zone" -> zoneSet)
- val tachyonTrueOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build())
- val tachyonFalseOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("false").build())
+ val noOsOffer = Map("zone" -> zoneSet)
+ val centosOffer = Map("os" -> Value.Text.newBuilder().setValue("centos").build())
+ val ubuntuOffer = Map("os" -> Value.Text.newBuilder().setValue("ubuntu").build())
- utils.matchesAttributeRequirements(parsedConstraints, noTachyonOffer) shouldBe false
- utils.matchesAttributeRequirements(parsedConstraints, tachyonTrueOffer) shouldBe true
- utils.matchesAttributeRequirements(parsedConstraints, tachyonFalseOffer) shouldBe true
+ utils.matchesAttributeRequirements(parsedConstraints, noOsOffer) shouldBe false
+ utils.matchesAttributeRequirements(parsedConstraints, centosOffer) shouldBe true
+ utils.matchesAttributeRequirements(parsedConstraints, ubuntuOffer) shouldBe true
}
test("subset match is performed for set attributes") {
val supersetConstraint = Map(
- "tachyon" -> Value.Text.newBuilder().setValue("true").build(),
+ "os" -> Value.Text.newBuilder().setValue("ubuntu").build(),
"zone" -> Value.Set.newBuilder()
.addItem("us-east-1a")
.addItem("us-east-1b")
.addItem("us-east-1c")
.build())
- val zoneConstraintStr = "tachyon:;zone:us-east-1a,us-east-1c"
+ val zoneConstraintStr = "os:;zone:us-east-1a,us-east-1c"
val parsedConstraints = utils.parseConstraintString(zoneConstraintStr)
utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true
@@ -131,10 +131,10 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS
}
test("equality match is performed for text attributes") {
- val offerAttribs = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build())
+ val offerAttribs = Map("os" -> Value.Text.newBuilder().setValue("centos7").build())
- val trueConstraint = utils.parseConstraintString("tachyon:true")
- val falseConstraint = utils.parseConstraintString("tachyon:false")
+ val trueConstraint = utils.parseConstraintString("os:centos7")
+ val falseConstraint = utils.parseConstraintString("os:ubuntu")
utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true
utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
index 7ee76aa4c6..9d1bd7ec89 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.storage
+import java.util.Properties
+
import scala.concurrent.{Await, ExecutionContext, Future}
import scala.language.implicitConversions
import scala.reflect.ClassTag
@@ -58,7 +60,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach {
private def withTaskId[T](taskAttemptId: Long)(block: => T): T = {
try {
- TaskContext.setTaskContext(new TaskContextImpl(0, 0, taskAttemptId, 0, null, null))
+ TaskContext.setTaskContext(
+ new TaskContextImpl(0, 0, taskAttemptId, 0, null, new Properties, null))
block
} finally {
TaskContext.unset()
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
index 98e8450fa1..d26df7e760 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
@@ -27,7 +27,7 @@ import org.scalatest.{BeforeAndAfter, Matchers}
import org.scalatest.concurrent.Eventually._
import org.apache.spark._
-import org.apache.spark.memory.StaticMemoryManager
+import org.apache.spark.memory.UnifiedMemoryManager
import org.apache.spark.network.BlockTransferService
import org.apache.spark.network.netty.NettyBlockTransferService
import org.apache.spark.rpc.RpcEnv
@@ -60,8 +60,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo
private def makeBlockManager(
maxMem: Long,
name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = {
- val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1)
- val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1)
+ conf.set("spark.testing.memory", maxMem.toString)
+ conf.set("spark.memory.offHeap.size", maxMem.toString)
+ val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1)
+ val memManager = UnifiedMemoryManager(conf, numCores = 1)
val serializerManager = new SerializerManager(serializer, conf)
val store = new BlockManager(name, rpcEnv, master, serializerManager, conf,
memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0)
@@ -76,6 +78,9 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo
conf.set("spark.authenticate", "false")
conf.set("spark.driver.port", rpcEnv.address.port.toString)
+ conf.set("spark.testing", "true")
+ conf.set("spark.memory.fraction", "1")
+ conf.set("spark.memory.storageFraction", "1")
conf.set("spark.storage.unrollFraction", "0.4")
conf.set("spark.storage.unrollMemoryThreshold", "512")
@@ -172,6 +177,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo
testReplication(5, storageLevels)
}
+ test("block replication - off-heap") {
+ testReplication(2, Seq(OFF_HEAP, StorageLevel(true, true, true, false, 2)))
+ }
+
test("block replication - 2x replication without peers") {
intercept[org.scalatest.exceptions.TestFailedException] {
testReplication(1,
@@ -262,7 +271,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo
val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work
when(failableTransfer.hostName).thenReturn("some-hostname")
when(failableTransfer.port).thenReturn(1000)
- val memManager = new StaticMemoryManager(conf, Long.MaxValue, 10000, numCores = 1)
+ conf.set("spark.testing.memory", "10000")
+ val memManager = UnifiedMemoryManager(conf, numCores = 1)
val serializerManager = new SerializerManager(serializer, conf)
val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf,
memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0)
@@ -392,10 +402,14 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo
// If the block is supposed to be in memory, then drop the copy of the block in
// this store test whether master is updated with zero memory usage this store
if (storageLevel.useMemory) {
+ val sl = if (storageLevel.useOffHeap) {
+ StorageLevel(false, true, true, false, 1)
+ } else {
+ MEMORY_ONLY_SER
+ }
// Force the block to be dropped by adding a number of dummy blocks
(1 to 10).foreach {
- i =>
- testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER)
+ i => testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), sl)
}
(1 to 10).foreach {
i => testStore.removeBlock(s"dummy-block-$i")
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 6fc32cb30a..a1c2933584 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -34,7 +34,7 @@ import org.scalatest.concurrent.Timeouts._
import org.apache.spark._
import org.apache.spark.executor.DataReadMethod
-import org.apache.spark.memory.{MemoryMode, StaticMemoryManager}
+import org.apache.spark.memory.UnifiedMemoryManager
import org.apache.spark.network.{BlockDataManager, BlockTransferService}
import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
import org.apache.spark.network.netty.NettyBlockTransferService
@@ -74,10 +74,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
name: String = SparkContext.DRIVER_IDENTIFIER,
master: BlockManagerMaster = this.master,
transferService: Option[BlockTransferService] = Option.empty): BlockManager = {
+ conf.set("spark.testing.memory", maxMem.toString)
+ conf.set("spark.memory.offHeap.size", maxMem.toString)
val serializer = new KryoSerializer(conf)
val transfer = transferService
- .getOrElse(new NettyBlockTransferService(conf, securityMgr, numCores = 1))
- val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1)
+ .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1))
+ val memManager = UnifiedMemoryManager(conf, numCores = 1)
val serializerManager = new SerializerManager(serializer, conf)
val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, conf,
memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0)
@@ -92,6 +94,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
System.setProperty("os.arch", "amd64")
conf = new SparkConf(false)
.set("spark.app.id", "test")
+ .set("spark.testing", "true")
+ .set("spark.memory.fraction", "1")
+ .set("spark.memory.storageFraction", "1")
.set("spark.kryoserializer.buffer", "1m")
.set("spark.test.useCompressedOops", "true")
.set("spark.storage.unrollFraction", "0.4")
@@ -485,7 +490,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
val blockManager = makeBlockManager(128, "exec", bmMaster)
val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations)
val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0))
- assert(locations.map(_.host) === Seq(localHost, localHost, otherHost))
+ assert(locations.map(_.host).toSet === Set(localHost, localHost, otherHost))
}
test("SPARK-9591: getRemoteBytes from another location when Exception throw") {
@@ -510,6 +515,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
}
+ test("SPARK-14252: getOrElseUpdate should still read from remote storage") {
+ store = makeBlockManager(8000, "executor1")
+ store2 = makeBlockManager(8000, "executor2")
+ val list1 = List(new Array[Byte](4000))
+ store2.putIterator(
+ "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true)
+ assert(store.getOrElseUpdate(
+ "list1",
+ StorageLevel.MEMORY_ONLY,
+ ClassTag.Any,
+ () => throw new AssertionError("attempted to compute locally")).isLeft)
+ }
+
test("in-memory LRU storage") {
testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY)
}
@@ -518,6 +536,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY_SER)
}
+ test("in-memory LRU storage with off-heap") {
+ testInMemoryLRUStorage(StorageLevel(
+ useDisk = false,
+ useMemory = true,
+ useOffHeap = true,
+ deserialized = false, replication = 1))
+ }
+
private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = {
store = makeBlockManager(12000)
val a1 = new Array[Byte](4000)
@@ -608,6 +634,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, getAsBytes = true)
}
+ test("disk and off-heap memory storage") {
+ testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = false)
+ }
+
+ test("disk and off-heap memory storage with getLocalBytes") {
+ testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = true)
+ }
+
def testDiskAndMemoryStorage(
storageLevel: StorageLevel,
getAsBytes: Boolean): Unit = {
@@ -817,12 +851,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("block store put failure") {
// Use Java serializer so we can create an unserializable error.
- val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1)
- val memoryManager = new StaticMemoryManager(
- conf,
- maxOnHeapExecutionMemory = Long.MaxValue,
- maxOnHeapStorageMemory = 1200,
- numCores = 1)
+ conf.set("spark.testing.memory", "1200")
+ val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1)
+ val memoryManager = UnifiedMemoryManager(conf, numCores = 1)
val serializerManager = new SerializerManager(new JavaSerializer(conf), conf)
store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master,
serializerManager, conf, memoryManager, mapOutputTracker,
@@ -928,6 +959,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
assert(!store.diskStore.contains("list3"), "list3 was in disk store")
assert(!store.diskStore.contains("list4"), "list4 was in disk store")
assert(!store.diskStore.contains("list5"), "list5 was in disk store")
+
+ // remove block - list2 should be removed from disk
+ val updatedBlocks6 = getUpdatedBlocks {
+ store.removeBlock(
+ "list2", tellMaster = true)
+ }
+ assert(updatedBlocks6.size === 1)
+ assert(updatedBlocks6.head._1 === TestBlockId("list2"))
+ assert(updatedBlocks6.head._2.storageLevel == StorageLevel.NONE)
+ assert(!store.diskStore.contains("list2"), "list2 was in disk store")
}
test("query block statuses") {
diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
index 43e832dc02..145d432afe 100644
--- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala
@@ -27,7 +27,7 @@ import scala.reflect.ClassTag
import org.scalatest._
import org.apache.spark._
-import org.apache.spark.memory.StaticMemoryManager
+import org.apache.spark.memory.{MemoryMode, StaticMemoryManager}
import org.apache.spark.serializer.{KryoSerializer, SerializerManager}
import org.apache.spark.storage.memory.{BlockEvictionHandler, MemoryStore, PartiallySerializedBlock, PartiallyUnrolledIterator}
import org.apache.spark.util._
@@ -86,7 +86,7 @@ class MemoryStoreSuite
assert(memoryStore.currentUnrollMemoryForThisTask === 0)
def reserveUnrollMemoryForThisTask(memory: Long): Boolean = {
- memoryStore.reserveUnrollMemoryForThisTask(TestBlockId(""), memory)
+ memoryStore.reserveUnrollMemoryForThisTask(TestBlockId(""), memory, MemoryMode.ON_HEAP)
}
// Reserve
@@ -99,9 +99,9 @@ class MemoryStoreSuite
assert(!reserveUnrollMemoryForThisTask(1000000))
assert(memoryStore.currentUnrollMemoryForThisTask === 800) // not granted
// Release
- memoryStore.releaseUnrollMemoryForThisTask(100)
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100)
assert(memoryStore.currentUnrollMemoryForThisTask === 700)
- memoryStore.releaseUnrollMemoryForThisTask(100)
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100)
assert(memoryStore.currentUnrollMemoryForThisTask === 600)
// Reserve again
assert(reserveUnrollMemoryForThisTask(4400))
@@ -109,9 +109,9 @@ class MemoryStoreSuite
assert(!reserveUnrollMemoryForThisTask(20000))
assert(memoryStore.currentUnrollMemoryForThisTask === 5000) // not granted
// Release again
- memoryStore.releaseUnrollMemoryForThisTask(1000)
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 1000)
assert(memoryStore.currentUnrollMemoryForThisTask === 4000)
- memoryStore.releaseUnrollMemoryForThisTask() // release all
+ memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) // release all
assert(memoryStore.currentUnrollMemoryForThisTask === 0)
}
@@ -254,7 +254,7 @@ class MemoryStoreSuite
assert(blockInfoManager.lockNewBlockForWriting(
blockId,
new BlockInfo(StorageLevel.MEMORY_ONLY_SER, classTag, tellMaster = false)))
- val res = memoryStore.putIteratorAsBytes(blockId, iter, classTag)
+ val res = memoryStore.putIteratorAsBytes(blockId, iter, classTag, MemoryMode.ON_HEAP)
blockInfoManager.unlock(blockId)
res
}
@@ -312,7 +312,7 @@ class MemoryStoreSuite
assert(blockInfoManager.lockNewBlockForWriting(
"b1",
new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false)))
- val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any)
+ val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP)
blockInfoManager.unlock("b1")
assert(res.isLeft)
assert(memoryStore.currentUnrollMemoryForThisTask > 0)
@@ -333,7 +333,7 @@ class MemoryStoreSuite
assert(blockInfoManager.lockNewBlockForWriting(
"b1",
new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false)))
- val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any)
+ val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP)
blockInfoManager.unlock("b1")
assert(res.isLeft)
assert(memoryStore.currentUnrollMemoryForThisTask > 0)
@@ -395,7 +395,7 @@ class MemoryStoreSuite
val blockId = BlockId("rdd_3_10")
blockInfoManager.lockNewBlockForWriting(
blockId, new BlockInfo(StorageLevel.MEMORY_ONLY, ClassTag.Any, tellMaster = false))
- memoryStore.putBytes(blockId, 13000, () => {
+ memoryStore.putBytes(blockId, 13000, MemoryMode.ON_HEAP, () => {
fail("A big ByteBuffer that cannot be put into MemoryStore should not be created")
})
}
@@ -404,7 +404,7 @@ class MemoryStoreSuite
val (memoryStore, _) = makeMemoryStore(12000)
val blockId = BlockId("rdd_3_10")
var bytes: ChunkedByteBuffer = null
- memoryStore.putBytes(blockId, 10000, () => {
+ memoryStore.putBytes(blockId, 10000, MemoryMode.ON_HEAP, () => {
bytes = new ChunkedByteBuffer(ByteBuffer.allocate(10000))
bytes
})
diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala
index 14daa003bc..9835f11a2f 100644
--- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala
@@ -82,48 +82,51 @@ class StorageStatusListenerSuite extends SparkFunSuite {
assert(listener.executorIdToStorageStatus("fat").numBlocks === 0)
}
- test("task end with updated blocks") {
+ test("updated blocks") {
val listener = new StorageStatusListener(conf)
listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L))
listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L))
- val taskMetrics1 = new TaskMetrics
- val taskMetrics2 = new TaskMetrics
- val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L))
- val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L))
- val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L))
- taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2))
- taskMetrics2.setUpdatedBlockStatuses(Seq(block3))
-
- // Task end with new blocks
+
+ val blockUpdateInfos1 = Seq(
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L),
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L)
+ )
+ val blockUpdateInfos2 =
+ Seq(BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L))
+
+ // Add some new blocks
assert(listener.executorIdToStorageStatus("big").numBlocks === 0)
assert(listener.executorIdToStorageStatus("fat").numBlocks === 0)
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1))
+ postUpdateBlock(listener, blockUpdateInfos1)
assert(listener.executorIdToStorageStatus("big").numBlocks === 2)
assert(listener.executorIdToStorageStatus("fat").numBlocks === 0)
assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1)))
assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2)))
assert(listener.executorIdToStorageStatus("fat").numBlocks === 0)
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2))
+ postUpdateBlock(listener, blockUpdateInfos2)
assert(listener.executorIdToStorageStatus("big").numBlocks === 2)
assert(listener.executorIdToStorageStatus("fat").numBlocks === 1)
assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1)))
assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2)))
assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0)))
- // Task end with dropped blocks
- val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L))
- val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L))
- val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L))
- taskMetrics1.setUpdatedBlockStatuses(Seq(droppedBlock1, droppedBlock3))
- taskMetrics2.setUpdatedBlockStatuses(Seq(droppedBlock2, droppedBlock3))
+ // Dropped the blocks
+ val droppedBlockInfo1 = Seq(
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.NONE, 0L, 0L),
+ BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L)
+ )
+ val droppedBlockInfo2 = Seq(
+ BlockUpdatedInfo(bm2, RDDBlockId(1, 2), StorageLevel.NONE, 0L, 0L),
+ BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L)
+ )
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1))
+ postUpdateBlock(listener, droppedBlockInfo1)
assert(listener.executorIdToStorageStatus("big").numBlocks === 1)
assert(listener.executorIdToStorageStatus("fat").numBlocks === 1)
assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1)))
assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2)))
assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0)))
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2))
+ postUpdateBlock(listener, droppedBlockInfo2)
assert(listener.executorIdToStorageStatus("big").numBlocks === 1)
assert(listener.executorIdToStorageStatus("fat").numBlocks === 0)
assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1)))
@@ -134,15 +137,14 @@ class StorageStatusListenerSuite extends SparkFunSuite {
test("unpersist RDD") {
val listener = new StorageStatusListener(conf)
listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L))
- val taskMetrics1 = new TaskMetrics
- val taskMetrics2 = new TaskMetrics
- val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L))
- val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L))
- val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L))
- taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2))
- taskMetrics2.setUpdatedBlockStatuses(Seq(block3))
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1))
- listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2))
+ val blockUpdateInfos1 = Seq(
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L),
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L)
+ )
+ val blockUpdateInfos2 =
+ Seq(BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L))
+ postUpdateBlock(listener, blockUpdateInfos1)
+ postUpdateBlock(listener, blockUpdateInfos2)
assert(listener.executorIdToStorageStatus("big").numBlocks === 3)
// Unpersist RDD
@@ -155,4 +157,11 @@ class StorageStatusListenerSuite extends SparkFunSuite {
listener.onUnpersistRDD(SparkListenerUnpersistRDD(1))
assert(listener.executorIdToStorageStatus("big").numBlocks === 0)
}
+
+ private def postUpdateBlock(
+ listener: StorageStatusListener, updateBlockInfos: Seq[BlockUpdatedInfo]): Unit = {
+ updateBlockInfos.foreach { updateBlockInfo =>
+ listener.onBlockUpdated(SparkListenerBlockUpdated(updateBlockInfo))
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala
index 9876bded33..7d4c0863bc 100644
--- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala
@@ -322,11 +322,11 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with
assert(stage1Data.inputBytes == 207)
assert(stage0Data.outputBytes == 116)
assert(stage1Data.outputBytes == 208)
- assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get
+ assert(stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.get
.totalBlocksFetched == 2)
- assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get
+ assert(stage0Data.taskData.get(1235L).get.metrics.get.shuffleReadMetrics.get
.totalBlocksFetched == 102)
- assert(stage1Data.taskData.get(1236L).get.taskMetrics.get.shuffleReadMetrics.get
+ assert(stage1Data.taskData.get(1236L).get.metrics.get.shuffleReadMetrics.get
.totalBlocksFetched == 202)
// task that was included in a heartbeat
@@ -355,9 +355,9 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with
assert(stage1Data.inputBytes == 614)
assert(stage0Data.outputBytes == 416)
assert(stage1Data.outputBytes == 616)
- assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get
+ assert(stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.get
.totalBlocksFetched == 302)
- assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get
+ assert(stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.get
.totalBlocksFetched == 402)
}
}
diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala
index 6b7c538ac8..7d77deeb60 100644
--- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala
@@ -106,7 +106,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter {
assert(storageListener.rddInfoList.size === 0)
}
- test("task end") {
+ test("block update") {
val myRddInfo0 = rddInfo0
val myRddInfo1 = rddInfo1
val myRddInfo2 = rddInfo2
@@ -120,19 +120,13 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter {
assert(!storageListener._rddInfoMap(1).isCached)
assert(!storageListener._rddInfoMap(2).isCached)
- // Task end with no updated blocks. This should not change anything.
- bus.postToAll(SparkListenerTaskEnd(0, 0, "obliteration", Success, taskInfo, new TaskMetrics))
- assert(storageListener._rddInfoMap.size === 3)
- assert(storageListener.rddInfoList.size === 0)
-
- // Task end with a few new persisted blocks, some from the same RDD
- val metrics1 = new TaskMetrics
- metrics1.setUpdatedBlockStatuses(Seq(
- (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L)),
- (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L)),
- (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L))
- ))
- bus.postToAll(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo, metrics1))
+ // Some blocks updated
+ val blockUpdateInfos = Seq(
+ BlockUpdatedInfo(bm1, RDDBlockId(0, 100), memAndDisk, 400L, 0L),
+ BlockUpdatedInfo(bm1, RDDBlockId(0, 101), memAndDisk, 0L, 400L),
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 20), memAndDisk, 0L, 240L)
+ )
+ postUpdateBlocks(bus, blockUpdateInfos)
assert(storageListener._rddInfoMap(0).memSize === 400L)
assert(storageListener._rddInfoMap(0).diskSize === 400L)
assert(storageListener._rddInfoMap(0).numCachedPartitions === 2)
@@ -144,15 +138,14 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter {
assert(!storageListener._rddInfoMap(2).isCached)
assert(storageListener._rddInfoMap(2).numCachedPartitions === 0)
- // Task end with a few dropped blocks
- val metrics2 = new TaskMetrics
- metrics2.setUpdatedBlockStatuses(Seq(
- (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L)),
- (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L)),
- (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L)), // doesn't actually exist
- (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L)) // doesn't actually exist
- ))
- bus.postToAll(SparkListenerTaskEnd(2, 0, "obliteration", Success, taskInfo, metrics2))
+ // Drop some blocks
+ val blockUpdateInfos2 = Seq(
+ BlockUpdatedInfo(bm1, RDDBlockId(0, 100), none, 0L, 0L),
+ BlockUpdatedInfo(bm1, RDDBlockId(1, 20), none, 0L, 0L),
+ BlockUpdatedInfo(bm1, RDDBlockId(2, 40), none, 0L, 0L), // doesn't actually exist
+ BlockUpdatedInfo(bm1, RDDBlockId(4, 80), none, 0L, 0L) // doesn't actually exist
+ )
+ postUpdateBlocks(bus, blockUpdateInfos2)
assert(storageListener._rddInfoMap(0).memSize === 0L)
assert(storageListener._rddInfoMap(0).diskSize === 400L)
assert(storageListener._rddInfoMap(0).numCachedPartitions === 1)
@@ -169,24 +162,27 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter {
val rddInfo1 = new RDDInfo(1, "rdd1", 1, memOnly, Seq(4))
val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo0), Seq.empty, "details")
val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfo1), Seq.empty, "details")
- val taskMetrics0 = new TaskMetrics
- val taskMetrics1 = new TaskMetrics
- val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L))
- val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L))
- taskMetrics0.setUpdatedBlockStatuses(Seq(block0))
- taskMetrics1.setUpdatedBlockStatuses(Seq(block1))
+ val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L))
+ val blockUpdateInfos2 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(1, 1), memOnly, 200L, 0L))
bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L))
bus.postToAll(SparkListenerStageSubmitted(stageInfo0))
assert(storageListener.rddInfoList.size === 0)
- bus.postToAll(SparkListenerTaskEnd(0, 0, "big", Success, taskInfo, taskMetrics0))
+ postUpdateBlocks(bus, blockUpdateInfos1)
assert(storageListener.rddInfoList.size === 1)
bus.postToAll(SparkListenerStageSubmitted(stageInfo1))
assert(storageListener.rddInfoList.size === 1)
bus.postToAll(SparkListenerStageCompleted(stageInfo0))
assert(storageListener.rddInfoList.size === 1)
- bus.postToAll(SparkListenerTaskEnd(1, 0, "small", Success, taskInfo1, taskMetrics1))
+ postUpdateBlocks(bus, blockUpdateInfos2)
assert(storageListener.rddInfoList.size === 2)
bus.postToAll(SparkListenerStageCompleted(stageInfo1))
assert(storageListener.rddInfoList.size === 2)
}
+
+ private def postUpdateBlocks(
+ bus: SparkListenerBus, blockUpdateInfos: Seq[BlockUpdatedInfo]): Unit = {
+ blockUpdateInfos.foreach { blockUpdateInfo =>
+ bus.postToAll(SparkListenerBlockUpdated(blockUpdateInfo))
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala b/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala
new file mode 100644
index 0000000000..4a80e3f1f4
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.util
+
+import org.apache.spark.SparkFunSuite
+
+class CausedBySuite extends SparkFunSuite {
+
+ test("For an error without a cause, should return the error") {
+ val error = new Exception
+
+ val causedBy = error match {
+ case CausedBy(e) => e
+ }
+
+ assert(causedBy === error)
+ }
+
+ test("For an error with a cause, should return the cause of the error") {
+ val cause = new Exception
+ val error = new Exception(cause)
+
+ val causedBy = error match {
+ case CausedBy(e) => e
+ }
+
+ assert(causedBy === cause)
+ }
+
+ test("For an error with a cause that itself has a cause, return the root cause") {
+ val causeOfCause = new Exception
+ val cause = new Exception(causeOfCause)
+ val error = new Exception(cause)
+
+ val causedBy = error match {
+ case CausedBy(e) => e
+ }
+
+ assert(causedBy === causeOfCause)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
index 280e496498..4fa9f9a8f5 100644
--- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
@@ -201,24 +201,29 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {
// Make sure only logging errors
val logger = Logger.getRootLogger
+ val oldLogLevel = logger.getLevel
logger.setLevel(Level.ERROR)
- logger.addAppender(mockAppender)
+ try {
+ logger.addAppender(mockAppender)
- val testOutputStream = new PipedOutputStream()
- val testInputStream = new PipedInputStream(testOutputStream)
+ val testOutputStream = new PipedOutputStream()
+ val testInputStream = new PipedInputStream(testOutputStream)
- // Close the stream before appender tries to read will cause an IOException
- testInputStream.close()
- testOutputStream.close()
- val appender = FileAppender(testInputStream, testFile, new SparkConf)
+ // Close the stream before appender tries to read will cause an IOException
+ testInputStream.close()
+ testOutputStream.close()
+ val appender = FileAppender(testInputStream, testFile, new SparkConf)
- appender.awaitTermination()
+ appender.awaitTermination()
- // If InputStream was closed without first stopping the appender, an exception will be logged
- verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture)
- val loggingEvent = loggingEventCaptor.getValue
- assert(loggingEvent.getThrowableInformation !== null)
- assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
+ // If InputStream was closed without first stopping the appender, an exception will be logged
+ verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture)
+ val loggingEvent = loggingEventCaptor.getValue
+ assert(loggingEvent.getThrowableInformation !== null)
+ assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
+ } finally {
+ logger.setLevel(oldLogLevel)
+ }
}
test("file appender async close stream gracefully") {
@@ -228,30 +233,35 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {
// Make sure only logging errors
val logger = Logger.getRootLogger
+ val oldLogLevel = logger.getLevel
logger.setLevel(Level.ERROR)
- logger.addAppender(mockAppender)
+ try {
+ logger.addAppender(mockAppender)
- val testOutputStream = new PipedOutputStream()
- val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream
+ val testOutputStream = new PipedOutputStream()
+ val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream
- // Close the stream before appender tries to read will cause an IOException
- testInputStream.close()
- testOutputStream.close()
- val appender = FileAppender(testInputStream, testFile, new SparkConf)
+ // Close the stream before appender tries to read will cause an IOException
+ testInputStream.close()
+ testOutputStream.close()
+ val appender = FileAppender(testInputStream, testFile, new SparkConf)
- // Stop the appender before an IOException is called during read
- testInputStream.latchReadStarted.await()
- appender.stop()
- testInputStream.latchReadProceed.countDown()
+ // Stop the appender before an IOException is called during read
+ testInputStream.latchReadStarted.await()
+ appender.stop()
+ testInputStream.latchReadProceed.countDown()
- appender.awaitTermination()
+ appender.awaitTermination()
- // Make sure no IOException errors have been logged as a result of appender closing gracefully
- verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture)
- import scala.collection.JavaConverters._
- loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent =>
- assert(loggingEvent.getThrowableInformation === null
- || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
+ // Make sure no IOException errors have been logged as a result of appender closing gracefully
+ verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture)
+ import scala.collection.JavaConverters._
+ loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent =>
+ assert(loggingEvent.getThrowableInformation === null
+ || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException])
+ }
+ } finally {
+ logger.setLevel(oldLogLevel)
}
}
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index 6a2d4c9f2c..de6f408fa8 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -853,7 +853,7 @@ private[spark] object JsonProtocolSuite extends Assertions {
if (hasHadoopInput) {
val inputMetrics = t.registerInputMetrics(DataReadMethod.Hadoop)
inputMetrics.setBytesRead(d + e + f)
- inputMetrics.incRecordsReadInternal(if (hasRecords) (d + e + f) / 100 else -1)
+ inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1)
} else {
val sr = t.registerTempShuffleReadMetrics()
sr.incRemoteBytesRead(b + d)
diff --git a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala
new file mode 100644
index 0000000000..39b31f8dde
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala
@@ -0,0 +1,159 @@
+/*
+ * 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.util
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+
+import scala.util.Random
+
+import com.google.common.util.concurrent.Uninterruptibles
+
+import org.apache.spark.SparkFunSuite
+
+class UninterruptibleThreadSuite extends SparkFunSuite {
+
+ /** Sleep millis and return true if it's interrupted */
+ private def sleep(millis: Long): Boolean = {
+ try {
+ Thread.sleep(millis)
+ false
+ } catch {
+ case _: InterruptedException =>
+ true
+ }
+ }
+
+ test("interrupt when runUninterruptibly is running") {
+ val enterRunUninterruptibly = new CountDownLatch(1)
+ @volatile var hasInterruptedException = false
+ @volatile var interruptStatusBeforeExit = false
+ val t = new UninterruptibleThread("test") {
+ override def run(): Unit = {
+ runUninterruptibly {
+ enterRunUninterruptibly.countDown()
+ hasInterruptedException = sleep(1000)
+ }
+ interruptStatusBeforeExit = Thread.interrupted()
+ }
+ }
+ t.start()
+ assert(enterRunUninterruptibly.await(10, TimeUnit.SECONDS), "await timeout")
+ t.interrupt()
+ t.join()
+ assert(hasInterruptedException === false)
+ assert(interruptStatusBeforeExit === true)
+ }
+
+ test("interrupt before runUninterruptibly runs") {
+ val interruptLatch = new CountDownLatch(1)
+ @volatile var hasInterruptedException = false
+ @volatile var interruptStatusBeforeExit = false
+ val t = new UninterruptibleThread("test") {
+ override def run(): Unit = {
+ Uninterruptibles.awaitUninterruptibly(interruptLatch, 10, TimeUnit.SECONDS)
+ try {
+ runUninterruptibly {
+ assert(false, "Should not reach here")
+ }
+ } catch {
+ case _: InterruptedException => hasInterruptedException = true
+ }
+ interruptStatusBeforeExit = Thread.interrupted()
+ }
+ }
+ t.start()
+ t.interrupt()
+ interruptLatch.countDown()
+ t.join()
+ assert(hasInterruptedException === true)
+ assert(interruptStatusBeforeExit === false)
+ }
+
+ test("nested runUninterruptibly") {
+ val enterRunUninterruptibly = new CountDownLatch(1)
+ val interruptLatch = new CountDownLatch(1)
+ @volatile var hasInterruptedException = false
+ @volatile var interruptStatusBeforeExit = false
+ val t = new UninterruptibleThread("test") {
+ override def run(): Unit = {
+ runUninterruptibly {
+ enterRunUninterruptibly.countDown()
+ Uninterruptibles.awaitUninterruptibly(interruptLatch, 10, TimeUnit.SECONDS)
+ hasInterruptedException = sleep(1)
+ runUninterruptibly {
+ if (sleep(1)) {
+ hasInterruptedException = true
+ }
+ }
+ if (sleep(1)) {
+ hasInterruptedException = true
+ }
+ }
+ interruptStatusBeforeExit = Thread.interrupted()
+ }
+ }
+ t.start()
+ assert(enterRunUninterruptibly.await(10, TimeUnit.SECONDS), "await timeout")
+ t.interrupt()
+ interruptLatch.countDown()
+ t.join()
+ assert(hasInterruptedException === false)
+ assert(interruptStatusBeforeExit === true)
+ }
+
+ test("stress test") {
+ @volatile var hasInterruptedException = false
+ val t = new UninterruptibleThread("test") {
+ override def run(): Unit = {
+ for (i <- 0 until 100) {
+ try {
+ runUninterruptibly {
+ if (sleep(Random.nextInt(10))) {
+ hasInterruptedException = true
+ }
+ runUninterruptibly {
+ if (sleep(Random.nextInt(10))) {
+ hasInterruptedException = true
+ }
+ }
+ if (sleep(Random.nextInt(10))) {
+ hasInterruptedException = true
+ }
+ }
+ Uninterruptibles.sleepUninterruptibly(Random.nextInt(10), TimeUnit.MILLISECONDS)
+ // 50% chance to clear the interrupted status
+ if (Random.nextBoolean()) {
+ Thread.interrupted()
+ }
+ } catch {
+ case _: InterruptedException =>
+ // The first runUninterruptibly may throw InterruptedException if the interrupt status
+ // is set before running `f`.
+ }
+ }
+ }
+ }
+ t.start()
+ for (i <- 0 until 400) {
+ Thread.sleep(Random.nextInt(10))
+ t.interrupt()
+ }
+ t.join()
+ assert(hasInterruptedException === false)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala
index 361ec95654..226622075a 100644
--- a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala
@@ -17,48 +17,53 @@
package org.apache.spark.util.io
+import java.nio.ByteBuffer
+
import scala.util.Random
import org.apache.spark.SparkFunSuite
-class ByteArrayChunkOutputStreamSuite extends SparkFunSuite {
+class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite {
test("empty output") {
- val o = new ByteArrayChunkOutputStream(1024)
- assert(o.toArrays.length === 0)
+ val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate)
+ assert(o.toChunkedByteBuffer.size === 0)
}
test("write a single byte") {
- val o = new ByteArrayChunkOutputStream(1024)
+ val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate)
o.write(10)
- assert(o.toArrays.length === 1)
- assert(o.toArrays.head.toSeq === Seq(10.toByte))
+ val chunkedByteBuffer = o.toChunkedByteBuffer
+ assert(chunkedByteBuffer.getChunks().length === 1)
+ assert(chunkedByteBuffer.getChunks().head.array().toSeq === Seq(10.toByte))
}
test("write a single near boundary") {
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(new Array[Byte](9))
o.write(99)
- assert(o.toArrays.length === 1)
- assert(o.toArrays.head(9) === 99.toByte)
+ val chunkedByteBuffer = o.toChunkedByteBuffer
+ assert(chunkedByteBuffer.getChunks().length === 1)
+ assert(chunkedByteBuffer.getChunks().head.array()(9) === 99.toByte)
}
test("write a single at boundary") {
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(new Array[Byte](10))
o.write(99)
- assert(o.toArrays.length === 2)
- assert(o.toArrays(1).length === 1)
- assert(o.toArrays(1)(0) === 99.toByte)
+ val arrays = o.toChunkedByteBuffer.getChunks().map(_.array())
+ assert(arrays.length === 2)
+ assert(arrays(1).length === 1)
+ assert(arrays(1)(0) === 99.toByte)
}
test("single chunk output") {
val ref = new Array[Byte](8)
Random.nextBytes(ref)
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(ref)
- val arrays = o.toArrays
+ val arrays = o.toChunkedByteBuffer.getChunks().map(_.array())
assert(arrays.length === 1)
assert(arrays.head.length === ref.length)
assert(arrays.head.toSeq === ref.toSeq)
@@ -67,9 +72,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite {
test("single chunk output at boundary size") {
val ref = new Array[Byte](10)
Random.nextBytes(ref)
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(ref)
- val arrays = o.toArrays
+ val arrays = o.toChunkedByteBuffer.getChunks().map(_.array())
assert(arrays.length === 1)
assert(arrays.head.length === ref.length)
assert(arrays.head.toSeq === ref.toSeq)
@@ -78,9 +83,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite {
test("multiple chunk output") {
val ref = new Array[Byte](26)
Random.nextBytes(ref)
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(ref)
- val arrays = o.toArrays
+ val arrays = o.toChunkedByteBuffer.getChunks().map(_.array())
assert(arrays.length === 3)
assert(arrays(0).length === 10)
assert(arrays(1).length === 10)
@@ -94,9 +99,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite {
test("multiple chunk output at boundary size") {
val ref = new Array[Byte](30)
Random.nextBytes(ref)
- val o = new ByteArrayChunkOutputStream(10)
+ val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate)
o.write(ref)
- val arrays = o.toArrays
+ val arrays = o.toChunkedByteBuffer.getChunks().map(_.array())
assert(arrays.length === 3)
assert(arrays(0).length === 10)
assert(arrays(1).length === 10)
diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
index 791491daf0..7eb2f56c20 100644
--- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala
@@ -129,6 +129,13 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
t(m / 2)
}
+ def replacementSampling(data: Iterator[Int], sampler: PoissonSampler[Int]): Iterator[Int] = {
+ data.flatMap { item =>
+ val count = sampler.sample()
+ if (count == 0) Iterator.empty else Iterator.fill(count)(item)
+ }
+ }
+
test("utilities") {
val s1 = Array(0, 1, 1, 0, 2)
val s2 = Array(1, 0, 3, 2, 1)
@@ -189,6 +196,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
d should be > D
}
+ test("bernoulli sampling without iterator") {
+ // Tests expect maximum gap sampling fraction to be this value
+ RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
+
+ var d: Double = 0.0
+
+ val data = Iterator.from(0)
+
+ var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.5)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.5)))
+ d should be < D
+
+ sampler = new BernoulliSampler[Int](0.7)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.7)))
+ d should be < D
+
+ sampler = new BernoulliSampler[Int](0.9)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.9)))
+ d should be < D
+
+ // sampling at different frequencies should show up as statistically different:
+ sampler = new BernoulliSampler[Int](0.5)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.6)))
+ d should be > D
+ }
+
test("bernoulli sampling with gap sampling optimization") {
// Tests expect maximum gap sampling fraction to be this value
RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
@@ -217,6 +254,37 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
d should be > D
}
+ test("bernoulli sampling (without iterator) with gap sampling optimization") {
+ // Tests expect maximum gap sampling fraction to be this value
+ RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
+
+ var d: Double = 0.0
+
+ val data = Iterator.from(0)
+
+ var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.01)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)),
+ gaps(sample(Iterator.from(0), 0.01)))
+ d should be < D
+
+ sampler = new BernoulliSampler[Int](0.1)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.1)))
+ d should be < D
+
+ sampler = new BernoulliSampler[Int](0.3)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.3)))
+ d should be < D
+
+ // sampling at different frequencies should show up as statistically different:
+ sampler = new BernoulliSampler[Int](0.3)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.4)))
+ d should be > D
+ }
+
test("bernoulli boundary cases") {
val data = (1 to 100).toArray
@@ -233,6 +301,22 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
sampler.sample(data.iterator).toArray should be (data)
}
+ test("bernoulli (without iterator) boundary cases") {
+ val data = (1 to 100).toArray
+
+ var sampler = new BernoulliSampler[Int](0.0)
+ data.filter(_ => sampler.sample() > 0) should be (Array.empty[Int])
+
+ sampler = new BernoulliSampler[Int](1.0)
+ data.filter(_ => sampler.sample() > 0) should be (data)
+
+ sampler = new BernoulliSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0))
+ data.filter(_ => sampler.sample() > 0) should be (Array.empty[Int])
+
+ sampler = new BernoulliSampler[Int](1.0 + (RandomSampler.roundingEpsilon / 2.0))
+ data.filter(_ => sampler.sample() > 0) should be (data)
+ }
+
test("bernoulli data types") {
// Tests expect maximum gap sampling fraction to be this value
RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
@@ -341,6 +425,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
d should be > D
}
+ test("replacement sampling without iterator") {
+ // Tests expect maximum gap sampling fraction to be this value
+ RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
+
+ var d: Double = 0.0
+
+ val data = Iterator.from(0)
+
+ var sampler = new PoissonSampler[Int](0.5)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.5)))
+ d should be < D
+
+ sampler = new PoissonSampler[Int](0.7)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.7)))
+ d should be < D
+
+ sampler = new PoissonSampler[Int](0.9)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.9)))
+ d should be < D
+
+ // sampling at different frequencies should show up as statistically different:
+ sampler = new PoissonSampler[Int](0.5)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.6)))
+ d should be > D
+ }
+
test("replacement sampling with gap sampling") {
// Tests expect maximum gap sampling fraction to be this value
RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
@@ -369,6 +483,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
d should be > D
}
+ test("replacement sampling (without iterator) with gap sampling") {
+ // Tests expect maximum gap sampling fraction to be this value
+ RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
+
+ var d: Double = 0.0
+
+ val data = Iterator.from(0)
+
+ var sampler = new PoissonSampler[Int](0.01)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.01)))
+ d should be < D
+
+ sampler = new PoissonSampler[Int](0.1)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.1)))
+ d should be < D
+
+ sampler = new PoissonSampler[Int](0.3)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.3)))
+ d should be < D
+
+ // sampling at different frequencies should show up as statistically different:
+ sampler = new PoissonSampler[Int](0.3)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.4)))
+ d should be > D
+ }
+
test("replacement boundary cases") {
val data = (1 to 100).toArray
@@ -383,6 +527,20 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
sampler.sample(data.iterator).length should be > (data.length)
}
+ test("replacement (without) boundary cases") {
+ val data = (1 to 100).toArray
+
+ var sampler = new PoissonSampler[Int](0.0)
+ replacementSampling(data.iterator, sampler).toArray should be (Array.empty[Int])
+
+ sampler = new PoissonSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0))
+ replacementSampling(data.iterator, sampler).toArray should be (Array.empty[Int])
+
+ // sampling with replacement has no upper bound on sampling fraction
+ sampler = new PoissonSampler[Int](2.0)
+ replacementSampling(data.iterator, sampler).length should be > (data.length)
+ }
+
test("replacement data types") {
// Tests expect maximum gap sampling fraction to be this value
RandomSampler.defaultMaxGapSamplingFraction should be (0.4)
@@ -477,6 +635,22 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
d should be < D
}
+ test("bernoulli partitioning sampling without iterator") {
+ var d: Double = 0.0
+
+ val data = Iterator.from(0)
+
+ var sampler = new BernoulliCellSampler[Int](0.1, 0.2)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.1)))
+ d should be < D
+
+ sampler = new BernoulliCellSampler[Int](0.1, 0.2, true)
+ sampler.setSeed(rngSeed.nextLong)
+ d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.9)))
+ d should be < D
+ }
+
test("bernoulli partitioning boundary cases") {
val data = (1 to 100).toArray
val d = RandomSampler.roundingEpsilon / 2.0
@@ -500,6 +674,29 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers {
sampler.sample(data.iterator).toArray should be (Array.empty[Int])
}
+ test("bernoulli partitioning (without iterator) boundary cases") {
+ val data = (1 to 100).toArray
+ val d = RandomSampler.roundingEpsilon / 2.0
+
+ var sampler = new BernoulliCellSampler[Int](0.0, 0.0)
+ data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int])
+
+ sampler = new BernoulliCellSampler[Int](0.5, 0.5)
+ data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int])
+
+ sampler = new BernoulliCellSampler[Int](1.0, 1.0)
+ data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int])
+
+ sampler = new BernoulliCellSampler[Int](0.0, 1.0)
+ data.filter(_ => sampler.sample() > 0).toArray should be (data)
+
+ sampler = new BernoulliCellSampler[Int](0.0 - d, 1.0 + d)
+ data.filter(_ => sampler.sample() > 0).toArray should be (data)
+
+ sampler = new BernoulliCellSampler[Int](0.5, 0.5 - d)
+ data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int])
+ }
+
test("bernoulli partitioning data") {
val seed = rngSeed.nextLong
val data = (1 to 100).toArray