vinothchandar commented on code in PR #13236:
URL: https://github.com/apache/hudi/pull/13236#discussion_r2078184896
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -479,7 +479,14 @@ public class HoodieWriteConfig extends HoodieConfig {
public static final ConfigProperty<String> AUTO_COMMIT_ENABLE =
ConfigProperty
.key("hoodie.auto.commit")
- .defaultValue("true")
+ .defaultValue("false")
+ .markAdvanced()
+ .withDocumentation("Controls whether a write operation should auto
commit. This can be turned off to perform inspection"
+ + " of the uncommitted write before deciding to commit.");
+
+ public static final ConfigProperty<String> INTERNAL_AUTO_COMMIT_ENABLE =
ConfigProperty
+ .key("hoodie.internal.auto.commit")
Review Comment:
we cannot add anything for "internal" use here inside writeConfig.. lets
pass booleans around between functions.. and extend HoodieWriteConfig cleans
with an `HoodieInternalWriteConfig`
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -1093,6 +1117,139 @@ public void buildMetadataPartitions(HoodieEngineContext
engineContext, List<Hood
initializeFromFilesystem(instantTime, partitionTypes, Option.empty());
}
+ public void startCommit(String instantTime) {
Review Comment:
this file is getting way too monolithic. organize better?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -479,7 +479,14 @@ public class HoodieWriteConfig extends HoodieConfig {
public static final ConfigProperty<String> AUTO_COMMIT_ENABLE =
ConfigProperty
.key("hoodie.auto.commit")
- .defaultValue("true")
+ .defaultValue("false")
Review Comment:
can we remove this altogether.. at the very least deprecate
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java:
##########
@@ -83,6 +96,11 @@ public abstract class HoodieWriteHandle<T, I, K, O> extends
HoodieIOHandle<T, I,
protected final boolean schemaOnReadEnabled;
private boolean closed = false;
+ protected List<HoodieRecord> recordList = new ArrayList<>();
Review Comment:
are you storing all records in full here.. if so, it will OOM?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/common/WriteStatusHandlerCallback.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.callback.common;
+
+import org.apache.hudi.client.LeanWriteStatus;
+
+import java.util.List;
+
+/**
+ * WriteStatus handler callback to assist caller to process errors if any.
Caller can dictate if we wanted to proceed with the commit or not by means of
the return
+ * value of the call back (processWriteStatuses). We noticed that sometimes
callers invoke the dag just to process it there are any errors before
proceeding with the commit.
+ * With this callback, we are avoiding additional dag triggers from the
callers side.
Review Comment:
this is a good goal. but i would love to understand how exactly the triggers
are avoided
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java:
##########
@@ -262,6 +282,18 @@ protected void finalizeWrite(HoodieTable table, String
instantTime, List<HoodieW
}
}
+ class GetMetadataWriterFunc implements Functions.Function2<String,
HoodieTableMetaClient, Option<HoodieTableMetadataWriter>> {
Review Comment:
+1
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -860,6 +867,14 @@ public class HoodieWriteConfig extends HoodieConfig {
.sinceVersion("1.0.0")
.withDocumentation("Whether to enable incremental table service. So far
Clustering and Compaction support incremental processing.");
+ public static final ConfigProperty<Boolean>
STREAMING_WRITES_TO_METADATA_TABLE = ConfigProperty
+ .key("hoodie.write.streaming.writes.to.metadata")
+ .defaultValue(false)
Review Comment:
this should be `true` with all tests passing for landing this PR.
and also prefer to shed extra code vs maintaining multiple paths.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -115,11 +126,17 @@ public abstract class BaseHoodieTableServiceClient<I, T,
O> extends BaseHoodieCl
protected transient AsyncArchiveService asyncArchiveService;
protected Set<String> pendingInflightAndRequestedInstants;
+ protected Functions.Function2<String, HoodieTableMetaClient,
Option<HoodieTableMetadataWriter>> getMetadataWriterFunc;
Review Comment:
can we accomplish more using just java lambdas and functional interfaces vs
proliferating the `Functions` abstractions we have. Is this a conscious decision
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java:
##########
@@ -75,13 +78,16 @@ public abstract class BaseHoodieClient implements
Serializable, AutoCloseable {
protected final TransactionManager txnManager;
private final TimeGenerator timeGenerator;
+
/**
* Timeline Server has the same lifetime as that of Client. Any operations
done on the same timeline service will be
* able to take advantage of the cached file-system view. New completed
actions will be synced automatically in an
* incremental fashion.
*/
private transient Option<EmbeddedTimelineService> timelineServer;
private final boolean shouldStopTimelineServer;
+ // Cached HoodieTableMetadataWriter for each action in data table. This will
be cleaned up when action is completed or when write client is closed.
+ protected Map<String, Option<HoodieTableMetadataWriter>> metadataWriterMap =
new ConcurrentHashMap<>();
Review Comment:
why is this needed now specifically for the dag change
--
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]