codope commented on code in PR #7982:
URL: https://github.com/apache/hudi/pull/7982#discussion_r1109675049


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BaseQuarantineTableWriter.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+
+public abstract class BaseQuarantineTableWriter<T extends QuarantineEvent> {

Review Comment:
   Should we rename to `BaseHoodieAuditTableWriter`, `HoodieAuditTableConfigs`, 
`AuditEvent`, `AuditErrorEvent` and `AuditUtils`? Basically, I am suggesting to 
evolve this into a more general audit framework. Error, is just one type of 
event. Others could be to keep record of updates/deletions in this audit table.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/QuarantineEvent.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.types.DataType;
+
+public abstract class QuarantineEvent<T> {
+
+  QuarantineReason reason;
+  T payload;
+
+  public abstract DataType getPayloadType();

Review Comment:
   this method is not used. remove it?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -746,6 +786,29 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> 
writeToSink(JavaRDD<HoodieRec
     return Pair.of(scheduledCompactionInstant, writeStatusRDD);
   }
 
+  protected JavaRDD<QuarantineJsonEvent> 
getErrorEventsForWriteStatus(JavaRDD<WriteStatus> writeStatusRDD) {
+    HoodieWriteConfig config = writeClient.getConfig();
+
+    return writeStatusRDD
+        .filter(WriteStatus::hasErrors)
+        .flatMap(x -> {
+          Schema schema = Schema.parse(config.getSchema());
+          Properties props = config.getPayloadConfig().getProps();
+          return x.getFailedRecords().stream()
+              .map(z -> {
+                HoodieRecordPayload hoodieRecordPayload = 
(HoodieRecordPayload)z.getData();
+                String recordStr;
+                try {
+                  recordStr = (String) 
hoodieRecordPayload.getInsertValue(schema,
+                      props).map(value -> value.toString()).get();
+                } catch (IOException e) {
+                  recordStr = null;

Review Comment:
   Should it be a reasonable message to distinguish IOException from Hudi?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/QuarantineEvent.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.types.DataType;
+
+public abstract class QuarantineEvent<T> {
+
+  QuarantineReason reason;
+  T payload;
+
+  public abstract DataType getPayloadType();
+
+  public abstract String getPayload();
+
+  public QuarantineReason getReason() {
+    return reason;
+  }
+
+  public abstract GenericRecord getAvroPayload();
+
+  public enum QuarantineReason {
+    JSON_AVRO_DESERIALIZATION_FAILURE,

Review Comment:
   Should we have error code for these failure reason? If you agree, it can be 
taken up as a follow-up, but what do you think of the suggestion?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java:
##########
@@ -117,11 +174,23 @@ public InputBatch<Dataset<Row>> 
fetchNewDataInRowFormat(Option<String> lastCkptS
       case JSON: {
         InputBatch<JavaRDD<String>> r = ((Source<JavaRDD<String>>) 
source).fetchNext(lastCkptStr, sourceLimit);
         Schema sourceSchema = r.getSchemaProvider().getSourceSchema();
-        StructType dataType = 
AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema);
-        return new InputBatch<>(
-            Option.ofNullable(
-                r.getBatch().map(rdd -> 
source.getSparkSession().read().schema(dataType).json(rdd)).orElse(null)),
-            r.getCheckpointForNextBatch(), r.getSchemaProvider());
+        if (quarantineTableWriterInterface.isPresent()) {
+          StructType dataType = 
AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema)
+              .add(new StructField(QUARANTINE_TABLE_CURRUPT_RECORD_COL_NAME, 
DataTypes.StringType, true, Metadata.empty()));
+          Option<Dataset<Row>> dataset = r.getBatch().map(rdd -> 
source.getSparkSession().read()
+              .option("mode", 
"PERMISSIVE").option("columnNameOfCorruptRecord", 
QUARANTINE_TABLE_CURRUPT_RECORD_COL_NAME).schema(dataType)

Review Comment:
   mode is PERMISSIVE by default. No need to set explicitly unless the default 
has changed in later Spark versions. Can you check?



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -236,6 +239,8 @@ public class DeltaSync implements Serializable, Closeable {
    */
   private transient SparkRDDWriteClient writeClient;
 
+  private Option<BaseQuarantineTableWriter> quarantineTableWriterInterfaceImpl 
= Option.empty();
+

Review Comment:
   +1



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java:
##########
@@ -710,6 +738,18 @@ private Pair<Option<String>, JavaRDD<WriteStatus>> 
writeToSink(JavaRDD<HoodieRec
             + totalErrorRecords + "/" + totalRecords);
       }
       String commitActionType = CommitUtils.getCommitActionType(cfg.operation, 
HoodieTableType.valueOf(cfg.tableType));
+      if (quarantineTableWriterInterfaceImpl.isPresent()) {
+        String qurantineTableStartInstant = 
quarantineTableWriterInterfaceImpl.get().startCommit();
+        
quarantineTableWriterInterfaceImpl.get().addErrorEvents(getErrorEventsForWriteStatus(writeStatusRDD));
+        Option<String> commitedInstantTime = 
getLatestInstantWithValidCheckpointInfo(commitTimelineOpt);
+        boolean quarantineTableSuccess = 
quarantineTableWriterInterfaceImpl.get().upsertAndCommit(qurantineTableStartInstant,
 instantTime, commitedInstantTime);
+        if (!quarantineTableSuccess) {
+          LOG.info("Qurantine Table Commit " + qurantineTableStartInstant + " 
failed!");
+          LOG.info("Commit " + instantTime + " failed!");
+          writeClient.rollback(instantTime);

Review Comment:
   that's a good suggestion. keep rollback as the default behavior



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/QuarantineEvent.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.utilities.deltastreamer;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.types.DataType;
+
+public abstract class QuarantineEvent<T> {
+
+  QuarantineReason reason;
+  T payload;
+
+  public abstract DataType getPayloadType();
+
+  public abstract String getPayload();
+
+  public QuarantineReason getReason() {
+    return reason;
+  }
+
+  public abstract GenericRecord getAvroPayload();

Review Comment:
   remove this as well? i think the payload type is implementation detail and 
should not be part of abstraction.



-- 
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]

Reply via email to