You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by "yunqingmoswu (via GitHub)" <gi...@apache.org> on 2023/03/15 07:29:48 UTC

[GitHub] [inlong] yunqingmoswu commented on a diff in pull request #7580: [INLONG-7249][Sort] JDBC accurate dirty data archive and metric calculation

yunqingmoswu commented on code in PR #7580:
URL: https://github.com/apache/inlong/pull/7580#discussion_r1136622431


##########
inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/internal/JdbcBatchingOutputFormat.java:
##########
@@ -184,7 +191,7 @@ public void open(int taskNumber, int numTasks) throws IOException {
                                     if (!closed) {
                                         try {
                                             flush();
-                                            if (sinkMetricData != null) {
+                                            if (sinkMetricData != null && dirtySink == null) {

Review Comment:
   Why metric data report dependent on 'dirtySink == null'?



##########
inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/internal/JdbcBatchingOutputFormat.java:
##########
@@ -365,6 +381,41 @@ public synchronized void flush() throws IOException {
         }
     }
 
+    private JdbcExec enhanceExecutor(JdbcExec exec) throws NoSuchFieldException, IllegalAccessException {
+        if (dirtySink == null) {
+            return null;
+        }
+        final DirtySinkHelper dirtySinkHelper = new DirtySinkHelper<>(dirtyOptions, dirtySink);
+        // enhance the actual executor to tablemetricstatementexecutor
+        Field ExecutorType;
+        if (exec instanceof TableBufferReducedStatementExecutor) {
+            ExecutorType = TableBufferReducedStatementExecutor.class.getDeclaredField("upsertExecutor");
+        } else if (exec instanceof TableBufferedStatementExecutor) {
+            ExecutorType = TableBufferedStatementExecutor.class.getDeclaredField("statementExecutor");
+        } else {
+            throw new RuntimeException("table enhance failed, can't enhance " + exec.getClass());
+        }
+        ExecutorType.setAccessible(true);
+        TableSimpleStatementExecutor executor = (TableSimpleStatementExecutor) ExecutorType.get(exec);
+        Field statementFactory = TableSimpleStatementExecutor.class.getDeclaredField("stmtFactory");
+        Field rowConverter = TableSimpleStatementExecutor.class.getDeclaredField("converter");
+        statementFactory.setAccessible(true);
+        rowConverter.setAccessible(true);
+        final StatementFactory stmtFactory = (StatementFactory) statementFactory.get(executor);
+        final JdbcRowConverter converter = (JdbcRowConverter) rowConverter.get(executor);
+        TableMetricStatementExecutor newExecutor =
+                new TableMetricStatementExecutor(stmtFactory, converter, dirtySinkHelper, sinkMetricData);
+        if (exec instanceof TableBufferedStatementExecutor) {
+            Field transform = TableBufferedStatementExecutor.class.getDeclaredField("valueTransform");
+            transform.setAccessible(true);
+            Function<RowData, RowData> valueTransform = (Function<RowData, RowData>) transform.get(exec);
+            newExecutor.setValueTransform(valueTransform);
+            return (JdbcExec) newExecutor;
+        }
+        ExecutorType.set(exec, newExecutor);

Review Comment:
   Please add some description for it



##########
inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/internal/JdbcMultiBatchingOutputFormat.java:
##########
@@ -302,6 +323,41 @@ private JdbcExec getOrCreateStatementExecutor(
         return jdbcExec;
     }
 
+    private JdbcExec enhanceExecutor(JdbcExec exec) throws NoSuchFieldException, IllegalAccessException {
+        if (dirtySinkHelper.getDirtySink() == null) {
+            return null;
+        }
+        // enhance the actual executor to tablemetricstatementexecutor
+        Field subExecutor;
+        if (exec instanceof TableBufferReducedStatementExecutor) {
+            subExecutor = TableBufferReducedStatementExecutor.class.getDeclaredField("upsertExecutor");
+        } else if (exec instanceof TableBufferedStatementExecutor) {
+            subExecutor = TableBufferedStatementExecutor.class.getDeclaredField("statementExecutor");
+        } else {
+            throw new RuntimeException("table enhance failed, can't enhance " + exec.getClass());
+        }
+        subExecutor.setAccessible(true);
+        TableSimpleStatementExecutor executor = (TableSimpleStatementExecutor) subExecutor.get(exec);
+        Field statementFactory = TableSimpleStatementExecutor.class.getDeclaredField("stmtFactory");
+        Field rowConverter = TableSimpleStatementExecutor.class.getDeclaredField("converter");
+        statementFactory.setAccessible(true);
+        rowConverter.setAccessible(true);
+        final StatementFactory stmtFactory = (StatementFactory) statementFactory.get(executor);
+        final JdbcRowConverter converter = (JdbcRowConverter) rowConverter.get(executor);
+        TableMetricStatementExecutor newExecutor =
+                new TableMetricStatementExecutor(stmtFactory, converter, dirtySinkHelper, sinkMetricData);
+        newExecutor.setMultipleSink(true);
+        if (exec instanceof TableBufferedStatementExecutor) {
+            subExecutor = TableBufferedStatementExecutor.class.getDeclaredField("valueTransform");
+            subExecutor.setAccessible(true);
+            Function<RowData, RowData> valueTransform = (Function<RowData, RowData>) subExecutor.get(exec);
+            newExecutor.setValueTransform(valueTransform);
+            return (JdbcExec) newExecutor;
+        }
+        subExecutor.set(exec, newExecutor);

Review Comment:
   Please add description for it



##########
inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/internal/JdbcMultiBatchingOutputFormat.java:
##########
@@ -332,29 +388,35 @@ private void checkFlushException() {
     @Override
     public final synchronized void writeRecord(In row) throws IOException {
         checkFlushException();
-        JsonDynamicSchemaFormat jsonDynamicSchemaFormat =
-                (JsonDynamicSchemaFormat) DynamicSchemaFormatFactory.getFormat(sinkMultipleFormat);
+        jsonDynamicSchemaFormat = (JsonDynamicSchemaFormat) DynamicSchemaFormatFactory.getFormat(sinkMultipleFormat);
+
         if (row instanceof RowData) {
             RowData rowData = (RowData) row;
             JsonNode rootNode = jsonDynamicSchemaFormat.deserialize(rowData.getBinary(0));
-            String tableIdentifier = null;
+            String tableIdentifier;
+            String database;
+            String table;
+            String schema = null;
             try {
                 if (StringUtils.isBlank(schemaPattern)) {
-                    tableIdentifier = StringUtils.join(
-                            jsonDynamicSchemaFormat.parse(rootNode, databasePattern), ".",
-                            jsonDynamicSchemaFormat.parse(rootNode, tablePattern));
+                    database = jsonDynamicSchemaFormat.parse(rootNode, databasePattern);
+                    table = jsonDynamicSchemaFormat.parse(rootNode, tablePattern);
+                    tableIdentifier = StringUtils.join(database, ".", table);
                 } else {
+                    database = jsonDynamicSchemaFormat.parse(rootNode, databasePattern);
+                    table = jsonDynamicSchemaFormat.parse(rootNode, tablePattern);
+                    schema = jsonDynamicSchemaFormat.parse(rootNode, schemaPattern);
                     tableIdentifier = StringUtils.join(
-                            jsonDynamicSchemaFormat.parse(rootNode, databasePattern), ".",

Review Comment:
   What is the better for it ?



##########
inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/internal/JdbcBatchingOutputFormat.java:
##########
@@ -365,6 +381,41 @@ public synchronized void flush() throws IOException {
         }
     }
 
+    private JdbcExec enhanceExecutor(JdbcExec exec) throws NoSuchFieldException, IllegalAccessException {
+        if (dirtySink == null) {
+            return null;
+        }
+        final DirtySinkHelper dirtySinkHelper = new DirtySinkHelper<>(dirtyOptions, dirtySink);
+        // enhance the actual executor to tablemetricstatementexecutor
+        Field ExecutorType;

Review Comment:
   ExecutorType -> executorType



-- 
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: commits-unsubscribe@inlong.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org