Skip to content

[SPARK-22827][CORE] Avoid throwing OutOfMemoryError in case of exception in spill #20014

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -88,7 +88,7 @@ public void spill() throws IOException {
* `LongArray` is too large to fit in a single page. The caller side should take care of these
* two exceptions, or make sure the `size` is small enough that won't trigger exceptions.
*
* @throws OutOfMemoryError
* @throws SparkOutOfMemoryError
* @throws TooLargePageException
*/
public LongArray allocateArray(long size) {
Expand Down Expand Up @@ -154,6 +154,6 @@ private void throwOom(final MemoryBlock page, final long required) {
taskMemoryManager.freePage(page, this);
}
taskMemoryManager.showMemoryUsage();
throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
throw new SparkOutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
}
}
Original file line number Diff line number Diff line change
@@ -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.memory;

import org.apache.spark.annotation.Private;

/**
* This exception is thrown when a task can not acquire memory from the Memory manager.
* Instead of throwing {@link OutOfMemoryError}, which kills the executor,
* we should use throw this exception, which just kills the current task.
*/
@Private
public final class SparkOutOfMemoryError extends OutOfMemoryError {

public SparkOutOfMemoryError(String s) {
super(s);
}

public SparkOutOfMemoryError(OutOfMemoryError e) {
super(e.getMessage());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
throw new RuntimeException(e.getMessage());
} catch (IOException e) {
logger.error("error while calling spill() on " + c, e);
throw new OutOfMemoryError("error while calling spill() on " + c + " : "
throw new SparkOutOfMemoryError("error while calling spill() on " + c + " : "
+ e.getMessage());
}
}
Expand All @@ -213,7 +213,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
throw new RuntimeException(e.getMessage());
} catch (IOException e) {
logger.error("error while calling spill() on " + consumer, e);
throw new OutOfMemoryError("error while calling spill() on " + consumer + " : "
throw new SparkOutOfMemoryError("error while calling spill() on " + consumer + " : "
+ e.getMessage());
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.spark.executor.ShuffleWriteMetrics;
import org.apache.spark.internal.config.package$;
import org.apache.spark.memory.MemoryConsumer;
import org.apache.spark.memory.SparkOutOfMemoryError;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.memory.TooLargePageException;
import org.apache.spark.serializer.DummySerializerInstance;
Expand Down Expand Up @@ -337,7 +338,7 @@ private void growPointerArrayIfNecessary() throws IOException {
// The pointer array is too big to fix in a single page, spill.
spill();
return;
} catch (OutOfMemoryError e) {
} catch (SparkOutOfMemoryError e) {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.function.Supplier;

import com.google.common.annotations.VisibleForTesting;
import org.apache.spark.memory.SparkOutOfMemoryError;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -349,7 +350,7 @@ private void growPointerArrayIfNecessary() throws IOException {
// The pointer array is too big to fix in a single page, spill.
spill();
return;
} catch (OutOfMemoryError e) {
} catch (SparkOutOfMemoryError e) {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import org.apache.spark.TaskContext;
import org.apache.spark.memory.MemoryConsumer;
import org.apache.spark.memory.SparkOutOfMemoryError;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.UnsafeAlignedOffset;
Expand Down Expand Up @@ -212,7 +213,7 @@ public boolean hasSpaceForAnotherRecord() {

public void expandPointerArray(LongArray newArray) {
if (newArray.size() < array.size()) {
throw new OutOfMemoryError("Not enough memory to grow pointer array");
throw new SparkOutOfMemoryError("Not enough memory to grow pointer array");
}
Platform.copyMemory(
array.getBaseObject(),
Expand Down
5 changes: 2 additions & 3 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
import org.apache.spark.shuffle.FetchFailedException
Expand Down Expand Up @@ -553,10 +553,9 @@ private[spark] class Executor(

// Don't forcibly exit unless the exception was inherently fatal, to avoid
// stopping other tasks unnecessarily.
if (Utils.isFatalError(t)) {
if (!t.isInstanceOf[SparkOutOfMemoryError] && Utils.isFatalError(t)) {
uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), t)
}

} finally {
runningTasks.remove(taskId)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.aggregate

import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import org.apache.spark.memory.SparkOutOfMemoryError
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
Expand Down Expand Up @@ -205,7 +206,7 @@ class TungstenAggregationIterator(
buffer = hashMap.getAggregationBufferFromUnsafeRow(groupingKey)
if (buffer == null) {
// failed to allocate the first page
throw new OutOfMemoryError("No enough memory for aggregation")
throw new SparkOutOfMemoryError("No enough memory for aggregation")
}
}
processRow(buffer, newInput)
Expand Down