alexeykudinkin commented on code in PR #7238:
URL: https://github.com/apache/hudi/pull/7238#discussion_r1026975000
##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java:
##########
@@ -19,139 +19,55 @@
package org.apache.hudi.common.util.queue;
import org.apache.hudi.common.util.DefaultSizeEstimator;
-import org.apache.hudi.common.util.Functions;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.exception.HoodieException;
-
-import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
-import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
import java.util.function.Function;
/**
* Executor which orchestrates concurrent producers and consumers
communicating through 'BoundedInMemoryQueue'. This
* class takes as input the size limit, queue producer(s), consumer and
transformer and exposes API to orchestrate
* concurrent execution of these actors communicating through a central
bounded queue
*/
-public class BoundedInMemoryExecutor<I, O, E> extends HoodieExecutorBase<I, O,
E> {
+public class BoundedInMemoryExecutor<I, O, E> extends
BaseHoodieQueueBasedExecutor<I, O, E> {
private static final Logger LOG =
LogManager.getLogger(BoundedInMemoryExecutor.class);
- private final HoodieMessageQueue<I, O> queue;
public BoundedInMemoryExecutor(final long bufferLimitInBytes, final
Iterator<I> inputItr,
- IteratorBasedQueueConsumer<O, E> consumer,
Function<I, O> transformFunction, Runnable preExecuteRunnable) {
- this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr),
Option.of(consumer), transformFunction, preExecuteRunnable);
- }
-
- public BoundedInMemoryExecutor(final long bufferLimitInBytes,
HoodieProducer<I> producer,
- Option<IteratorBasedQueueConsumer<O, E>>
consumer, final Function<I, O> transformFunction) {
- this(bufferLimitInBytes, producer, consumer, transformFunction,
Functions.noop());
- }
-
- public BoundedInMemoryExecutor(final long bufferLimitInBytes,
HoodieProducer<I> producer,
- Option<IteratorBasedQueueConsumer<O, E>>
consumer, final Function<I, O> transformFunction, Runnable preExecuteRunnable) {
- this(bufferLimitInBytes, Collections.singletonList(producer), consumer,
transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable);
+ HoodieConsumer<O, E> consumer, Function<I, O>
transformFunction, Runnable preExecuteRunnable) {
+ this(bufferLimitInBytes, Collections.singletonList(new
IteratorBasedQueueProducer<>(inputItr)),
+ Option.of(consumer), transformFunction, new DefaultSizeEstimator<>(),
preExecuteRunnable);
}
public BoundedInMemoryExecutor(final long bufferLimitInBytes,
List<HoodieProducer<I>> producers,
- Option<IteratorBasedQueueConsumer<O, E>>
consumer, final Function<I, O> transformFunction,
+ Option<HoodieConsumer<O, E>> consumer, final
Function<I, O> transformFunction,
final SizeEstimator<O> sizeEstimator,
Runnable preExecuteRunnable) {
- super(producers, consumer, preExecuteRunnable);
- this.queue = new BoundedInMemoryQueueIterable<>(bufferLimitInBytes,
transformFunction, sizeEstimator);
- }
-
- /**
- * Start all producers at once.
- */
- @Override
- public CompletableFuture<Void> startProducers() {
Review Comment:
This method (w/ modifications) have been moved into base class to be shared
across impls
##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/BaseHoodieQueueBasedExecutor.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.common.util.queue;
+
+import org.apache.hudi.common.util.CustomizedThreadFactory;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.util.FutureUtils.allOf;
+import static org.apache.hudi.common.util.ValidationUtils.checkState;
+
+/**
+ * HoodieExecutorBase holds common elements producerExecutorService,
consumerExecutorService, producers and a single consumer.
+ * Also HoodieExecutorBase control the lifecycle of producerExecutorService
and consumerExecutorService.
+ */
+public abstract class BaseHoodieQueueBasedExecutor<I, O, E> implements
HoodieExecutor<I, O, E> {
+
+ private static final long TERMINATE_WAITING_TIME_SECS = 60L;
+
+ private final Logger logger = LogManager.getLogger(getClass());
+
+ // Executor service used for launching write thread.
+ private final ExecutorService producerExecutorService;
+ // Executor service used for launching read thread.
+ private final ExecutorService consumerExecutorService;
+ // Queue
+ protected final HoodieMessageQueue<I, O> queue;
+ // Producers
+ private final List<HoodieProducer<I>> producers;
+ // Consumer
+ protected final Option<HoodieConsumer<O, E>> consumer;
+
+ public BaseHoodieQueueBasedExecutor(List<HoodieProducer<I>> producers,
Review Comment:
Ideas underpinning this class were
- Abstract common logic across impls
- Make sure all concurrency is handled exclusively in the base class (impl
should just write sync code of how to produce/consume to/from the queue)
- Make sure all resource lifecycle handling happens in the base class
##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieIterableMessageQueue.java:
##########
@@ -1,29 +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.common.util.queue;
-
-import java.util.Iterator;
-
-/**
- * IteratorBasedHoodieMessageQueue implements HoodieMessageQueue with Iterable
- */
-public abstract class HoodieIterableMessageQueue<I, O> implements
HoodieMessageQueue<I, O>, Iterable<O> {
Review Comment:
This is not needed anymore
##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/IteratorBasedQueueConsumer.java:
##########
@@ -1,63 +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.common.util.queue;
-
-import java.util.Iterator;
-
-/**
- * Consume entries from queue and execute callback function.
- */
-public abstract class IteratorBasedQueueConsumer<I, O> implements
HoodieConsumer<I, O> {
Review Comment:
This is not needed anymore
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java:
##########
@@ -208,19 +205,33 @@ public BoundedMemoryRecords(
Schema logSchema,
Configuration hadoopConf,
org.apache.flink.configuration.Configuration flinkConf) {
+ HoodieUnMergedLogRecordScanner.Builder scannerBuilder =
HoodieUnMergedLogRecordScanner.newBuilder()
Review Comment:
This refactoring is necessary to avoid exposing internals of the executor
(queue), instead relying on the provided interfaces to configure it
##########
hudi-common/src/main/java/org/apache/hudi/common/util/queue/HoodieExecutorBase.java:
##########
@@ -1,143 +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.common.util.queue;
-
-import org.apache.hudi.common.util.CustomizedThreadFactory;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
-import org.apache.hudi.exception.HoodieException;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-/**
- * HoodieExecutorBase holds common elements producerExecutorService,
consumerExecutorService, producers and a single consumer.
- * Also HoodieExecutorBase control the lifecycle of producerExecutorService
and consumerExecutorService.
- */
-public abstract class HoodieExecutorBase<I, O, E> implements HoodieExecutor<I,
O, E> {
Review Comment:
This class has been replaced by `BaseHoodieQueueBasedExecutor`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]