Skip to content
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

[HUDI-5238] Fixing HoodieMergeHandle shutdown sequence #7245

Merged
merged 7 commits into from May 19, 2023
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -0,0 +1,41 @@
/*
* 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.hudi.client.utils;

import org.apache.hudi.common.util.collection.ClosableIterator;

import java.util.function.BiFunction;

/**
* Closeable counterpart of {@link MergingIterator}
*/
public class ClosableMergingIterator<T1, T2, R> extends MergingIterator<T1, T2, R> implements ClosableIterator<R> {

public ClosableMergingIterator(ClosableIterator<T1> leftIterator,
ClosableIterator<T2> rightIterator,
BiFunction<T1, T2, R> mergeFunction) {
super(leftIterator, rightIterator, mergeFunction);
}

@Override
public void close() {
((ClosableIterator<T1>) leftIterator).close();
((ClosableIterator<T2>) rightIterator).close();
}
}
Expand Up @@ -18,19 +18,27 @@

package org.apache.hudi.client.utils;

import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.ValidationUtils;

import java.util.Iterator;
import java.util.function.BiFunction;

public class MergingIterator<T extends HoodieRecord> implements Iterator<T> {
/**
* Iterator providing for the semantic of simultaneously iterating over 2 other iterators
* and combining their respective output
*
* @param <T1> type returned by the first iterator
* @param <T2> type returned by the second iterator
* @param <R> type returned by this iterator
*/
public class MergingIterator<T1, T2, R> implements Iterator<R> {

protected final Iterator<T1> leftIterator;
protected final Iterator<T2> rightIterator;

private final Iterator<T> leftIterator;
private final Iterator<T> rightIterator;
private final BiFunction<T, T, T> mergeFunction;
private final BiFunction<T1, T2, R> mergeFunction;

public MergingIterator(Iterator<T> leftIterator, Iterator<T> rightIterator, BiFunction<T, T, T> mergeFunction) {
public MergingIterator(Iterator<T1> leftIterator, Iterator<T2> rightIterator, BiFunction<T1, T2, R> mergeFunction) {
this.leftIterator = leftIterator;
this.rightIterator = rightIterator;
this.mergeFunction = mergeFunction;
Expand All @@ -45,7 +53,7 @@ public boolean hasNext() {
}

@Override
public T next() {
public R next() {
return mergeFunction.apply(leftIterator.next(), rightIterator.next());
}
}
Expand Up @@ -65,6 +65,7 @@

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
Expand Down Expand Up @@ -498,21 +499,26 @@ protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props
@Override
public List<WriteStatus> close() {
try {
if (isClosed()) {
// Handle has already been closed
return Collections.emptyList();
}

markClosed();
// flush any remaining records to disk
appendDataAndDeleteBlocks(header, true);
recordItr = null;
if (writer != null) {
writer.close();
writer = null;

// update final size, once for all log files
// TODO we can actually deduce file size purely from AppendResult (based on offset and size
// of the appended block)
for (WriteStatus status : statuses) {
long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath()));
status.getStat().setFileSizeInBytes(logFileSize);
}

writer.close();

// update final size, once for all log files
// TODO we can actually deduce file size purely from AppendResult (based on offset and size
// of the appended block)
for (WriteStatus status : statuses) {
long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath()));
status.getStat().setFileSizeInBytes(logFileSize);
}

return statuses;
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
Expand Down
Expand Up @@ -203,6 +203,12 @@ public IOType getIOType() {
public List<WriteStatus> close() {
LOG.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
try {
if (isClosed()) {
// Handle has already been closed
return Collections.emptyList();
}

markClosed();

if (fileWriter != null) {
fileWriter.close();
Expand Down
Expand Up @@ -407,6 +407,12 @@ protected void writeIncomingRecords() throws IOException {
@Override
public List<WriteStatus> close() {
try {
if (isClosed()) {
// Handle has already been closed
return Collections.emptyList();
}

markClosed();
writeIncomingRecords();

if (keyToNewRecords instanceof ExternalSpillableMap) {
Expand All @@ -416,10 +422,8 @@ public List<WriteStatus> close() {
keyToNewRecords = null;
writtenRecordKeys = null;

if (fileWriter != null) {
fileWriter.close();
fileWriter = null;
}
fileWriter.close();
fileWriter = null;

long fileSizeInBytes = FSUtils.getFileSize(fs, newFilePath);
HoodieWriteStat stat = writeStatus.getStat();
Expand Down
Expand Up @@ -76,6 +76,8 @@ public abstract class HoodieWriteHandle<T, I, K, O> extends HoodieIOHandle<T, I,
// For full schema evolution
protected final boolean schemaOnReadEnabled;

private boolean closed = false;

public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
String fileId, HoodieTable<T, I, K, O> hoodieTable, TaskContextSupplier taskContextSupplier) {
this(config, instantTime, partitionPath, fileId, hoodieTable,
Expand Down Expand Up @@ -175,6 +177,14 @@ public void write(HoodieRecord record, Schema schema, TypedProperties props) {
doWrite(record, schema, props);
}

protected boolean isClosed() {
return closed;
}

protected void markClosed() {
this.closed = true;
}

public abstract List<WriteStatus> close();

public List<WriteStatus> writeStatuses() {
Expand Down
Expand Up @@ -41,6 +41,7 @@ public void consume(HoodieRecord record) {

@Override
public Void finish() {
bootstrapHandle.close();
return null;
}
}
Expand Up @@ -56,6 +56,7 @@ public void consume(HoodieRecord record) {

@Override
public Void finish() {
upsertHandle.close();
return null;
}
}
Expand Down
Expand Up @@ -18,14 +18,15 @@

package org.apache.hudi.table.action.commit;

import org.apache.hudi.client.utils.ClosableMergingIterator;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.common.util.InternalSchemaCache;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ClosableIterator;
import org.apache.hudi.common.util.queue.HoodieExecutor;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
Expand All @@ -49,7 +50,6 @@
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand Down Expand Up @@ -105,10 +105,10 @@ public void runMerge(HoodieTable<?, ?, ?, ?> table,
|| !isPureProjection
|| baseFile.getBootstrapBaseFile().isPresent();

HoodieExecutor<Void> wrapper = null;
HoodieExecutor<Void> executor = null;

try {
Iterator<HoodieRecord> recordIterator;
ClosableIterator<HoodieRecord> recordIterator;

// In case writer's schema is simply a projection of the reader's one we can read
// the records in the projected schema directly
Expand All @@ -123,16 +123,19 @@ public void runMerge(HoodieTable<?, ?, ?, ?> table,
HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(bootstrapFileConfig, bootstrapFilePath),
mergeHandle.getPartitionFields(),
mergeHandle.getPartitionValues());
recordIterator = bootstrapFileReader.getRecordIterator(mergeHandle.getWriterSchemaWithMetaFields());
recordSchema = mergeHandle.getWriterSchemaWithMetaFields();
recordIterator = new ClosableMergingIterator<>(
baseFileRecordIterator,
(ClosableIterator<HoodieRecord>) bootstrapFileReader.getRecordIterator(recordSchema),
(left, right) -> left.joinWith(right, recordSchema));
} else {
recordIterator = baseFileRecordIterator;
recordSchema = isPureProjection ? writerSchema : readerSchema;
}

boolean isBufferingRecords = ExecutorFactory.isBufferingRecords(writeConfig);

wrapper = ExecutorFactory.create(writeConfig, recordIterator, new UpdateHandler(mergeHandle), record -> {
executor = ExecutorFactory.create(writeConfig, recordIterator, new UpdateHandler(mergeHandle), record -> {
HoodieRecord newRecord;
if (schemaEvolutionTransformerOpt.isPresent()) {
newRecord = schemaEvolutionTransformerOpt.get().apply(record);
Expand All @@ -148,21 +151,19 @@ public void runMerge(HoodieTable<?, ?, ?, ?> table,
return isBufferingRecords ? newRecord.copy() : newRecord;
}, table.getPreExecuteRunnable());

wrapper.execute();
executor.execute();
} catch (Exception e) {
throw new HoodieException(e);
} finally {
// HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting
// and executor firstly and then close mergeHandle.
baseFileReader.close();
if (bootstrapFileReader != null) {
bootstrapFileReader.close();
}
if (null != wrapper) {
wrapper.shutdownNow();
wrapper.awaitTermination();
// NOTE: If executor is initialized it's responsible for gracefully shutting down
// both producer and consumer
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
} else {
baseFileReader.close();
mergeHandle.close();
}
mergeHandle.close();
}
}

Expand Down
Expand Up @@ -56,20 +56,20 @@ public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
@Override
protected List<WriteStatus> computeNext() {
// Executor service used for launching writer thread.
HoodieExecutor<List<WriteStatus>> bufferedIteratorExecutor = null;
HoodieExecutor<List<WriteStatus>> executor = null;
try {
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
bufferedIteratorExecutor = ExecutorFactory.create(hoodieConfig, inputItr, getExplicitInsertHandler(),
Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
executor = ExecutorFactory.create(hoodieConfig, inputItr, getExplicitInsertHandler(),
getTransformer(schema, hoodieConfig));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
final List<WriteStatus> result = executor.execute();
checkState(result != null && !result.isEmpty());
return result;
} catch (Exception e) {
throw new HoodieException(e);
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
}
}
}
Expand Down
Expand Up @@ -66,7 +66,7 @@ public HoodieWriteMetadata<List<WriteStatus>> execute() {
DeletePartitionUtils.checkForPendingTableServiceActions(table, partitions);

try {
HoodieTimer timer = new HoodieTimer().startTimer();
HoodieTimer timer = HoodieTimer.start();
context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions.");
Map<String, List<String>> partitionToReplaceFileIds =
context.parallelize(partitions).distinct().collectAsList()
Expand Down
Expand Up @@ -59,21 +59,20 @@ public JavaLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
@Override
protected List<WriteStatus> computeNext() {
// Executor service used for launching writer thread.
HoodieExecutor<List<WriteStatus>> bufferedIteratorExecutor =
HoodieExecutor<List<WriteStatus>> executor =
null;
try {
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
bufferedIteratorExecutor =
ExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getTransformer(schema, hoodieConfig));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
executor = ExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getTransformer(schema, hoodieConfig));
final List<WriteStatus> result = executor.execute();
checkState(result != null && !result.isEmpty());
return result;
} catch (Exception e) {
throw new HoodieException(e);
} finally {
if (null != bufferedIteratorExecutor) {
bufferedIteratorExecutor.shutdownNow();
bufferedIteratorExecutor.awaitTermination();
if (executor != null) {
executor.shutdownNow();
executor.awaitTermination();
}
}
}
Expand Down