swapna267 commented on code in PR #15471:
URL: https://github.com/apache/iceberg/pull/15471#discussion_r2908702623


##########
flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java:
##########
@@ -399,6 +400,74 @@ public void testCreateDynamicIcebergSink() throws 
DatabaseAlreadyExistException
         .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), 
Row.of(3L, "CCC"));
   }
 
+  @TestTemplate
+  public void testVariantAvroDynamicIcebergSink() throws 
DatabaseAlreadyExistException {
+    Map<String, String> tableProps = createTableProps();
+    Map<String, String> dynamicTableProps = Maps.newHashMap(tableProps);
+    dynamicTableProps.put("use-dynamic-iceberg-sink", "true");
+    dynamicTableProps.put(
+        "dynamic-record-generator-impl", 
VariantAvroDynamicTableRecordGenerator.class.getName());
+
+    FlinkCatalogFactory factory = new FlinkCatalogFactory();
+    FlinkCatalog flinkCatalog =
+        (FlinkCatalog) factory.createCatalog(catalogName, tableProps, new 
Configuration());
+    flinkCatalog.createDatabase(
+        databaseName(), new CatalogDatabaseImpl(Maps.newHashMap(), null), 
true);
+
+    String avroSchema =
+        """
+      {
+        "type": "record",
+        "name": "TestSchema",
+        "fields": [
+          {
+            "name": "id",
+            "type": "long"
+          },
+          {
+            "name": "name",
+            "type": "string"
+          }
+        ]
+      }
+      """;
+
+    String avroSchemaId = "TestSchema:1";
+    // Create table with dynamic sink enabled
+    sql(
+        "CREATE TABLE %s (data VARIANT, `catalog-database` STRING, 
`catalog-table` STRING, avro_schema STRING, avro_schema_id STRING, branch 
STRING, `write-parallelism` INT) WITH %s",
+        TABLE_NAME + "_dynamic", toWithClause(dynamicTableProps));
+
+    // Insert data with database and table information
+    sql(
+        "INSERT INTO %s VALUES "
+            + "(PARSE_JSON('{\"id\": 1, \"name\": \"AAA\"}'), '%s', '%s', 
'%s', '%s', 'main', 1), "
+            + "(PARSE_JSON('{\"id\": 2, \"name\": \"BBB\"}'), '%s', '%s', 
'%s', '%s', 'main', 1)",
+        TABLE_NAME + "_dynamic",
+        databaseName(),
+        tableName(),
+        avroSchema,
+        avroSchemaId,
+        databaseName(),
+        tableName(),
+        avroSchema,
+        avroSchemaId);
+
+    // Verify the table and data exists
+    ObjectPath objectPath = new ObjectPath(databaseName(), tableName());
+    assertThat(flinkCatalog.tableExists(objectPath)).isTrue();
+    Table table =
+        flinkCatalog
+            .getCatalogLoader()
+            .loadCatalog()
+            .loadTable(TableIdentifier.of(databaseName(), tableName()));
+
+    tableProps.put("catalog-database", databaseName());
+    sql("CREATE TABLE %s (id BIGINT, name STRING) WITH %s", tableName(), 
toWithClause(tableProps));
+    assertThat(sql("SELECT * FROM %s", tableName()))
+        .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"));

Review Comment:
   Table doesn't need to be loaded in line 459. Table is created to verify 
Table properties only. 
   
   I missed to that the following assert there, which makes it confusing. Added 
it.
   `assertThat(table.properties()).containsEntry("key1", "val1");` 



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to