Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
00bcf8a
Avoid IO operations on empty files in BlockObjectWriter.
JoshRosen Apr 21, 2015
8fd89b4
Do not create empty files at all.
JoshRosen Apr 21, 2015
0db87c3
Reduce scope of FileOutputStream in ExternalSorter
JoshRosen Apr 21, 2015
7e2340d
Revert "Reduce scope of FileOutputStream in ExternalSorter"
JoshRosen Apr 22, 2015
8113eac
Hacky WIP towards speculatively running w/o reset(), then retrying wi…
JoshRosen Jun 5, 2015
417f50e
Hackily comment out most of dev/run-tests to speed up Jenkins iteration.
JoshRosen Jun 5, 2015
f90dc94
Don't pass configuration to ObjectWritable in SerializableWritable
JoshRosen Jun 5, 2015
480d20a
Broadcast configuration in hiveWriterContainers (WIP hack)
JoshRosen Jun 5, 2015
55041d2
Use local[*] instead of local[2]
JoshRosen Jun 5, 2015
57c2cb4
try in-memory Derby
JoshRosen Jun 5, 2015
9db0abc
Avoid writing empty files in BypassMergeSortShuffleWriter
JoshRosen Jun 5, 2015
9e116d1
Rework SPARK-7041 for BypassMergeSort split
JoshRosen Jun 5, 2015
3fe16e8
Revert "Broadcast configuration in hiveWriterContainers (WIP hack)"
JoshRosen Jun 5, 2015
54cd5ce
Merge remote-tracking branch 'origin/master' into file-handle-optimiz…
JoshRosen Jun 5, 2015
5c777cf
Rework SPARK-7041 for BypassMergeSort split
JoshRosen Jun 5, 2015
5ac11d1
Revert "Use local[*] instead of local[2]"
JoshRosen Jun 5, 2015
895de59
Revert "try in-memory Derby"
JoshRosen Jun 5, 2015
bf30fee
Revert "Don't pass configuration to ObjectWritable in SerializableWri…
JoshRosen Jun 5, 2015
b1e3f82
Revert "Hacky WIP towards speculatively running w/o reset(), then ret…
JoshRosen Jun 5, 2015
5113370
Revert "Rework SPARK-7041 for BypassMergeSort split"
JoshRosen Jun 5, 2015
fac08d5
SPARK-8135. In SerializableWritable, don't load defaults when instant…
sryza Jun 5, 2015
c7caa5c
Merge remote-tracking branch 'origin/master' into file-handle-optimiz…
JoshRosen Jun 6, 2015
8c1e1ff
Merge branch 'file-handle-optimizations' into hive-compat-suite-speedup
JoshRosen Jun 8, 2015
2b500b9
Only run unsafe tests (testing a jenkins job)
JoshRosen Jun 12, 2015
fbd3d03
Add log4j test properties to unsafe project.
JoshRosen Jun 12, 2015
46dd005
Try only testing bagel instead (since I don't think Maven logs to uni…
JoshRosen Jun 12, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -107,8 +107,16 @@ public void insertAll(Iterator<Product2<K, V>> records) throws IOException {
blockManager.diskBlockManager().createTempShuffleBlock();
final File file = tempShuffleBlockIdPlusFile._2();
final BlockId blockId = tempShuffleBlockIdPlusFile._1();
// Note that we purposely do not call open() on the disk writers here; DiskBlockObjectWriter
// will automatically open() itself if necessary. This is an optimization to avoid file
// creation and truncation for empty partitions; this optimization probably doesn't make sense
// for most realistic production workloads, but it can make a large difference when playing
// around with Spark SQL queries in spark-shell on toy datasets: if you performed a query over
// an extremely small number of records then Spark SQL's default parallelism of 200 would
// result in slower out-of-the-box performance due to these constant-factor overheads. This
// optimization speeds up local microbenchmarking and SQL unit tests.
partitionWriters[i] =
blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics).open();
blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics);
}
// Creating the file to write to and creating a disk writer both involve interacting with
// the disk, and can take a long time in aggregate when we open many files, so should be
Expand Down Expand Up @@ -143,6 +151,13 @@ public long[] writePartitionedFile(
boolean threwException = true;
try {
for (int i = 0; i < numPartitions; i++) {
if (partitionWriters[i].fileSegment().length() == 0) {
// In insertAll(), we didn't create empty files for empty reduce partitions; this branch
// handles that case. Since we'll be skipping deletion of these files, verify that they
// don't exist:
assert(!partitionWriters[i].fileSegment().file().exists());
continue;
}
final FileInputStream in = new FileInputStream(partitionWriters[i].fileSegment().file());
boolean copyThrewException = true;
try {
Expand Down Expand Up @@ -172,7 +187,8 @@ public void stop() throws IOException {
for (BlockObjectWriter writer : partitionWriters) {
// This method explicitly does _not_ throw exceptions:
writer.revertPartialWritesAndClose();
if (!diskBlockManager.getFile(writer.blockId()).delete()) {
final File file = diskBlockManager.getFile(writer.blockId());
if (file.exists() && !file.delete()) {
logger.error("Error while deleting file for block {}", writer.blockId());
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class SerializableWritable[T <: Writable](@transient var t: T) extends Serializa
private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException {
in.defaultReadObject()
val ow = new ObjectWritable()
ow.setConf(new Configuration())
ow.setConf(new Configuration(false))
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that this, and SerilaizableWritable more generally, may be a huge source of perf. bottlenecks for short tasks. A common use of SerializableWritable is in serializing Hadoop Configurations, but it seems kind of crazy to create and discard a new Configuration in order to be able to deserialize the driver-provided conf. Maybe we can make a substitute for SerializableWritable which only deals with Configuration subclasses and just calls writeFields() and readFields() directly. This would sidestep a lot of the performance penalties involved in creating Configuration objects and having them spend tons of time loading defaults.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apparently other folks have noticed Configuration's expensive instantiation costs, too: https://issues.apache.org/jira/browse/MAPREDUCE-5399

ow.readFields(in)
t = ow.get().asInstanceOf[T]
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -187,13 +187,12 @@ private[spark] class DiskBlockObjectWriter(
objOut.flush()
bs.flush()
close()
}

val truncateStream = new FileOutputStream(file, true)
try {
truncateStream.getChannel.truncate(initialPosition)
} finally {
truncateStream.close()
val truncateStream = new FileOutputStream(file, true)
try {
truncateStream.getChannel.truncate(initialPosition)
} finally {
truncateStream.close()
}
}
} catch {
case e: Exception =>
Expand Down
108 changes: 2 additions & 106 deletions dev/run-tests
Original file line number Diff line number Diff line change
Expand Up @@ -77,69 +77,10 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl"
fi
}

# Only run Hive tests if there are SQL changes.
# Partial solution for SPARK-1455.
if [ -n "$AMPLAB_JENKINS" ]; then
target_branch="$ghprbTargetBranch"
git fetch origin "$target_branch":"$target_branch"

# AMP_JENKINS_PRB indicates if the current build is a pull request build.
if [ -n "$AMP_JENKINS_PRB" ]; then
# It is a pull request build.
sql_diffs=$(
git diff --name-only "$target_branch" \
| grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh"
)

non_sql_diffs=$(
git diff --name-only "$target_branch" \
| grep -v -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh"
)

if [ -n "$sql_diffs" ]; then
echo "[info] Detected changes in SQL. Will run Hive test suite."
_RUN_SQL_TESTS=true

if [ -z "$non_sql_diffs" ]; then
echo "[info] Detected no changes except in SQL. Will only run SQL tests."
_SQL_TESTS_ONLY=true
fi
fi
else
# It is a regular build. We should run SQL tests.
_RUN_SQL_TESTS=true
fi
fi

set -o pipefail
trap 'handle_error $LINENO' ERR

echo ""
echo "========================================================================="
echo "Running Apache RAT checks"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_RAT

./dev/check-license

echo ""
echo "========================================================================="
echo "Running Scala style checks"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_SCALA_STYLE

./dev/lint-scala

echo ""
echo "========================================================================="
echo "Running Python style checks"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_PYTHON_STYLE

./dev/lint-python

echo ""
echo "========================================================================="
Expand All @@ -163,15 +104,6 @@ CURRENT_BLOCK=$BLOCK_BUILD
fi
}

echo ""
echo "========================================================================="
echo "Detecting binary incompatibilities with MiMa"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_MIMA

./dev/mima

echo ""
echo "========================================================================="
echo "Running Spark unit tests"
Expand All @@ -180,19 +112,8 @@ echo "========================================================================="
CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS

{
# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled.
# This must be a single argument, as it is.
if [ -n "$_RUN_SQL_TESTS" ]; then
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver"
fi

if [ -n "$_SQL_TESTS_ONLY" ]; then
# This must be an array of individual arguments. Otherwise, having one long string
# will be interpreted as a single test, which doesn't work.
SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test" "mllib/test")
else
SBT_MAVEN_TEST_ARGS=("test")
fi
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver"
SBT_MAVEN_TEST_ARGS=("bagel/test")

echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}"

Expand All @@ -213,28 +134,3 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS
fi
}

echo ""
echo "========================================================================="
echo "Running PySpark tests"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS

# add path for python 3 in jenkins
export PATH="${PATH}:/home/anaconda/envs/py3k/bin"
./python/run-tests

echo ""
echo "========================================================================="
echo "Running SparkR tests"
echo "========================================================================="

CURRENT_BLOCK=$BLOCK_SPARKR_UNIT_TESTS

if [ $(command -v R) ]; then
./R/install-dev.sh
./R/run-tests.sh
else
echo "Ignoring SparkR tests as R was not found in PATH"
fi

28 changes: 28 additions & 0 deletions unsafe/src/test/resources/log4j.properties
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
#
# 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.
#

# Set everything to be logged to the file target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=true
log4j.appender.file.file=target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %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