yihua commented on a change in pull request #3233:
URL: https://github.com/apache/hudi/pull/3233#discussion_r681559342
##########
File path:
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -304,7 +304,7 @@ protected void postCommit(HoodieTable<T,
List<HoodieRecord<T>>, List<HoodieKey>,
Option<Map<String, String>> extraMetadata) {
try {
// Delete the marker directory for the instant.
- new MarkerFiles(createTable(config, hadoopConf), instantTime)
+ MarkerFilesFactory.get(config.getMarkersType(), createTable(config,
hadoopConf), instantTime)
Review comment:
I haven't tested that. Created a ticket here to track this:
https://issues.apache.org/jira/projects/HUDI/issues/HUDI-2271
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectMarkerFiles.java
##########
@@ -175,70 +165,30 @@ public static String stripMarkerSuffix(String path) {
return markerFiles;
}
- private String stripMarkerFolderPrefix(String fullMarkerPath) {
-
ValidationUtils.checkArgument(fullMarkerPath.contains(HoodieTableMetaClient.MARKER_EXTN));
- String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
- new Path(String.format("%s/%s/%s", basePath,
HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime))).toString();
- int begin = fullMarkerPath.indexOf(markerRootPath);
- ValidationUtils.checkArgument(begin >= 0,
- "Not in marker dir. Marker Path=" + fullMarkerPath + ", Expected
Marker Root=" + markerRootPath);
- return fullMarkerPath.substring(begin + markerRootPath.length() + 1);
- }
-
- /**
- * The marker path will be
<base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
- */
- public Path create(String partitionPath, String dataFileName, IOType type) {
+ @Override
+ protected Option<Path> create(String partitionPath, String dataFileName,
IOType type, boolean checkIfExists) {
+ HoodieTimer timer = new HoodieTimer().startTimer();
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
+ Path dirPath = markerPath.getParent();
try {
- LOG.info("Creating Marker Path=" + markerPath);
- fs.create(markerPath, false).close();
+ if (!fs.exists(dirPath)) {
Review comment:
I keep the actual logic of marker creation in the existing marker file
mechanism the same. In the old class, each marker creation (`create()`) calls
`getMarkerPath()`, which invokes `fs.exists(path)`. In the new class, I
changed the `getMarkerPath()` method to only construct the path instead of
doing any file system I/O operations, so that `getMarkerPath()` can be also
used in the timeline-server-based marker file class. In such a way, to keep
the logic the same for `DirectMarkerFiles`, the `fs.exists(dirPath)` is added
here.
We can optimize the flow in a follow-up PR.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+ public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+ private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ // Marker directory
+ private final String markerDirPath;
+ private final FileSystem fileSystem;
+ private final Registry metricsRegistry;
+ // A cached copy of all markers in memory
+ private final Set<String> allMarkers = new HashSet<>();
+ // A cached copy of marker entries in each marker file, stored in
StringBuilder
+ // for efficient appending
+ // Mapping: {markerFileIndex -> markers}
+ private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+ // A list of use status of underlying files storing markers by a thread.
+ // {@code true} means the file is in use by a {@code
BatchCreateMarkerRunnable}.
+ // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+ private final List<Boolean> threadUseStatus;
+ // A list of pending futures from async marker creation requests
+ private final List<MarkerCreationCompletableFuture> markerCreationFutures =
new ArrayList<>();
+ private final int parallelism;
+ private final AtomicBoolean lazyInitComplete;
+ private final Object markerCreationProcessingLock = new Object();
+ private transient HoodieEngineContext hoodieEngineContext;
+ // Last underlying file index used, for finding the next file index
+ // in a round-robin fashion
+ private int lastFileIndex = 0;
+
+ public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
FileSystem fileSystem,
+ Registry metricsRegistry, HoodieEngineContext
hoodieEngineContext, int parallelism) {
+ this.markerDirPath = markerDirPath;
+ this.fileSystem = fileSystem;
+ this.metricsRegistry = metricsRegistry;
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.parallelism = parallelism;
+ this.threadUseStatus =
+ Stream.generate(() ->
false).limit(markerBatchNumThreads).collect(Collectors.toList());
+ this.lazyInitComplete = new AtomicBoolean(false);
+ }
+
+ /**
+ * @return {@code true} if the marker directory exists in the system.
+ */
+ public boolean exists() {
+ try {
+ return fileSystem.exists(new Path(markerDirPath));
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ }
+
+ /**
+ * @return all markers in the marker directory.
+ */
+ public Set<String> getAllMarkers() {
+ maybeSyncOnFirstRequest();
+ return allMarkers;
+ }
+
+ /**
+ * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+ * creation request to the queue.
+ *
+ * @param future {@code MarkerCreationCompletableFuture} instance.
+ */
+ public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+ synchronized (markerCreationFutures) {
+ markerCreationFutures.add(future);
+ }
+ }
+
+ /**
+ * @return the next file index to use in a round-robin fashion,
+ * or -1 if no file is available.
+ */
+ public int getNextFileIndexToUse() {
+ int fileIndex = -1;
+ synchronized (threadUseStatus) {
+ for (int i = 0; i < threadUseStatus.size(); i++) {
+ int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+ if (!threadUseStatus.get(index)) {
+ fileIndex = index;
+ threadUseStatus.set(index, true);
+ break;
+ }
+ }
+ if (fileIndex >= 0) {
+ lastFileIndex = fileIndex;
+ }
+ }
+ return fileIndex;
+ }
+
+ /**
+ * Marks the file available to use again.
+ *
+ * @param fileIndex file index
+ */
+ public void markFileAvailable(int fileIndex) {
Review comment:
Fixed.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+ public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+ private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ // Marker directory
+ private final String markerDirPath;
+ private final FileSystem fileSystem;
+ private final Registry metricsRegistry;
+ // A cached copy of all markers in memory
+ private final Set<String> allMarkers = new HashSet<>();
+ // A cached copy of marker entries in each marker file, stored in
StringBuilder
+ // for efficient appending
+ // Mapping: {markerFileIndex -> markers}
+ private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+ // A list of use status of underlying files storing markers by a thread.
+ // {@code true} means the file is in use by a {@code
BatchCreateMarkerRunnable}.
+ // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+ private final List<Boolean> threadUseStatus;
+ // A list of pending futures from async marker creation requests
+ private final List<MarkerCreationCompletableFuture> markerCreationFutures =
new ArrayList<>();
+ private final int parallelism;
+ private final AtomicBoolean lazyInitComplete;
+ private final Object markerCreationProcessingLock = new Object();
+ private transient HoodieEngineContext hoodieEngineContext;
+ // Last underlying file index used, for finding the next file index
+ // in a round-robin fashion
+ private int lastFileIndex = 0;
+
+ public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
FileSystem fileSystem,
+ Registry metricsRegistry, HoodieEngineContext
hoodieEngineContext, int parallelism) {
+ this.markerDirPath = markerDirPath;
+ this.fileSystem = fileSystem;
+ this.metricsRegistry = metricsRegistry;
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.parallelism = parallelism;
+ this.threadUseStatus =
+ Stream.generate(() ->
false).limit(markerBatchNumThreads).collect(Collectors.toList());
+ this.lazyInitComplete = new AtomicBoolean(false);
+ }
+
+ /**
+ * @return {@code true} if the marker directory exists in the system.
+ */
+ public boolean exists() {
+ try {
+ return fileSystem.exists(new Path(markerDirPath));
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ }
+
+ /**
+ * @return all markers in the marker directory.
+ */
+ public Set<String> getAllMarkers() {
+ maybeSyncOnFirstRequest();
+ return allMarkers;
+ }
+
+ /**
+ * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+ * creation request to the queue.
+ *
+ * @param future {@code MarkerCreationCompletableFuture} instance.
+ */
+ public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+ synchronized (markerCreationFutures) {
+ markerCreationFutures.add(future);
+ }
+ }
+
+ /**
+ * @return the next file index to use in a round-robin fashion,
+ * or -1 if no file is available.
+ */
+ public int getNextFileIndexToUse() {
+ int fileIndex = -1;
+ synchronized (threadUseStatus) {
+ for (int i = 0; i < threadUseStatus.size(); i++) {
+ int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+ if (!threadUseStatus.get(index)) {
+ fileIndex = index;
+ threadUseStatus.set(index, true);
+ break;
+ }
+ }
+ if (fileIndex >= 0) {
+ lastFileIndex = fileIndex;
+ }
+ }
+ return fileIndex;
+ }
+
+ /**
+ * Marks the file available to use again.
+ *
+ * @param fileIndex file index
+ */
+ public void markFileAvailable(int fileIndex) {
+ synchronized (threadUseStatus) {
+ threadUseStatus.set(fileIndex, false);
+ }
+ }
+
+ /**
+ * @return futures of pending marker creation requests and removes them
from the list.
+ */
+ public List<MarkerCreationCompletableFuture>
fetchPendingMarkerCreationRequests() {
+ if (markerCreationFutures.isEmpty()) {
+ return new ArrayList<>();
+ }
+ maybeSyncOnFirstRequest();
+
+ List<MarkerCreationCompletableFuture> pendingFutures;
+ synchronized (markerCreationFutures) {
+ pendingFutures = new ArrayList<>(markerCreationFutures);
+ markerCreationFutures.clear();
+ }
+ return pendingFutures;
+ }
+
+ /**
+ * Processes pending marker creation requests.
+ *
+ * @param pendingMarkerCreationFutures futures of pending marker creation
requests
+ * @param fileIndex file index to use to write markers
+ */
+ public void processMarkerCreationRequests(
+ final List<MarkerCreationCompletableFuture>
pendingMarkerCreationFutures, int fileIndex) {
+ if (pendingMarkerCreationFutures.isEmpty()) {
+ markFileAvailable(fileIndex);
Review comment:
Yes, we should return here.
The fileIndex is per marker directory so it's to mark the file as available
again within each `MarkerDirState` instance.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {
+ public static final Logger LOG =
LogManager.getLogger(MarkerCreationDispatchingRunnable.class);
+
+ // Marker directory states, {markerDirPath -> MarkerDirState instance}
+ private final Map<String, MarkerDirState> markerDirStateMap;
+ private final ExecutorService executorService;
+
+ public MarkerCreationDispatchingRunnable(
+ Map<String, MarkerDirState> markerDirStateMap, int batchNumThreads) {
+ this.markerDirStateMap = markerDirStateMap;
+ this.executorService = Executors.newFixedThreadPool(batchNumThreads);
+ }
+
+ @Override
+ public void run() {
+ Map<String, MarkerDirRequestContext> requestContextMap = new HashMap<>();
+
Review comment:
Yes. I added the javadocs.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {
+ public static final Logger LOG =
LogManager.getLogger(MarkerCreationDispatchingRunnable.class);
+
+ // Marker directory states, {markerDirPath -> MarkerDirState instance}
+ private final Map<String, MarkerDirState> markerDirStateMap;
+ private final ExecutorService executorService;
+
+ public MarkerCreationDispatchingRunnable(
+ Map<String, MarkerDirState> markerDirStateMap, int batchNumThreads) {
+ this.markerDirStateMap = markerDirStateMap;
+ this.executorService = Executors.newFixedThreadPool(batchNumThreads);
Review comment:
Right, good catch. Fixed.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java
##########
@@ -228,7 +342,9 @@ public static void main(String[] args) throws Exception {
Configuration conf = FSUtils.prepareHadoopConf(new Configuration());
FileSystemViewManager viewManager = buildFileSystemViewManager(cfg, new
SerializableConfiguration(conf));
- TimelineService service = new TimelineService(cfg.serverPort, viewManager);
+ TimelineService service = new TimelineService(
+ new HoodieLocalEngineContext(FSUtils.prepareHadoopConf(new
Configuration())),
Review comment:
Previously there is a constructor like below where each configuration
instance is a new one. So I followed the same pattern.
```
public TimelineService(Config config) throws IOException {
this(config.serverPort, buildFileSystemViewManager(config,
new SerializableConfiguration(FSUtils.prepareHadoopConf(new
Configuration()))), new Configuration(),
config.numThreads, config.compress, config.async);
}
```
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/MarkerHandler.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.timeline.service.handlers;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.table.view.FileSystemViewManager;
+import org.apache.hudi.timeline.service.TimelineService;
+import
org.apache.hudi.timeline.service.handlers.marker.MarkerCreationCompletableFuture;
+import
org.apache.hudi.timeline.service.handlers.marker.MarkerCreationDispatchingRunnable;
+import org.apache.hudi.timeline.service.handlers.marker.MarkerDirState;
+
+import io.javalin.Context;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+/**
+ * REST Handler servicing marker requests.
+ *
+ * The marker creation requests are handled asynchronous, while other types of
requests
+ * are handled synchronous.
+ *
+ * Marker creation requests are batch processed periodically by a thread.
Each batch
+ * processing thread adds new markers to a marker file. Given that marker
file operation
+ * can take time, multiple concurrent threads can run at the same, while they
operate
+ * on different marker files storing mutually exclusive marker entries. At
any given
+ * time, a marker file is touched by at most one thread to guarantee
consistency.
+ * Below is an example of running batch processing threads.
+ *
+ * |-----| batch interval
+ * Thread 1 |-------------------------->| writing to MARKERS1
+ * Thread 2 |-------------------------->| writing to MARKERS2
+ * Thread 3 |-------------------------->| writing to MARKERS3
+ */
+public class MarkerHandler extends Handler {
+ private static final Logger LOG = LogManager.getLogger(MarkerHandler.class);
+
+ private final Registry metricsRegistry;
+ // a scheduled executor service to schedule dispatching of marker creation
requests
+ private final ScheduledExecutorService executorService;
+ // Parallelism for reading and deleting marker files
+ private final int parallelism;
+ // Marker directory states, {markerDirPath -> MarkerDirState instance}
+ private final Map<String, MarkerDirState> markerDirStateMap = new
HashMap<>();
+ // A long-running thread to dispatch marker creation requests to batch
processing threads
+ private final MarkerCreationDispatchingRunnable
markerCreationDispatchingRunnable;
+ private final AtomicBoolean firstMarkerCreationRequest;
+ private transient HoodieEngineContext hoodieEngineContext;
+ private ScheduledFuture<?> dispatchingScheduledFuture;
+
+ public MarkerHandler(Configuration conf, TimelineService.Config
timelineServiceConfig,
+ HoodieEngineContext hoodieEngineContext, FileSystem
fileSystem,
+ FileSystemViewManager viewManager, Registry
metricsRegistry) throws IOException {
+ super(conf, timelineServiceConfig, fileSystem, viewManager);
+ LOG.debug("MarkerHandler FileSystem: " + this.fileSystem.getScheme());
+ LOG.debug("MarkerHandler batching params: batchNumThreads=" +
timelineServiceConfig.markerBatchNumThreads
+ + " batchIntervalMs=" + timelineServiceConfig.markerBatchIntervalMs +
"ms");
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.metricsRegistry = metricsRegistry;
+ this.parallelism = timelineServiceConfig.markerParallelism;
+ this.executorService = Executors.newSingleThreadScheduledExecutor();
+ this.markerCreationDispatchingRunnable = new
MarkerCreationDispatchingRunnable(
+ markerDirStateMap, timelineServiceConfig.markerBatchNumThreads);
+ this.firstMarkerCreationRequest = new AtomicBoolean(true);
Review comment:
Fixed.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+ public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+ private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ // Marker directory
+ private final String markerDirPath;
+ private final FileSystem fileSystem;
+ private final Registry metricsRegistry;
+ // A cached copy of all markers in memory
+ private final Set<String> allMarkers = new HashSet<>();
+ // A cached copy of marker entries in each marker file, stored in
StringBuilder
+ // for efficient appending
+ // Mapping: {markerFileIndex -> markers}
+ private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+ // A list of use status of underlying files storing markers by a thread.
+ // {@code true} means the file is in use by a {@code
BatchCreateMarkerRunnable}.
+ // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+ private final List<Boolean> threadUseStatus;
+ // A list of pending futures from async marker creation requests
+ private final List<MarkerCreationCompletableFuture> markerCreationFutures =
new ArrayList<>();
+ private final int parallelism;
+ private final AtomicBoolean lazyInitComplete;
+ private final Object markerCreationProcessingLock = new Object();
+ private transient HoodieEngineContext hoodieEngineContext;
+ // Last underlying file index used, for finding the next file index
+ // in a round-robin fashion
+ private int lastFileIndex = 0;
+
+ public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
FileSystem fileSystem,
+ Registry metricsRegistry, HoodieEngineContext
hoodieEngineContext, int parallelism) {
+ this.markerDirPath = markerDirPath;
+ this.fileSystem = fileSystem;
+ this.metricsRegistry = metricsRegistry;
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.parallelism = parallelism;
+ this.threadUseStatus =
+ Stream.generate(() ->
false).limit(markerBatchNumThreads).collect(Collectors.toList());
+ this.lazyInitComplete = new AtomicBoolean(false);
+ }
+
+ /**
+ * @return {@code true} if the marker directory exists in the system.
+ */
+ public boolean exists() {
+ try {
+ return fileSystem.exists(new Path(markerDirPath));
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ }
+
+ /**
+ * @return all markers in the marker directory.
+ */
+ public Set<String> getAllMarkers() {
+ maybeSyncOnFirstRequest();
+ return allMarkers;
+ }
+
+ /**
+ * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+ * creation request to the queue.
+ *
+ * @param future {@code MarkerCreationCompletableFuture} instance.
+ */
+ public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+ synchronized (markerCreationFutures) {
+ markerCreationFutures.add(future);
+ }
+ }
+
+ /**
+ * @return the next file index to use in a round-robin fashion,
+ * or -1 if no file is available.
+ */
+ public int getNextFileIndexToUse() {
+ int fileIndex = -1;
+ synchronized (threadUseStatus) {
+ for (int i = 0; i < threadUseStatus.size(); i++) {
+ int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+ if (!threadUseStatus.get(index)) {
+ fileIndex = index;
+ threadUseStatus.set(index, true);
+ break;
+ }
+ }
+ if (fileIndex >= 0) {
+ lastFileIndex = fileIndex;
+ }
+ }
+ return fileIndex;
+ }
+
+ /**
+ * Marks the file available to use again.
+ *
+ * @param fileIndex file index
+ */
+ public void markFileAvailable(int fileIndex) {
+ synchronized (threadUseStatus) {
+ threadUseStatus.set(fileIndex, false);
+ }
+ }
+
+ /**
+ * @return futures of pending marker creation requests and removes them
from the list.
+ */
+ public List<MarkerCreationCompletableFuture>
fetchPendingMarkerCreationRequests() {
+ if (markerCreationFutures.isEmpty()) {
+ return new ArrayList<>();
+ }
+ maybeSyncOnFirstRequest();
+
+ List<MarkerCreationCompletableFuture> pendingFutures;
+ synchronized (markerCreationFutures) {
+ pendingFutures = new ArrayList<>(markerCreationFutures);
+ markerCreationFutures.clear();
+ }
+ return pendingFutures;
+ }
+
+ /**
+ * Processes pending marker creation requests.
+ *
+ * @param pendingMarkerCreationFutures futures of pending marker creation
requests
+ * @param fileIndex file index to use to write markers
+ */
+ public void processMarkerCreationRequests(
+ final List<MarkerCreationCompletableFuture>
pendingMarkerCreationFutures, int fileIndex) {
+ if (pendingMarkerCreationFutures.isEmpty()) {
+ markFileAvailable(fileIndex);
+ }
+
+ LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" +
markerDirPath
+ + " numRequests=" + pendingMarkerCreationFutures.size() + "
fileIndex=" + fileIndex);
+
+ synchronized (markerCreationProcessingLock) {
+ for (MarkerCreationCompletableFuture future :
pendingMarkerCreationFutures) {
+ String markerName = future.getMarkerName();
+ boolean exists = allMarkers.contains(markerName);
+ if (!exists) {
+ allMarkers.add(markerName);
+ StringBuilder stringBuilder =
fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384));
+ stringBuilder.append(markerName);
+ stringBuilder.append('\n');
+ }
+ future.setResult(!exists);
+ }
+ }
+ flushMarkersToFile(fileIndex);
+ markFileAvailable(fileIndex);
+
+ for (MarkerCreationCompletableFuture future :
pendingMarkerCreationFutures) {
+ try {
+ future.complete(jsonifyResult(
+ future.getContext(), future.getResult(), metricsRegistry,
OBJECT_MAPPER, LOG));
+ } catch (JsonProcessingException e) {
+ throw new HoodieException("Failed to JSON encode the value", e);
+ }
+ }
+ }
+
+ /**
+ * Deletes markers in the directory.
+ *
+ * @return {@code true} if successful; {@code false} otherwise.
+ */
+ public boolean deleteAllMarkers() {
+ Path dirPath = new Path(markerDirPath);
+ boolean result = false;
+ try {
+ if (fileSystem.exists(dirPath)) {
+ FileStatus[] fileStatuses = fileSystem.listStatus(dirPath);
+ List<String> markerDirSubPaths = Arrays.stream(fileStatuses)
+ .map(fileStatus -> fileStatus.getPath().toString())
+ .collect(Collectors.toList());
+
+ if (markerDirSubPaths.size() > 0) {
+ SerializableConfiguration conf = new
SerializableConfiguration(fileSystem.getConf());
+ int actualParallelism = Math.min(markerDirSubPaths.size(),
parallelism);
+ hoodieEngineContext.foreach(markerDirSubPaths, subPathStr -> {
+ Path subPath = new Path(subPathStr);
+ FileSystem fileSystem = subPath.getFileSystem(conf.get());
+ fileSystem.delete(subPath, true);
+ }, actualParallelism);
+ }
+
+ result = fileSystem.delete(dirPath, true);
+ LOG.info("Removing marker directory at " + dirPath);
+ }
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ allMarkers.clear();
+ fileMarkersMap.clear();
+ return result;
+ }
+
+ /**
+ * Syncs the markers from the underlying files for the first request.
+ */
+ private void maybeSyncOnFirstRequest() {
+ if (!lazyInitComplete.getAndSet(true)) {
Review comment:
That's a good call. I didn't catch that when I switched to use
AtomicBoolean. Now I revert it back to what it was.
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/DirectMarkerFiles.java
##########
@@ -175,70 +165,30 @@ public static String stripMarkerSuffix(String path) {
return markerFiles;
}
- private String stripMarkerFolderPrefix(String fullMarkerPath) {
-
ValidationUtils.checkArgument(fullMarkerPath.contains(HoodieTableMetaClient.MARKER_EXTN));
- String markerRootPath = Path.getPathWithoutSchemeAndAuthority(
- new Path(String.format("%s/%s/%s", basePath,
HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime))).toString();
- int begin = fullMarkerPath.indexOf(markerRootPath);
- ValidationUtils.checkArgument(begin >= 0,
- "Not in marker dir. Marker Path=" + fullMarkerPath + ", Expected
Marker Root=" + markerRootPath);
- return fullMarkerPath.substring(begin + markerRootPath.length() + 1);
- }
-
- /**
- * The marker path will be
<base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename.marker.writeIOType.
- */
- public Path create(String partitionPath, String dataFileName, IOType type) {
+ @Override
+ protected Option<Path> create(String partitionPath, String dataFileName,
IOType type, boolean checkIfExists) {
+ HoodieTimer timer = new HoodieTimer().startTimer();
Path markerPath = getMarkerPath(partitionPath, dataFileName, type);
+ Path dirPath = markerPath.getParent();
try {
- LOG.info("Creating Marker Path=" + markerPath);
- fs.create(markerPath, false).close();
+ if (!fs.exists(dirPath)) {
Review comment:
Tracking this action item here:
https://issues.apache.org/jira/projects/HUDI/issues/HUDI-2271
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+ public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+ private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ // Marker directory
+ private final String markerDirPath;
+ private final FileSystem fileSystem;
+ private final Registry metricsRegistry;
+ // A cached copy of all markers in memory
+ private final Set<String> allMarkers = new HashSet<>();
+ // A cached copy of marker entries in each marker file, stored in
StringBuilder
+ // for efficient appending
+ // Mapping: {markerFileIndex -> markers}
+ private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+ // A list of use status of underlying files storing markers by a thread.
+ // {@code true} means the file is in use by a {@code
BatchCreateMarkerRunnable}.
+ // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+ private final List<Boolean> threadUseStatus;
+ // A list of pending futures from async marker creation requests
+ private final List<MarkerCreationCompletableFuture> markerCreationFutures =
new ArrayList<>();
+ private final int parallelism;
+ private final AtomicBoolean lazyInitComplete;
+ private final Object markerCreationProcessingLock = new Object();
+ private transient HoodieEngineContext hoodieEngineContext;
+ // Last underlying file index used, for finding the next file index
+ // in a round-robin fashion
+ private int lastFileIndex = 0;
+
+ public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
FileSystem fileSystem,
+ Registry metricsRegistry, HoodieEngineContext
hoodieEngineContext, int parallelism) {
+ this.markerDirPath = markerDirPath;
+ this.fileSystem = fileSystem;
+ this.metricsRegistry = metricsRegistry;
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.parallelism = parallelism;
+ this.threadUseStatus =
+ Stream.generate(() ->
false).limit(markerBatchNumThreads).collect(Collectors.toList());
+ this.lazyInitComplete = new AtomicBoolean(false);
+ }
+
+ /**
+ * @return {@code true} if the marker directory exists in the system.
+ */
+ public boolean exists() {
+ try {
+ return fileSystem.exists(new Path(markerDirPath));
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ }
+
+ /**
+ * @return all markers in the marker directory.
+ */
+ public Set<String> getAllMarkers() {
+ maybeSyncOnFirstRequest();
+ return allMarkers;
+ }
+
+ /**
+ * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+ * creation request to the queue.
+ *
+ * @param future {@code MarkerCreationCompletableFuture} instance.
+ */
+ public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+ synchronized (markerCreationFutures) {
+ markerCreationFutures.add(future);
+ }
+ }
+
+ /**
+ * @return the next file index to use in a round-robin fashion,
+ * or -1 if no file is available.
+ */
+ public int getNextFileIndexToUse() {
+ int fileIndex = -1;
+ synchronized (threadUseStatus) {
+ for (int i = 0; i < threadUseStatus.size(); i++) {
+ int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
Review comment:
I made it more intuitive based on your suggestion.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationDispatchingRunnable.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * A runnable for scheduling batch processing of marker creation requests.
+ */
+public class MarkerCreationDispatchingRunnable implements Runnable {
Review comment:
I invoke shutdown for both executor services, one for the dispatching
and the other for the worker threads. Let me know if that looks good.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirRequestContext.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import java.util.List;
+
+/**
+ * Input of batch processing of marker creation requests for a single marker
directory.
+ */
+public class MarkerDirRequestContext {
+ // List of marker creation futures to process
+ private final List<MarkerCreationCompletableFuture> futures;
Review comment:
I added more info here for better abstraction. Let me know how that
looks.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerCreationBatchingRunnable.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.util.HoodieTimer;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Map;
+
+/**
+ * A runnable for batch processing marker creation requests.
+ */
+public class MarkerCreationBatchingRunnable implements Runnable {
+ private static final Logger LOG =
LogManager.getLogger(MarkerCreationBatchingRunnable.class);
+
+ private final Map<String, MarkerDirState> markerDirStateMap;
+ private final Map<String, MarkerDirRequestContext> requestContextMap;
+
+ public MarkerCreationBatchingRunnable(
+ Map<String, MarkerDirState> markerDirStateMap,
+ Map<String, MarkerDirRequestContext> requestContextMap) {
+ this.markerDirStateMap = markerDirStateMap;
+ this.requestContextMap = requestContextMap;
+ }
+
+ @Override
+ public void run() {
+ LOG.debug("Start processing create marker requests");
+ HoodieTimer timer = new HoodieTimer().startTimer();
+
+ for (String markerDir : requestContextMap.keySet()) {
+ MarkerDirState markerDirState = markerDirStateMap.get(markerDir);
+
+ if (markerDirState == null) {
+ LOG.error("MarkerDirState of " + markerDir + " does not exist!");
Review comment:
This should not really happen since `requestContextMap` is derived from
`markerDirStateMap`. I've refactored the code so that all necessary
information are stored in MarkerDirRequestContext and we don't need the check
anymore.
##########
File path:
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerDirState.java
##########
@@ -0,0 +1,383 @@
+/*
+ * 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.timeline.service.handlers.marker;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.util.FileIOUtils.closeQuietly;
+import static org.apache.hudi.timeline.service.RequestHandler.jsonifyResult;
+
+/**
+ * Stores the state of a marker directory.
+ *
+ * The operations inside this class is designed to be thread-safe.
+ */
+public class MarkerDirState implements Serializable {
+ public static final String MARKERS_FILENAME_PREFIX = "MARKERS";
+ private static final Logger LOG = LogManager.getLogger(MarkerDirState.class);
+ private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ // Marker directory
+ private final String markerDirPath;
+ private final FileSystem fileSystem;
+ private final Registry metricsRegistry;
+ // A cached copy of all markers in memory
+ private final Set<String> allMarkers = new HashSet<>();
+ // A cached copy of marker entries in each marker file, stored in
StringBuilder
+ // for efficient appending
+ // Mapping: {markerFileIndex -> markers}
+ private final Map<Integer, StringBuilder> fileMarkersMap = new HashMap<>();
+ // A list of use status of underlying files storing markers by a thread.
+ // {@code true} means the file is in use by a {@code
BatchCreateMarkerRunnable}.
+ // Index of the list is used for the filename, i.e., "1" -> "MARKERS1"
+ private final List<Boolean> threadUseStatus;
+ // A list of pending futures from async marker creation requests
+ private final List<MarkerCreationCompletableFuture> markerCreationFutures =
new ArrayList<>();
+ private final int parallelism;
+ private final AtomicBoolean lazyInitComplete;
+ private final Object markerCreationProcessingLock = new Object();
+ private transient HoodieEngineContext hoodieEngineContext;
+ // Last underlying file index used, for finding the next file index
+ // in a round-robin fashion
+ private int lastFileIndex = 0;
+
+ public MarkerDirState(String markerDirPath, int markerBatchNumThreads,
FileSystem fileSystem,
+ Registry metricsRegistry, HoodieEngineContext
hoodieEngineContext, int parallelism) {
+ this.markerDirPath = markerDirPath;
+ this.fileSystem = fileSystem;
+ this.metricsRegistry = metricsRegistry;
+ this.hoodieEngineContext = hoodieEngineContext;
+ this.parallelism = parallelism;
+ this.threadUseStatus =
+ Stream.generate(() ->
false).limit(markerBatchNumThreads).collect(Collectors.toList());
+ this.lazyInitComplete = new AtomicBoolean(false);
+ }
+
+ /**
+ * @return {@code true} if the marker directory exists in the system.
+ */
+ public boolean exists() {
+ try {
+ return fileSystem.exists(new Path(markerDirPath));
+ } catch (IOException ioe) {
+ throw new HoodieIOException(ioe.getMessage(), ioe);
+ }
+ }
+
+ /**
+ * @return all markers in the marker directory.
+ */
+ public Set<String> getAllMarkers() {
+ maybeSyncOnFirstRequest();
+ return allMarkers;
+ }
+
+ /**
+ * Adds a {@code MarkerCreationCompletableFuture} instance from a marker
+ * creation request to the queue.
+ *
+ * @param future {@code MarkerCreationCompletableFuture} instance.
+ */
+ public void addMarkerCreationFuture(MarkerCreationCompletableFuture future) {
+ synchronized (markerCreationFutures) {
+ markerCreationFutures.add(future);
+ }
+ }
+
+ /**
+ * @return the next file index to use in a round-robin fashion,
+ * or -1 if no file is available.
+ */
+ public int getNextFileIndexToUse() {
+ int fileIndex = -1;
+ synchronized (threadUseStatus) {
+ for (int i = 0; i < threadUseStatus.size(); i++) {
+ int index = (lastFileIndex + 1 + i) % threadUseStatus.size();
+ if (!threadUseStatus.get(index)) {
+ fileIndex = index;
+ threadUseStatus.set(index, true);
+ break;
+ }
+ }
+ if (fileIndex >= 0) {
+ lastFileIndex = fileIndex;
+ }
+ }
+ return fileIndex;
+ }
+
+ /**
+ * Marks the file available to use again.
+ *
+ * @param fileIndex file index
+ */
+ public void markFileAvailable(int fileIndex) {
+ synchronized (threadUseStatus) {
+ threadUseStatus.set(fileIndex, false);
+ }
+ }
+
+ /**
+ * @return futures of pending marker creation requests and removes them
from the list.
+ */
+ public List<MarkerCreationCompletableFuture>
fetchPendingMarkerCreationRequests() {
+ if (markerCreationFutures.isEmpty()) {
+ return new ArrayList<>();
+ }
+ maybeSyncOnFirstRequest();
+
+ List<MarkerCreationCompletableFuture> pendingFutures;
+ synchronized (markerCreationFutures) {
+ pendingFutures = new ArrayList<>(markerCreationFutures);
+ markerCreationFutures.clear();
+ }
+ return pendingFutures;
+ }
+
+ /**
+ * Processes pending marker creation requests.
+ *
+ * @param pendingMarkerCreationFutures futures of pending marker creation
requests
+ * @param fileIndex file index to use to write markers
+ */
+ public void processMarkerCreationRequests(
+ final List<MarkerCreationCompletableFuture>
pendingMarkerCreationFutures, int fileIndex) {
+ if (pendingMarkerCreationFutures.isEmpty()) {
+ markFileAvailable(fileIndex);
+ }
+
+ LOG.debug("timeMs=" + System.currentTimeMillis() + " markerDirPath=" +
markerDirPath
+ + " numRequests=" + pendingMarkerCreationFutures.size() + "
fileIndex=" + fileIndex);
+
+ synchronized (markerCreationProcessingLock) {
+ for (MarkerCreationCompletableFuture future :
pendingMarkerCreationFutures) {
+ String markerName = future.getMarkerName();
+ boolean exists = allMarkers.contains(markerName);
+ if (!exists) {
+ allMarkers.add(markerName);
+ StringBuilder stringBuilder =
fileMarkersMap.computeIfAbsent(fileIndex, k -> new StringBuilder(16384));
+ stringBuilder.append(markerName);
+ stringBuilder.append('\n');
+ }
+ future.setResult(!exists);
+ }
+ }
+ flushMarkersToFile(fileIndex);
+ markFileAvailable(fileIndex);
Review comment:
We can, given the reason above.
--
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]