1
0

[HUDI-2439] Remove SparkBoundedInMemoryExecutor (#4860)

This commit is contained in:
Raymond Xu
2022-02-26 05:02:12 -08:00
committed by GitHub
parent 1379300b5b
commit c77b2591d0
14 changed files with 99 additions and 113 deletions

View File

@@ -1,56 +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.hudi.execution;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
import java.util.Iterator;
import java.util.function.Function;
public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecutor<I, O, E> {
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
// TaskContext properties.
final TaskContext sparkThreadTaskContext;
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
}
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, BoundedInMemoryQueueProducer<I> producer,
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
super(hoodieConfig.getWriteBufferLimitBytes(), producer, Option.of(consumer), bufferedIteratorTransform);
this.sparkThreadTaskContext = TaskContext.get();
}
@Override
public void preExecute() {
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties.
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
}
}

View File

@@ -18,7 +18,6 @@
package org.apache.hudi.execution;
import org.apache.avro.Schema;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.TaskContextSupplier;
@@ -30,6 +29,8 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.WriteHandleFactory;
import org.apache.hudi.table.HoodieTable;
import org.apache.avro.Schema;
import java.util.Iterator;
import java.util.List;
@@ -84,8 +85,8 @@ public class SparkLazyInsertIterable<T extends HoodieRecordPayload> extends Hood
schema = HoodieAvroUtils.addMetadataFields(schema);
}
bufferedIteratorExecutor =
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(),
getTransformFunction(schema, hoodieConfig));
new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, getInsertHandler(),
getTransformFunction(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable());
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;

View File

@@ -18,8 +18,6 @@
package org.apache.hudi.table;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.data.HoodieData;
@@ -39,6 +37,11 @@ import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.avro.specific.SpecificRecordBase;
import org.apache.hadoop.fs.Path;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
@@ -111,8 +114,8 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
* @return instance of {@link HoodieTableMetadataWriter}
*/
@Override
public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
Option<T> actionMetadata) {
public <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
Option<R> actionMetadata) {
if (config.isMetadataTableEnabled()) {
// Create the metadata table writer. First time after the upgrade this creation might trigger
// metadata table bootstrapping. Bootstrapping process could fail and checking the table
@@ -132,4 +135,10 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
return Option.empty();
}
@Override
public Runnable getPreExecuteRunnable() {
final TaskContext taskContext = TaskContext.get();
return () -> TaskContext$.MODULE$.setTaskContext(taskContext);
}
}

View File

@@ -29,7 +29,6 @@ import org.apache.hudi.common.util.OrcReaderIterator;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieBootstrapHandle;
import org.apache.hudi.keygen.KeyGeneratorInterface;
import org.apache.hudi.table.HoodieTable;
@@ -68,7 +67,7 @@ class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf()));
TypeDescription orcSchema = orcReader.getSchema();
try (RecordReader reader = orcReader.rows(new Reader.Options(table.getHadoopConf()).schema(orcSchema))) {
wrapper = new SparkBoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config,
wrapper = new BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config.getWriteBufferLimitBytes(),
new OrcReaderIterator(reader, avroSchema, orcSchema), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
String recKey = keyGenerator.getKey(inp).getRecordKey();
GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA);
@@ -76,7 +75,7 @@ class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
BootstrapRecordPayload payload = new BootstrapRecordPayload(gr);
HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload);
return rec;
});
}, table.getPreExecuteRunnable());
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);

View File

@@ -28,7 +28,6 @@ import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieBootstrapHandle;
import org.apache.hudi.keygen.KeyGeneratorInterface;
import org.apache.hudi.table.HoodieTable;
@@ -72,7 +71,7 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
try {
ParquetReader<IndexedRecord> reader =
AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build();
wrapper = new SparkBoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config,
wrapper = new BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config.getWriteBufferLimitBytes(),
new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
String recKey = keyGenerator.getKey(inp).getRecordKey();
GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA);
@@ -80,7 +79,7 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
BootstrapRecordPayload payload = new BootstrapRecordPayload(gr);
HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload);
return rec;
});
}, table.getPreExecuteRunnable());
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);

View File

@@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
@@ -90,13 +89,13 @@ public class SparkMergeHelper<T extends HoodieRecordPayload> extends BaseMergeHe
ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator,
wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
new UpdateHandler(mergeHandle), record -> {
if (!externalSchemaTransformation) {
return record;
}
return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record);
});
}, table.getPreExecuteRunnable());
wrapper.execute();
} catch (Exception e) {
throw new HoodieException(e);