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,65 @@
/*
* 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.client.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
import org.apache.hudi.io.HoodieWriteHandle;

import java.util.ArrayList;
import java.util.List;

/**
* Consumes stream of hoodie records from in-memory queue and writes to one explicit create handle.
*/
public class ExplicitWriteHandler<T extends HoodieRecordPayload>
extends BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {

private final List<WriteStatus> statuses = new ArrayList<>();

private HoodieWriteHandle handle;

public ExplicitWriteHandler(HoodieWriteHandle handle) {
this.handle = handle;
}

@Override
public void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload) {
final HoodieRecord insertPayload = payload.record;
handle.write(insertPayload, payload.insertValue, payload.exception);
}

@Override
public void finish() {
closeOpenHandle();
assert statuses.size() > 0;
}

@Override
public List<WriteStatus> getResult() {
return statuses;
}

private void closeOpenHandle() {
statuses.addAll(handle.close());
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,8 @@
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.WriteHandleFactory;
import org.apache.hudi.io.ExplicitWriteHandleFactory;
import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;

import org.apache.avro.Schema;
Expand All @@ -36,15 +37,6 @@
import java.util.List;

public class FlinkLazyInsertIterable<T extends HoodieRecordPayload> extends HoodieLazyInsertIterable<T> {
public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
boolean areRecordsSorted,
HoodieWriteConfig config,
String instantTime,
HoodieTable hoodieTable,
String idPrefix,
TaskContextSupplier taskContextSupplier) {
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier);
}

public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
boolean areRecordsSorted,
Expand All @@ -53,7 +45,7 @@ public FlinkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
HoodieTable hoodieTable,
String idPrefix,
TaskContextSupplier taskContextSupplier,
WriteHandleFactory writeHandleFactory) {
ExplicitWriteHandleFactory writeHandleFactory) {
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory);
}

Expand All @@ -64,8 +56,8 @@ protected List<WriteStatus> computeNext() {
null;
try {
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
bufferedIteratorExecutor =
new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr), Option.of(getInsertHandler()), getTransformFunction(schema, hoodieConfig));
bufferedIteratorExecutor = new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr),
Option.of(getExplicitInsertHandler()), getTransformFunction(schema, hoodieConfig));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;
Expand All @@ -77,4 +69,10 @@ protected List<WriteStatus> computeNext() {
}
}
}

@SuppressWarnings("rawtypes")
private ExplicitWriteHandler getExplicitInsertHandler() {
HoodieWriteHandle handle = ((ExplicitWriteHandleFactory) writeHandleFactory).getWriteHandle();
return new ExplicitWriteHandler(handle);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
*/
public class ExplicitWriteHandleFactory<T extends HoodieRecordPayload, I, K, O>
extends WriteHandleFactory<T, I, K, O> {
private HoodieWriteHandle<T, I, K, O> writeHandle;
private final HoodieWriteHandle<T, I, K, O> writeHandle;

public ExplicitWriteHandleFactory(HoodieWriteHandle<T, I, K, O> writeHandle) {
this.writeHandle = writeHandle;
Expand All @@ -41,4 +41,8 @@ public HoodieWriteHandle<T, I, K, O> create(
String fileIdPrefix, TaskContextSupplier taskContextSupplier) {
return writeHandle;
}

public HoodieWriteHandle<T, I, K, O> getWriteHandle() {
return writeHandle;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,11 @@ protected void createMarkerFile(String partitionPath, String dataFileName) {
writeMarkers.createIfNotExists(partitionPath, dataFileName, getIOType());
}

@Override
public boolean canWrite(HoodieRecord record) {
return true;
}

@Override
protected boolean needsUpdateLocation() {
return false;
Expand Down