Skip to content
Merged
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
@@ -0,0 +1,193 @@
/*
* 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.iceberg.spark.extensions;

import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.AssertHelpers;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.hadoop.HadoopFileIO;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.io.SupportsBulkOperations;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.spark.SparkCatalog;
import org.apache.iceberg.spark.SparkWriteOptions;
import org.apache.iceberg.spark.source.SimpleRecord;
import org.apache.spark.SparkException;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.junit.After;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runners.Parameterized;

public class TestWriteAborts extends SparkExtensionsTestBase {

@Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
public static Object[][] parameters() {
return new Object[][] {
{
"testhive",
SparkCatalog.class.getName(),
ImmutableMap.of(
"type",
"hive",
CatalogProperties.FILE_IO_IMPL,
CustomFileIO.class.getName(),
"default-namespace",
"default")
},
{
"testhivebulk",
SparkCatalog.class.getName(),
ImmutableMap.of(
"type",
"hive",
CatalogProperties.FILE_IO_IMPL,
CustomBulkFileIO.class.getName(),
"default-namespace",
"default")
}
};
}

@Rule public TemporaryFolder temp = new TemporaryFolder();

public TestWriteAborts(String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
}

@After
public void removeTables() {
sql("DROP TABLE IF EXISTS %s", tableName);
}

@Test
public void testBatchAppend() throws Exception {
String dataLocation = temp.newFolder().toString();

sql(
"CREATE TABLE %s (id INT, data STRING) "
+ "USING iceberg "
+ "PARTITIONED BY (data)"
+ "TBLPROPERTIES ('%s' '%s')",
tableName, TableProperties.WRITE_DATA_LOCATION, dataLocation);

List<SimpleRecord> records =
ImmutableList.of(
new SimpleRecord(1, "a"),
new SimpleRecord(2, "b"),
new SimpleRecord(3, "a"),
new SimpleRecord(4, "b"));
Dataset<Row> inputDF = spark.createDataFrame(records, SimpleRecord.class);

AssertHelpers.assertThrows(
"Write must fail",
SparkException.class,
"Writing job aborted",
() -> {
try {
// incoming records are not ordered by partitions so the job must fail
inputDF
.coalesce(1)
.sortWithinPartitions("id")
.writeTo(tableName)
.option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false")
.append();
} catch (NoSuchTableException e) {
throw new RuntimeException(e);
}
});

assertEquals("Should be no records", sql("SELECT * FROM %s", tableName), ImmutableList.of());

assertEquals(
"Should be no orphan data files",
ImmutableList.of(),
sql(
"CALL %s.system.remove_orphan_files(table => '%s', older_than => %dL, location => '%s')",
catalogName, tableName, System.currentTimeMillis() + 5000, dataLocation));
}

public static class CustomFileIO implements FileIO {

private final FileIO delegate = new HadoopFileIO(new Configuration());

public CustomFileIO() {}

protected FileIO delegate() {
return delegate;
}

@Override
public InputFile newInputFile(String path) {
return delegate.newInputFile(path);
}

@Override
public OutputFile newOutputFile(String path) {
return delegate.newOutputFile(path);
}

@Override
public void deleteFile(String path) {
delegate.deleteFile(path);
}

@Override
public Map<String, String> properties() {
return delegate.properties();
}

@Override
public void initialize(Map<String, String> properties) {
delegate.initialize(properties);
}

@Override
public void close() {
delegate.close();
}
}

public static class CustomBulkFileIO extends CustomFileIO implements SupportsBulkOperations {

public CustomBulkFileIO() {}

@Override
public void deleteFile(String path) {
throw new UnsupportedOperationException("Only bulk deletes are supported");
}

@Override
public void deleteFiles(Iterable<String> paths) throws BulkDeletionFailureException {
for (String path : paths) {
delegate().deleteFile(path);
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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.iceberg.spark.source;

import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.iceberg.ContentFile;
import org.apache.iceberg.exceptions.NotFoundException;
import org.apache.iceberg.io.BulkDeletionFailureException;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.SupportsBulkOperations;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.util.Tasks;
import org.apache.iceberg.util.ThreadPools;
import org.apache.spark.TaskContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/** A utility for cleaning up written but not committed files. */
class SparkCleanupUtil {

private static final Logger LOG = LoggerFactory.getLogger(SparkCleanupUtil.class);

private static final int DELETE_NUM_RETRIES = 3;
private static final int DELETE_MIN_RETRY_WAIT_MS = 100; // 100 ms
private static final int DELETE_MAX_RETRY_WAIT_MS = 30 * 1000; // 30 seconds
private static final int DELETE_TOTAL_RETRY_TIME_MS = 2 * 60 * 1000; // 2 minutes

private SparkCleanupUtil() {}

/**
* Attempts to delete as many files produced by a task as possible.
*
* <p>Note this method will log Spark task info and is supposed to be called only on executors.
* Use {@link #deleteFiles(String, FileIO, List)} to delete files on the driver.
*
* @param io a {@link FileIO} instance used for deleting files
* @param files a list of files to delete
*/
public static void deleteTaskFiles(FileIO io, List<? extends ContentFile<?>> files) {
deleteFiles(taskInfo(), io, files);
}

// the format matches what Spark uses for internal logging
private static String taskInfo() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Nit: what do you think to move private method to bottom? Breaks the flow of code a bit (would have liked to see deleteFiles right after deleteTaskFiles as its the main delegate)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I was trying to group methods by logic instead of access. My reasoning here was that taskInfo() is only invoked in this method is directly related to deleteTaskFiles(). Let me know if that makes sense.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Yea , its definitely subjective, I prefer personally to see the public methods and their javadocs first to get a high level idea of what the class before diving in to details (especially given there's only two public methods in this class). But as its style preference, I'll leave it optional then.

TaskContext taskContext = TaskContext.get();
if (taskContext == null) {
return "unknown task";
} else {
return String.format(
"partition %d (task %d, attempt %d, stage %d.%d)",
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

can we show stage attempID better something like :
Task (id : <TaskID>, attempt : <attemptNumber>), Stage (id : <stageId>, attemp : <attempNumber>)
in place of
(task 0, attempt 0, stage 0.0)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

My idea is to follow the exact format used in Spark so that we can easily match Spark and Iceberg logs.

[Executor task launch worker for task 0.0 in stage 0.0 (TID 0)] ERROR org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask - Aborting commit for partition 0 (task 0, attempt 0, stage 0.0)
[Executor task launch worker for task 0.0 in stage 0.0 (TID 0)] INFO org.apache.iceberg.spark.source.SparkCleanupUtil - Deleted 2 file(s) (partition 0 (task 0, attempt 0, stage 0.0))

In this example, it is clear that these two records belong to the same context, even though they were produced by Spark and Iceberg. If we change the format, it won't be obvious.

taskContext.partitionId(),
taskContext.taskAttemptId(),
taskContext.attemptNumber(),
taskContext.stageId(),
taskContext.stageAttemptNumber());
}
}

/**
* Attempts to delete as many given files as possible.
*
* @param context a helpful description of the operation invoking this method
* @param io a {@link FileIO} instance used for deleting files
* @param files a list of files to delete
*/
public static void deleteFiles(String context, FileIO io, List<? extends ContentFile<?>> files) {
List<String> paths = Lists.transform(files, file -> file.path().toString());
deletePaths(context, io, paths);
}

private static void deletePaths(String context, FileIO io, List<String> paths) {
if (io instanceof SupportsBulkOperations) {
SupportsBulkOperations bulkIO = (SupportsBulkOperations) io;
bulkDelete(context, bulkIO, paths);
} else {
delete(context, io, paths);
}
}

private static void bulkDelete(String context, SupportsBulkOperations io, List<String> paths) {
try {
io.deleteFiles(paths);
LOG.info("Deleted {} file(s) using bulk deletes ({})", paths.size(), context);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Nit: unnecessary newline

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

We do this sometimes when either the try or catch block are non-trivial to separate them.

} catch (BulkDeletionFailureException e) {
int deletedFilesCount = paths.size() - e.numberFailedObjects();
LOG.warn(
"Deleted only {} of {} file(s) using bulk deletes ({})",
deletedFilesCount,
paths.size(),
context);
}
}

private static void delete(String context, FileIO io, List<String> paths) {
AtomicInteger deletedFilesCount = new AtomicInteger(0);

Tasks.foreach(paths)
.executeWith(ThreadPools.getWorkerPool())
.stopRetryOn(NotFoundException.class)
.suppressFailureWhenFinished()
.onFailure((path, exc) -> LOG.warn("Failed to delete {} ({})", path, context, exc))
.retry(DELETE_NUM_RETRIES)
.exponentialBackoff(
DELETE_MIN_RETRY_WAIT_MS,
DELETE_MAX_RETRY_WAIT_MS,
DELETE_TOTAL_RETRY_TIME_MS,
2 /* exponential */)
.run(
path -> {
io.deleteFile(path);
deletedFilesCount.incrementAndGet();
});

if (deletedFilesCount.get() < paths.size()) {
LOG.warn("Deleted only {} of {} file(s) ({})", deletedFilesCount, paths.size(), context);
} else {
LOG.info("Deleted {} file(s) ({})", paths.size(), context);
}
}
}
Loading