From 5198b37626a46574ee4994659e9d19300e1477a2 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 12 Nov 2025 16:35:54 +0530 Subject: [PATCH 01/22] Extend query config with query.masked config with Type.PASSWORD for secure query masking --- .../connect/jdbc/JdbcSourceConnector.java | 4 +- .../source/JdbcSourceConnectorConfig.java | 40 +++ .../connect/jdbc/source/JdbcSourceTask.java | 2 +- .../jdbc/source/TableQuerierProcessor.java | 2 +- .../JdbcSourceConnectorValidation.java | 32 ++- .../source/JdbcSourceConnectorConfigTest.java | 266 ++++++++++++++++++ .../JdbcSourceConnectorValidationTest.java | 91 ++++++ 7 files changed, 432 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java b/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java index 2d85039c2..a460ad08f 100644 --- a/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java +++ b/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java @@ -117,7 +117,7 @@ public void start(Map properties) throws ConnectException { List excludeList = config.tableExcludeListRegexes(); Set excludeListSet = excludeList.isEmpty() ? null : new HashSet<>(excludeList); - String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); + String query = config.getQuery(); if (!query.isEmpty()) { if (whitelistSet != null || blacklistSet != null || includeListSet != null || excludeListSet != null) { @@ -170,7 +170,7 @@ public Config validate(Map connectorConfigs) { @Override public List> taskConfigs(int maxTasks) { log.info("Starting with the task Configuration method."); - String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); + String query = config.getQuery(); List> taskConfigs; if (!query.isEmpty()) { log.info("Custom query provided, generating task configuration for the query"); diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 2fa613205..4a5e105da 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -54,6 +54,7 @@ import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.errors.ConnectException; @@ -336,6 +337,19 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { public static final String QUERY_DEFAULT = ""; private static final String QUERY_DISPLAY = "Query"; + public static final String QUERY_MASKED_CONFIG = "query.masked"; + private static final String QUERY_MASKED_DOC = + "If specified, the query to perform to select new or updated rows. This is the same as " + + "'query' config but with Type.PASSWORD to mask the query value from being visible to users. " + + "Use this setting when your query contains sensitive information. If used, this connector " + + "will only copy data using this query -- whole-table copying will be disabled. Different " + + "query modes may still be used for incremental updates, but in order to properly construct " + + "the incremental query, it must be possible to append a WHERE clause to this query (i.e. no " + + "WHERE clauses may be used). If you use a WHERE clause, it must handle incremental queries " + + "itself. Note: Only one of 'query' or 'query.masked' should be set, not both."; + private static final String QUERY_MASKED_DISPLAY = "Query (Masked)"; + + public static final String TOPIC_PREFIX_CONFIG = "topic.prefix"; private static final String TOPIC_PREFIX_DOC = "Prefix to prepend to table names to generate the name of the Kafka topic to publish data " @@ -927,6 +941,16 @@ private static final void addModeOptions(ConfigDef config) { ++orderInGroup, Width.SHORT, QUERY_DISPLAY + ).define( + QUERY_MASKED_CONFIG, + Type.PASSWORD, + QUERY_DEFAULT, + Importance.MEDIUM, + QUERY_MASKED_DOC, + MODE_GROUP, + ++orderInGroup, + Width.SHORT, + QUERY_MASKED_DISPLAY ).define( QUOTE_SQL_IDENTIFIERS_CONFIG, Type.STRING, @@ -1432,6 +1456,22 @@ public List incrementingColMappingRegexes() { .collect(java.util.stream.Collectors.toList()); } + /** + * Get query string from either query.masked (Type.PASSWORD) or query (Type.STRING) config. + * Prioritizes query.masked if set, otherwise falls back to query config for backward compatibility. + * + * @return The query string from whichever config is set, or empty string if neither is set. + */ + public String getQuery() { + // First check if query.masked is set + org.apache.kafka.common.config.types.Password maskedQuery = getPassword(QUERY_MASKED_CONFIG); + if (maskedQuery != null && maskedQuery.value() != null && !maskedQuery.value().isEmpty()) { + return maskedQuery.value(); + } + + return getString(QUERY_CONFIG); + } + public boolean modeUsesTimestampColumn() { String mode = getString(MODE_CONFIG); return Arrays.asList(MODE_TIMESTAMP, MODE_TIMESTAMP_INCREMENTING).contains(mode); diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 7508e05ce..402289233 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -95,7 +95,7 @@ public void start(Map properties) { List tables = config.getList(JdbcSourceTaskConfig.TABLES_CONFIG); Boolean tablesFetched = config.getBoolean(JdbcSourceTaskConfig.TABLES_FETCHED); - String query = config.getString(JdbcSourceTaskConfig.QUERY_CONFIG); + String query = config.getQuery(); List tableType = config.getList(JdbcSourceConnectorConfig.TABLE_TYPE_CONFIG); if ((tables.isEmpty() && query.isEmpty())) { diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index f25b5c482..7eb5924f1 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -95,7 +95,7 @@ private boolean isReadyToProcess() { // If the call to get tables has not completed we will not do anything. // This is only valid in table mode. Boolean tablesFetched = config.getBoolean(JdbcSourceTaskConfig.TABLES_FETCHED); - String query = config.getString(JdbcSourceTaskConfig.QUERY_CONFIG); + String query = config.getQuery(); return !query.isEmpty() || tablesFetched; } diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index b5445287e..6f794097f 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -80,7 +80,8 @@ public Config validate() { } boolean validationResult = validateMultiConfigs() - && validateLegacyNewConfigCompatibility(); + && validateLegacyNewConfigCompatibility() + && validateQueryConfigs(); if (validationResult && isUsingNewConfigs()) { validationResult = validateTableInclusionConfigs() @@ -305,6 +306,35 @@ private boolean validateTableInclusionConfigs() { return true; } + /** + * Validate that only one of query or query.masked configs is set at a time. + * Both configs should not be set simultaneously to avoid ambiguity. + */ + private boolean validateQueryConfigs() { + String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); + org.apache.kafka.common.config.types.Password queryMasked = + config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + + boolean hasQuery = query != null && !query.isEmpty(); + boolean hasQueryMasked = queryMasked != null + && queryMasked.value() != null + && !queryMasked.value().isEmpty(); + + if (hasQuery && hasQueryMasked) { + String msg = "Both 'query' and 'query.masked' configs cannot be set at the same time. " + + "Please use only one of them. Use 'query.masked' (Type.PASSWORD) if you want to hide " + + "the query value from being visible, or use 'query' (Type.STRING) for regular usage."; + + addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); + + log.error("Validation failed: Both query and query.masked configs are set"); + return false; + } + + return true; + } + /** * Validate plugin-specific needs. This method can be overridden by specific * connector implementations to add their own validation logic. diff --git a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java index eea717b34..9e8d8c074 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java @@ -424,4 +424,270 @@ public void testWhitespaceInRegexPatternsIsHandledCorrectly() { assertTrue(includeList.contains("schema1\\.users")); assertTrue(includeList.contains("schema2\\.orders")); } + + @Test + public void testQueryMaskedValueIsMaskedInConfigValue() { + // Setup config with query.masked + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String sensitiveQuery = "SELECT * FROM sensitive_table WHERE secret_column = 'confidential'"; + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, sensitiveQuery); + + // Validate and get config values + Map validatedConfig = JdbcSourceConnectorConfig.CONFIG_DEF.validateAll(props); + ConfigValue queryMaskedValue = validatedConfig.get(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + + assertNotNull(queryMaskedValue); + + // The value should be masked (shown as [hidden] or Password object) + Object maskedValue = queryMaskedValue.value(); + assertNotNull(maskedValue); + + // When converted to string, Password objects show as "[hidden]" + String maskedString = maskedValue.toString(); + assertEquals("[hidden]", maskedString); + + // The actual query should NOT be visible in the string representation + assertFalse(maskedString.contains(sensitiveQuery)); + } + + @Test + public void testQueryMaskedValueCanBeRetrievedViaConfig() { + // Setup config with query.masked + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String expectedQuery = "SELECT * FROM users WHERE status = 'active'"; + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); + + // Create config + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // The actual value can be retrieved when needed + org.apache.kafka.common.config.types.Password password = + config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + assertNotNull(password); + + String actualQuery = password.value(); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testGetQueryStringReturnsQueryMaskedWhenSet() { + // Setup config with query.masked (but not query) + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String expectedQuery = "SELECT id, name, email FROM users"; + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // getQueryString() should return the masked query value + String actualQuery = config.getQuery(); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testGetQueryStringReturnsQueryWhenOnlyQuerySet() { + // Setup config with only query (not query.masked) + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String expectedQuery = "SELECT * FROM public_table"; + props.put(JdbcSourceConnectorConfig.QUERY_CONFIG, expectedQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // getQueryString() should return the regular query value + String actualQuery = config.getQuery(); + assertEquals(expectedQuery, actualQuery); + } + + @Test + public void testGetQueryStringPrioritizesQueryMaskedOverQuery() { + // Setup config with both query and query.masked + // (This violates validation but tests the priority logic) + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String regularQuery = "SELECT * FROM table1"; + String maskedQuery = "SELECT * FROM sensitive_table"; + props.put(JdbcSourceConnectorConfig.QUERY_CONFIG, regularQuery); + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, maskedQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // getQueryString() should prioritize query.masked + String actualQuery = config.getQuery(); + assertEquals(maskedQuery, actualQuery); + } + + @Test + public void testGetQueryStringReturnsEmptyWhenNeitherSet() { + // Setup config without query or query.masked + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // getQueryString() should return empty string + String actualQuery = config.getQuery(); + assertEquals("", actualQuery); + } + + @Test + public void testQueryMaskedSupportsComplexQueryWithMultipleJoins() { + // Test that complex queries work fine with query.masked + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + + String complexQuery = "SELECT u.id, u.username, u.email, " + + "p.profile_pic, a.street, a.city, a.country, " + + "o.order_id, o.order_date, o.total_amount " + + "FROM users u " + + "INNER JOIN profiles p ON u.id = p.user_id " + + "LEFT JOIN addresses a ON u.id = a.user_id " + + "LEFT JOIN orders o ON u.id = o.user_id " + + "WHERE u.status = 'active' " + + "AND u.created_at >= '2024-01-01' " + + "AND o.total_amount > 100.00 " + + "ORDER BY o.order_date DESC " + + "LIMIT 1000"; + + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, complexQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // Verify the full complex query can be retrieved + String retrievedQuery = config.getQuery(); + assertEquals(complexQuery, retrievedQuery); + } + + @Test + public void testQueryMaskedSupportsVeryLongQueries() { + // Test that very long queries (thousands of characters) work fine + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + + // Build a very long query (over 2000 characters) + StringBuilder longQuery = new StringBuilder("SELECT "); + for (int i = 0; i < 100; i++) { + longQuery.append("column_").append(i).append(", "); + } + longQuery.append("id FROM very_wide_table WHERE "); + for (int i = 0; i < 50; i++) { + longQuery.append("column_").append(i).append(" IS NOT NULL"); + if (i < 49) { + longQuery.append(" AND "); + } + } + + String expectedQuery = longQuery.toString(); + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // Verify the full long query can be retrieved without truncation + String retrievedQuery = config.getQuery(); + assertEquals(expectedQuery, retrievedQuery); + assertTrue(retrievedQuery.length() > 2000); + } + + @Test + public void testQueryMaskedSupportsSpecialCharacters() { + // Test that queries with special characters work fine + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + + String queryWithSpecialChars = "SELECT * FROM users " + + "WHERE name LIKE '%O''Brien%' " + + "AND email = 'test@example.com' " + + "AND description LIKE '%Line1\nLine2%' " + + "AND data LIKE '%Tab\tSeparated%'"; + + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, queryWithSpecialChars); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // Verify special characters are preserved + String retrievedQuery = config.getQuery(); + assertEquals(queryWithSpecialChars, retrievedQuery); + } + + @Test + public void testPasswordObjectToStringShowsHidden() { + // Test that Password type properly masks the value in toString() + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + String secretQuery = "SELECT * FROM secret_table WHERE api_key = 'secret123'"; + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, secretQuery); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + org.apache.kafka.common.config.types.Password password = + config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + + // toString() should hide the value + String passwordString = password.toString(); + assertEquals("[hidden]", passwordString); + + // But value() should return the actual query + String actualValue = password.value(); + assertEquals(secretQuery, actualValue); + } + + @Test + public void testConnectionPasswordIsSimilarlyMasked() { + // Compare with existing connection.password config to ensure consistent behavior + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG, "mySecretPassword123"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + + Map validatedConfig = JdbcSourceConnectorConfig.CONFIG_DEF.validateAll(props); + ConfigValue passwordValue = validatedConfig.get(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG); + + // Password should be masked in ConfigValue + String maskedString = passwordValue.value().toString(); + assertEquals("[hidden]", maskedString); + + // But can be retrieved via config + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + org.apache.kafka.common.config.types.Password password = + config.getPassword(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG); + assertEquals("mySecretPassword123", password.value()); + } + + @Test + public void testQueryMaskedWithEmptyStringBehavior() { + // Test behavior with empty string + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, ""); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + // getQueryString() should return empty string + String actualQuery = config.getQuery(); + assertEquals("", actualQuery); + } } diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index d1ab169a4..341d0506b 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -777,5 +777,96 @@ public void validate_withModeBulkWithNewIncrementingMapping_setsError() { assertErrors(MODE_CONFIG, 1); assertErrorMatches(MODE_CONFIG, ".*Incrementing column configurations should not be provided.*"); } + + // ========== Query and Query.Masked Config Tests ========== + + @Test + public void validate_withBothQueryAndQueryMasked_setsError() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + props.put(QUERY_CONFIG, "SELECT * FROM users"); + props.put(QUERY_MASKED_CONFIG, "SELECT * FROM sensitive_data"); + + validate(); + + assertErrors(2); + assertErrors(QUERY_CONFIG, 1); + assertErrors(QUERY_MASKED_CONFIG, 1); + assertErrorMatches(QUERY_CONFIG, ".*Both 'query' and 'query.masked' configs cannot be set.*"); + assertErrorMatches(QUERY_MASKED_CONFIG, ".*Both 'query' and 'query.masked' configs cannot be set.*"); + } + + @Test + public void validate_withOnlyQuery_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + props.put(QUERY_CONFIG, "SELECT * FROM users WHERE active = true"); + + validate(); + + assertNoErrors(); + } + + @Test + public void validate_withOnlyQueryMasked_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + props.put(QUERY_MASKED_CONFIG, "SELECT * FROM users WHERE active = true"); + + validate(); + + assertNoErrors(); + } + + @Test + public void validate_withQueryMaskedContainingComplexQuery_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + // Test with a complex query containing multiple joins + String complexQuery = "SELECT a.id, a.name, b.email, c.address, d.phone " + + "FROM users a " + + "INNER JOIN emails b ON a.id = b.user_id " + + "LEFT JOIN addresses c ON a.id = c.user_id " + + "LEFT JOIN phones d ON a.id = d.user_id " + + "WHERE a.created_at > '2024-01-01' AND a.status = 'active'"; + props.put(QUERY_MASKED_CONFIG, complexQuery); + + validate(); + + assertNoErrors(); + } + + @Test + public void validate_withQueryMaskedContainingVeryLongQuery_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + // Test with a very long query to ensure no length restrictions + StringBuilder longQuery = new StringBuilder("SELECT "); + for (int i = 0; i < 100; i++) { + longQuery.append("column").append(i); + if (i < 99) { + longQuery.append(", "); + } + } + longQuery.append(" FROM large_table WHERE condition = 'test'"); + props.put(QUERY_MASKED_CONFIG, longQuery.toString()); + + validate(); + + assertNoErrors(); + } + + @Test + public void validate_withBothQueryAndQueryMaskedEmpty_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + // Both empty should be fine as it's equivalent to neither being set + props.put(QUERY_CONFIG, ""); + props.put(QUERY_MASKED_CONFIG, ""); + + validate(); + + assertNoErrors(); + } } From bb933668ce87032d379fff55d7ac822cb0b88564 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Thu, 13 Nov 2025 13:35:19 +0530 Subject: [PATCH 02/22] Fix checkstyle errors and added tests --- .../source/JdbcSourceConnectorConfig.java | 53 +++++-- .../JdbcSourceConnectorValidation.java | 2 +- .../source/JdbcSourceConnectorConfigTest.java | 140 ------------------ .../JdbcSourceConnectorValidationTest.java | 20 --- 4 files changed, 39 insertions(+), 176 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 4a5e105da..64d44659c 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -335,21 +335,22 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { + "to this query (i.e. no WHERE clauses may be used). If you use a WHERE clause, it must " + "handle incremental queries itself."; public static final String QUERY_DEFAULT = ""; - private static final String QUERY_DISPLAY = "Query"; + private static final String QUERY_DISPLAY = "Query (Deprecated)"; public static final String QUERY_MASKED_CONFIG = "query.masked"; private static final String QUERY_MASKED_DOC = "If specified, the query to perform to select new or updated rows. This is the same as " - + "'query' config but with Type.PASSWORD to mask the query value from being visible to users. " - + "Use this setting when your query contains sensitive information. If used, this connector " - + "will only copy data using this query -- whole-table copying will be disabled. Different " - + "query modes may still be used for incremental updates, but in order to properly construct " - + "the incremental query, it must be possible to append a WHERE clause to this query (i.e. no " - + "WHERE clauses may be used). If you use a WHERE clause, it must handle incremental queries " - + "itself. Note: Only one of 'query' or 'query.masked' should be set, not both."; + + "'query' config but with Type.PASSWORD to mask the query value from being visible " + + "to users. Use this setting when your query contains sensitive information. " + + "If used, this connector will only copy data using this query -- whole-table " + + "copying will be disabled. Different query modes may still be used for " + + "incremental updates, but in order to properly construct the incremental query," + + " it must be possible to append a WHERE clause to this query (i.e. no " + + "WHERE clauses may be used). If you use a WHERE clause, it must handle " + + "incremental queries itself. Note: Only one of 'query' or 'query.masked' " + + "should be set, not both."; private static final String QUERY_MASKED_DISPLAY = "Query (Masked)"; - public static final String TOPIC_PREFIX_CONFIG = "topic.prefix"; private static final String TOPIC_PREFIX_DOC = "Prefix to prepend to table names to generate the name of the Kafka topic to publish data " @@ -839,6 +840,13 @@ private static void addSchemaAndDialectOptions(ConfigDef config, int orderInGrou private static final void addModeOptions(ConfigDef config) { int orderInGroup = 0; + orderInGroup = defineModeConfig(config, orderInGroup); + orderInGroup = defineIncrementTimestampConfigs(config, orderInGroup); + orderInGroup = defineQueryAndQuoteConfigs(config, orderInGroup); + defineTransactionAndRetryConfigs(config, orderInGroup); + } + + private static int defineModeConfig(ConfigDef config, int orderInGroup) { config.define( MODE_CONFIG, Type.STRING, @@ -863,7 +871,12 @@ private static final void addModeOptions(ConfigDef config) { TIMESTAMP_COLUMN_MAPPING_CONFIG, VALIDATE_NON_NULL_CONFIG ) - ).define( + ); + return orderInGroup; + } + + private static int defineIncrementTimestampConfigs(ConfigDef config, int orderInGroup) { + config.define( INCREMENTING_COLUMN_NAME_CONFIG, Type.STRING, INCREMENTING_COLUMN_NAME_DEFAULT, @@ -931,7 +944,12 @@ private static final void addModeOptions(ConfigDef config) { Width.SHORT, VALIDATE_NON_NULL_DISPLAY, MODE_DEPENDENTS_RECOMMENDER - ).define( + ); + return orderInGroup; + } + + private static int defineQueryAndQuoteConfigs(ConfigDef config, int orderInGroup) { + config.define( QUERY_CONFIG, Type.STRING, QUERY_DEFAULT, @@ -972,7 +990,12 @@ private static final void addModeOptions(ConfigDef config) { ++orderInGroup, Width.MEDIUM, QUERY_SUFFIX_DISPLAY - ).define( + ); + return orderInGroup; + } + + private static void defineTransactionAndRetryConfigs(ConfigDef config, int orderInGroup) { + config.define( TRANSACTION_ISOLATION_MODE_CONFIG, Type.STRING, TRANSACTION_ISOLATION_MODE_DEFAULT, @@ -1458,13 +1481,13 @@ public List incrementingColMappingRegexes() { /** * Get query string from either query.masked (Type.PASSWORD) or query (Type.STRING) config. - * Prioritizes query.masked if set, otherwise falls back to query config for backward compatibility. + * Prioritizes query.masked if set, otherwise falls back to query config for backward + * compatibility. * * @return The query string from whichever config is set, or empty string if neither is set. */ public String getQuery() { - // First check if query.masked is set - org.apache.kafka.common.config.types.Password maskedQuery = getPassword(QUERY_MASKED_CONFIG); + Password maskedQuery = getPassword(QUERY_MASKED_CONFIG); if (maskedQuery != null && maskedQuery.value() != null && !maskedQuery.value().isEmpty()) { return maskedQuery.value(); } diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 6f794097f..305cf068c 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -306,7 +306,7 @@ private boolean validateTableInclusionConfigs() { return true; } - /** + /** * Validate that only one of query or query.masked configs is set at a time. * Both configs should not be set simultaneously to avoid ambiguity. */ diff --git a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java index 9e8d8c074..c2ea1f4b0 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java @@ -453,45 +453,6 @@ public void testQueryMaskedValueIsMaskedInConfigValue() { assertFalse(maskedString.contains(sensitiveQuery)); } - @Test - public void testQueryMaskedValueCanBeRetrievedViaConfig() { - // Setup config with query.masked - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - String expectedQuery = "SELECT * FROM users WHERE status = 'active'"; - props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); - - // Create config - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - - // The actual value can be retrieved when needed - org.apache.kafka.common.config.types.Password password = - config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); - assertNotNull(password); - - String actualQuery = password.value(); - assertEquals(expectedQuery, actualQuery); - } - - @Test - public void testGetQueryStringReturnsQueryMaskedWhenSet() { - // Setup config with query.masked (but not query) - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - String expectedQuery = "SELECT id, name, email FROM users"; - props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); - - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - - // getQueryString() should return the masked query value - String actualQuery = config.getQuery(); - assertEquals(expectedQuery, actualQuery); - } - @Test public void testGetQueryStringReturnsQueryWhenOnlyQuerySet() { // Setup config with only query (not query.masked) @@ -504,31 +465,10 @@ public void testGetQueryStringReturnsQueryWhenOnlyQuerySet() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - // getQueryString() should return the regular query value String actualQuery = config.getQuery(); assertEquals(expectedQuery, actualQuery); } - @Test - public void testGetQueryStringPrioritizesQueryMaskedOverQuery() { - // Setup config with both query and query.masked - // (This violates validation but tests the priority logic) - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - String regularQuery = "SELECT * FROM table1"; - String maskedQuery = "SELECT * FROM sensitive_table"; - props.put(JdbcSourceConnectorConfig.QUERY_CONFIG, regularQuery); - props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, maskedQuery); - - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - - // getQueryString() should prioritize query.masked - String actualQuery = config.getQuery(); - assertEquals(maskedQuery, actualQuery); - } - @Test public void testGetQueryStringReturnsEmptyWhenNeitherSet() { // Setup config without query or query.masked @@ -569,43 +509,10 @@ public void testQueryMaskedSupportsComplexQueryWithMultipleJoins() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - // Verify the full complex query can be retrieved String retrievedQuery = config.getQuery(); assertEquals(complexQuery, retrievedQuery); } - @Test - public void testQueryMaskedSupportsVeryLongQueries() { - // Test that very long queries (thousands of characters) work fine - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - - // Build a very long query (over 2000 characters) - StringBuilder longQuery = new StringBuilder("SELECT "); - for (int i = 0; i < 100; i++) { - longQuery.append("column_").append(i).append(", "); - } - longQuery.append("id FROM very_wide_table WHERE "); - for (int i = 0; i < 50; i++) { - longQuery.append("column_").append(i).append(" IS NOT NULL"); - if (i < 49) { - longQuery.append(" AND "); - } - } - - String expectedQuery = longQuery.toString(); - props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, expectedQuery); - - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - - // Verify the full long query can be retrieved without truncation - String retrievedQuery = config.getQuery(); - assertEquals(expectedQuery, retrievedQuery); - assertTrue(retrievedQuery.length() > 2000); - } - @Test public void testQueryMaskedSupportsSpecialCharacters() { // Test that queries with special characters work fine @@ -629,52 +536,6 @@ public void testQueryMaskedSupportsSpecialCharacters() { assertEquals(queryWithSpecialChars, retrievedQuery); } - @Test - public void testPasswordObjectToStringShowsHidden() { - // Test that Password type properly masks the value in toString() - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - String secretQuery = "SELECT * FROM secret_table WHERE api_key = 'secret123'"; - props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, secretQuery); - - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - org.apache.kafka.common.config.types.Password password = - config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); - - // toString() should hide the value - String passwordString = password.toString(); - assertEquals("[hidden]", passwordString); - - // But value() should return the actual query - String actualValue = password.value(); - assertEquals(secretQuery, actualValue); - } - - @Test - public void testConnectionPasswordIsSimilarlyMasked() { - // Compare with existing connection.password config to ensure consistent behavior - Map props = new HashMap<>(); - props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); - props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); - props.put(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG, "mySecretPassword123"); - props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1,table2"); - - Map validatedConfig = JdbcSourceConnectorConfig.CONFIG_DEF.validateAll(props); - ConfigValue passwordValue = validatedConfig.get(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG); - - // Password should be masked in ConfigValue - String maskedString = passwordValue.value().toString(); - assertEquals("[hidden]", maskedString); - - // But can be retrieved via config - JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - org.apache.kafka.common.config.types.Password password = - config.getPassword(JdbcSourceConnectorConfig.CONNECTION_PASSWORD_CONFIG); - assertEquals("mySecretPassword123", password.value()); - } - @Test public void testQueryMaskedWithEmptyStringBehavior() { // Test behavior with empty string @@ -686,7 +547,6 @@ public void testQueryMaskedWithEmptyStringBehavior() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - // getQueryString() should return empty string String actualQuery = config.getQuery(); assertEquals("", actualQuery); } diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index 341d0506b..443e74723 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -836,26 +836,6 @@ public void validate_withQueryMaskedContainingComplexQuery_noErrors() { assertNoErrors(); } - @Test - public void validate_withQueryMaskedContainingVeryLongQuery_noErrors() { - props.put(MODE_CONFIG, MODE_BULK); - props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); - // Test with a very long query to ensure no length restrictions - StringBuilder longQuery = new StringBuilder("SELECT "); - for (int i = 0; i < 100; i++) { - longQuery.append("column").append(i); - if (i < 99) { - longQuery.append(", "); - } - } - longQuery.append(" FROM large_table WHERE condition = 'test'"); - props.put(QUERY_MASKED_CONFIG, longQuery.toString()); - - validate(); - - assertNoErrors(); - } - @Test public void validate_withBothQueryAndQueryMaskedEmpty_noErrors() { props.put(MODE_CONFIG, MODE_BULK); From 330e9fec7182dbb4a46c7035023db18060b531d5 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 14 Nov 2025 09:06:31 +0530 Subject: [PATCH 03/22] Making query masked config internal and fixed validation --- .../source/JdbcSourceConnectorConfig.java | 19 +++++++++- .../JdbcSourceConnectorValidation.java | 38 +++++++++++++++++-- 2 files changed, 52 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 64d44659c..2d4954f01 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -435,6 +435,20 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { private static final EnumRecommender QUOTE_METHOD_RECOMMENDER = EnumRecommender.in(QuoteMethod.values()); + private static final ConfigDef.Recommender HIDDEN_RECOMMENDER = + new ConfigDef.Recommender() { + @Override + public java.util.List validValues( + String name, java.util.Map parsedConfig) { + return java.util.Collections.emptyList(); + } + + @Override + public boolean visible(String name, java.util.Map parsedConfig) { + return false; + } + }; + public static final String DATABASE_GROUP = "Database"; public static final String MODE_GROUP = "Mode"; public static final String CONNECTOR_GROUP = "Connector"; @@ -964,11 +978,12 @@ private static int defineQueryAndQuoteConfigs(ConfigDef config, int orderInGroup Type.PASSWORD, QUERY_DEFAULT, Importance.MEDIUM, - QUERY_MASKED_DOC, + QUERY_DOC, MODE_GROUP, ++orderInGroup, Width.SHORT, - QUERY_MASKED_DISPLAY + QUERY_DISPLAY, + HIDDEN_RECOMMENDER ).define( QUOTE_SQL_IDENTIFIERS_CONFIG, Type.STRING, diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 305cf068c..264cf1f6d 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -21,6 +21,7 @@ import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TransactionIsolationMode; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.config.types.Password; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -151,6 +152,16 @@ private boolean validateMultiConfigs() { return true; } + private boolean hasAnyQueryConfig() { + String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); + Password queryMasked = + config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + boolean hasQuery = query != null && !query.trim().isEmpty(); + boolean hasQueryMasked = + queryMasked != null && queryMasked.value() != null && !queryMasked.value().trim().isEmpty(); + return hasQuery || hasQueryMasked; + } + /** * Validate legacy/new config compatibility and requirements. * Implements the pattern: legacyKeys vs newKeys with early returns. @@ -159,6 +170,11 @@ private boolean validateLegacyNewConfigCompatibility() { // Define legacy and new config keys boolean usingLegacyConfigs = isUsingLegacyConfigs(); boolean usingNewConfigs = isUsingNewConfigs(); + boolean hasQuery = hasAnyQueryConfig(); + + if (hasQuery) { + return true; + } if (usingLegacyConfigs && usingNewConfigs) { return addConfigErrorsForLegacyAndNewConfigConflict(); @@ -274,6 +290,9 @@ private boolean addConfigErrorsForLegacyAndNewConfigConflict() { * Validate that at least one configuration is provided. */ private boolean addConfigErrorsForNoConfigProvided() { + if (hasAnyQueryConfig()) { + return true; + } String msg = "At least one table filtering configuration is required. " + "Provide one of: " + JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG + ", " + JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG + ", " @@ -312,7 +331,7 @@ private boolean validateTableInclusionConfigs() { */ private boolean validateQueryConfigs() { String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); - org.apache.kafka.common.config.types.Password queryMasked = + Password queryMasked = config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); boolean hasQuery = query != null && !query.isEmpty(); @@ -322,8 +341,7 @@ private boolean validateQueryConfigs() { if (hasQuery && hasQueryMasked) { String msg = "Both 'query' and 'query.masked' configs cannot be set at the same time. " - + "Please use only one of them. Use 'query.masked' (Type.PASSWORD) if you want to hide " - + "the query value from being visible, or use 'query' (Type.STRING) for regular usage."; + + "Please use only one of them."; addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); @@ -332,6 +350,20 @@ private boolean validateQueryConfigs() { return false; } + if ((hasQuery || hasQueryMasked) && (isUsingLegacyConfigs() || isUsingNewConfigs())) { + String msg = + "Do not specify table filtering configs with 'query' or 'query.masked'. " + + "Remove table.whitelist / table.blacklist / table.include.list / " + + "table.exclude.list."; + addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); + addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); + return false; + } + return true; } From b8d712ab1635d6499ad11798d402861e95b49d41 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 14 Nov 2025 09:39:13 +0530 Subject: [PATCH 04/22] Fix UTs with Validation errors --- .../jdbc/validation/JdbcSourceConnectorValidationTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index 443e74723..942200139 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -799,7 +799,6 @@ public void validate_withBothQueryAndQueryMasked_setsError() { @Test public void validate_withOnlyQuery_noErrors() { props.put(MODE_CONFIG, MODE_BULK); - props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); props.put(QUERY_CONFIG, "SELECT * FROM users WHERE active = true"); validate(); @@ -810,7 +809,6 @@ public void validate_withOnlyQuery_noErrors() { @Test public void validate_withOnlyQueryMasked_noErrors() { props.put(MODE_CONFIG, MODE_BULK); - props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); props.put(QUERY_MASKED_CONFIG, "SELECT * FROM users WHERE active = true"); validate(); @@ -821,7 +819,6 @@ public void validate_withOnlyQueryMasked_noErrors() { @Test public void validate_withQueryMaskedContainingComplexQuery_noErrors() { props.put(MODE_CONFIG, MODE_BULK); - props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); // Test with a complex query containing multiple joins String complexQuery = "SELECT a.id, a.name, b.email, c.address, d.phone " + "FROM users a " + From 3a22709f3f1749127cf04aedf3f771f3ae5d7619 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 14 Nov 2025 14:35:46 +0530 Subject: [PATCH 05/22] Update the documentation of new config --- .../source/JdbcSourceConnectorConfig.java | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 2d4954f01..334304532 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -335,20 +335,12 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { + "to this query (i.e. no WHERE clauses may be used). If you use a WHERE clause, it must " + "handle incremental queries itself."; public static final String QUERY_DEFAULT = ""; - private static final String QUERY_DISPLAY = "Query (Deprecated)"; + private static final String QUERY_DISPLAY = "Query"; public static final String QUERY_MASKED_CONFIG = "query.masked"; private static final String QUERY_MASKED_DOC = - "If specified, the query to perform to select new or updated rows. This is the same as " - + "'query' config but with Type.PASSWORD to mask the query value from being visible " - + "to users. Use this setting when your query contains sensitive information. " - + "If used, this connector will only copy data using this query -- whole-table " - + "copying will be disabled. Different query modes may still be used for " - + "incremental updates, but in order to properly construct the incremental query," - + " it must be possible to append a WHERE clause to this query (i.e. no " - + "WHERE clauses may be used). If you use a WHERE clause, it must handle " - + "incremental queries itself. Note: Only one of 'query' or 'query.masked' " - + "should be set, not both."; + "Same as 'query' configuration but the query string is masked" + + "Use this config to prevent sensitive information from being logged."; private static final String QUERY_MASKED_DISPLAY = "Query (Masked)"; public static final String TOPIC_PREFIX_CONFIG = "topic.prefix"; @@ -439,12 +431,12 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { new ConfigDef.Recommender() { @Override public java.util.List validValues( - String name, java.util.Map parsedConfig) { + String name, Map config) { return java.util.Collections.emptyList(); } @Override - public boolean visible(String name, java.util.Map parsedConfig) { + public boolean visible(String name, Map config) { return false; } }; @@ -978,11 +970,11 @@ private static int defineQueryAndQuoteConfigs(ConfigDef config, int orderInGroup Type.PASSWORD, QUERY_DEFAULT, Importance.MEDIUM, - QUERY_DOC, + QUERY_MASKED_DOC, MODE_GROUP, ++orderInGroup, Width.SHORT, - QUERY_DISPLAY, + QUERY_MASKED_DISPLAY, HIDDEN_RECOMMENDER ).define( QUOTE_SQL_IDENTIFIERS_CONFIG, From 9330777b19c1fa855bf599b562372de92f05318c Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 21 Nov 2025 15:00:40 +0530 Subject: [PATCH 06/22] Update getQuery() method to return optional --- .../connect/jdbc/JdbcSourceConnector.java | 8 +- .../source/JdbcSourceConnectorConfig.java | 31 +++-- .../connect/jdbc/source/JdbcSourceTask.java | 9 +- .../jdbc/source/TableQuerierProcessor.java | 3 +- .../JdbcSourceConnectorValidation.java | 107 ++++++++---------- .../source/JdbcSourceConnectorConfigTest.java | 21 ++-- 6 files changed, 81 insertions(+), 98 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java b/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java index a460ad08f..9cfa4a792 100644 --- a/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java +++ b/src/main/java/io/confluent/connect/jdbc/JdbcSourceConnector.java @@ -117,8 +117,7 @@ public void start(Map properties) throws ConnectException { List excludeList = config.tableExcludeListRegexes(); Set excludeListSet = excludeList.isEmpty() ? null : new HashSet<>(excludeList); - String query = config.getQuery(); - if (!query.isEmpty()) { + if (config.getQuery().isPresent()) { if (whitelistSet != null || blacklistSet != null || includeListSet != null || excludeListSet != null) { log.error( @@ -145,7 +144,7 @@ public void start(Map properties) throws ConnectException { excludeListSet, // New Time.SYSTEM ); - if (query.isEmpty()) { + if (!config.getQuery().isPresent()) { tableMonitorThread.start(); log.info("Starting Table Monitor Thread"); } @@ -170,9 +169,8 @@ public Config validate(Map connectorConfigs) { @Override public List> taskConfigs(int maxTasks) { log.info("Starting with the task Configuration method."); - String query = config.getQuery(); List> taskConfigs; - if (!query.isEmpty()) { + if (config.getQuery().isPresent()) { log.info("Custom query provided, generating task configuration for the query"); Map taskProps = new HashMap<>(configProperties); taskProps.put(JdbcSourceTaskConfig.TABLES_CONFIG, ""); diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 334304532..53feae5dc 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -19,14 +19,7 @@ import java.sql.Timestamp; import java.time.Duration; import java.time.ZoneId; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.atomic.AtomicReference; import com.microsoft.sqlserver.jdbc.SQLServerConnection; @@ -1487,19 +1480,23 @@ public List incrementingColMappingRegexes() { } /** - * Get query string from either query.masked (Type.PASSWORD) or query (Type.STRING) config. - * Prioritizes query.masked if set, otherwise falls back to query config for backward - * compatibility. - * - * @return The query string from whichever config is set, or empty string if neither is set. - */ - public String getQuery() { + * Get the query string from either query or query.masked config. + * Prioritizes query.masked over query if both are set (though validation should prevent this). + * + * @return Optional containing the query string if present, empty Optional otherwise. + */ + public Optional getQuery() { Password maskedQuery = getPassword(QUERY_MASKED_CONFIG); if (maskedQuery != null && maskedQuery.value() != null && !maskedQuery.value().isEmpty()) { - return maskedQuery.value(); + return Optional.of(maskedQuery.value()); + } + + String query = getString(QUERY_CONFIG); + if (query != null && !query.isEmpty()) { + return Optional.of(query); } - return getString(QUERY_CONFIG); + return Optional.empty(); } public boolean modeUsesTimestampColumn() { diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 402289233..093c6f7e4 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -95,10 +95,9 @@ public void start(Map properties) { List tables = config.getList(JdbcSourceTaskConfig.TABLES_CONFIG); Boolean tablesFetched = config.getBoolean(JdbcSourceTaskConfig.TABLES_FETCHED); - String query = config.getQuery(); List tableType = config.getList(JdbcSourceConnectorConfig.TABLE_TYPE_CONFIG); - if ((tables.isEmpty() && query.isEmpty())) { + if ((tables.isEmpty() && !config.getQuery().isPresent())) { // We are still waiting for the tables call to complete. // Start task but do nothing. if (!tablesFetched) { @@ -115,7 +114,7 @@ public void start(Map properties) { + " table name."); } - if ((!tables.isEmpty() && !query.isEmpty())) { + if ((!tables.isEmpty() && config.getQuery().isPresent())) { throw new ConfigException("Invalid configuration: a JdbcSourceTask" + " cannot have both a table and a query assigned to it"); } @@ -147,10 +146,10 @@ public void start(Map properties) { ) ) ); - TableQuerier.QueryMode queryMode = !query.isEmpty() ? TableQuerier.QueryMode.QUERY : + TableQuerier.QueryMode queryMode = config.getQuery().isPresent() ? TableQuerier.QueryMode.QUERY : TableQuerier.QueryMode.TABLE; final List tablesOrQuery = queryMode == TableQuerier.QueryMode.QUERY - ? Collections.singletonList(query) : tables; + ? Collections.singletonList(config.getQuery().get()) : tables; String mode = config.getString(JdbcSourceTaskConfig.MODE_CONFIG); //used only in table mode diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 7eb5924f1..3f5d2e97d 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -95,8 +95,7 @@ private boolean isReadyToProcess() { // If the call to get tables has not completed we will not do anything. // This is only valid in table mode. Boolean tablesFetched = config.getBoolean(JdbcSourceTaskConfig.TABLES_FETCHED); - String query = config.getQuery(); - return !query.isEmpty() || tablesFetched; + return config.getQuery().isPresent() || tablesFetched; } private void processQuerier(RecordDestination destination, TableQuerier querier) diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 264cf1f6d..a99dd94fc 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -21,7 +21,6 @@ import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TransactionIsolationMode; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigValue; -import org.apache.kafka.common.config.types.Password; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,17 +89,17 @@ && validateLegacyNewConfigCompatibility() } validationResult = validationResult && validatePluginSpecificNeeds(); - + if (!validationResult) { log.info("Validation failed"); } else { log.info("Validation succeeded"); } - + } catch (Exception e) { log.error("Error during validation", e); } - + return this.validationResult; } @@ -152,16 +151,6 @@ private boolean validateMultiConfigs() { return true; } - private boolean hasAnyQueryConfig() { - String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); - Password queryMasked = - config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); - boolean hasQuery = query != null && !query.trim().isEmpty(); - boolean hasQueryMasked = - queryMasked != null && queryMasked.value() != null && !queryMasked.value().trim().isEmpty(); - return hasQuery || hasQueryMasked; - } - /** * Validate legacy/new config compatibility and requirements. * Implements the pattern: legacyKeys vs newKeys with early returns. @@ -170,9 +159,8 @@ private boolean validateLegacyNewConfigCompatibility() { // Define legacy and new config keys boolean usingLegacyConfigs = isUsingLegacyConfigs(); boolean usingNewConfigs = isUsingNewConfigs(); - boolean hasQuery = hasAnyQueryConfig(); - if (hasQuery) { + if (config.getQuery().isPresent()) { return true; } @@ -196,18 +184,18 @@ private boolean isUsingLegacyConfigs() { Set blacklistSet = config.getTableBlacklistSet(); String incrementingColumnName = config.getIncrementingColumnName(); List timestampColumnName = config.getTimestampColumnName(); - + boolean hasWhitelist = !whitelistSet.isEmpty(); boolean hasBlacklist = !blacklistSet.isEmpty(); - boolean hasLegacyIncrementing = incrementingColumnName != null + boolean hasLegacyIncrementing = incrementingColumnName != null && !incrementingColumnName.trim().isEmpty(); - boolean hasLegacyTimestamp = timestampColumnName != null - && !timestampColumnName.isEmpty() + boolean hasLegacyTimestamp = timestampColumnName != null + && !timestampColumnName.isEmpty() && !timestampColumnName.get(0).trim().isEmpty(); - + return hasWhitelist || hasBlacklist || hasLegacyIncrementing || hasLegacyTimestamp; } - + /** * Check if any new config keys are being used. * New keys: table.include.list, table.exclude.list, incrementing.column.mapping, @@ -218,17 +206,17 @@ private boolean isUsingNewConfigs() { Set excludeListSet = config.getTableExcludeListSet(); List incrementingColumnMapping = config.getIncrementingColumnMapping(); List timestampColumnsMapping = config.getTimestampColumnMapping(); - + boolean hasIncludeList = !includeListSet.isEmpty(); boolean hasExcludeList = !excludeListSet.isEmpty(); - boolean hasNewIncrementing = incrementingColumnMapping != null + boolean hasNewIncrementing = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); - boolean hasNewTimestamp = timestampColumnsMapping != null + boolean hasNewTimestamp = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - + return hasIncludeList || hasExcludeList || hasNewIncrementing || hasNewTimestamp; } - + /** * Validate conflict between legacy and new configs. * Only add errors to configs that are actually present and conflicting. @@ -240,57 +228,57 @@ private boolean addConfigErrorsForLegacyAndNewConfigConflict() { + "(table.include.list, table.exclude.list, timestamp.columns.mapping, " + "incrementing.column.mapping). Please choose one approach: either use all legacy " + "configurations or all new configurations."; - + // Only add errors to configs that are actually present and non-empty Set whitelistSet = config.getTableWhitelistSet(); if (!whitelistSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); } - + Set blacklistSet = config.getTableBlacklistSet(); if (!blacklistSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); } - + Set includeListSet = config.getTableIncludeListSet(); if (!includeListSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); } - + Set excludeListSet = config.getTableExcludeListSet(); if (!excludeListSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); } - + List timestampColumnName = config.getTimestampColumnName(); - if (timestampColumnName != null && !timestampColumnName.isEmpty() + if (timestampColumnName != null && !timestampColumnName.isEmpty() && !timestampColumnName.get(0).trim().isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TIMESTAMP_COLUMN_NAME_CONFIG, msg); } - + List timestampColumnsMapping = config.getTimestampColumnMapping(); if (timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TIMESTAMP_COLUMN_MAPPING_CONFIG, msg); } - + String incrementingColumnName = config.getIncrementingColumnName(); if (incrementingColumnName != null && !incrementingColumnName.trim().isEmpty()) { addConfigError(JdbcSourceConnectorConfig.INCREMENTING_COLUMN_NAME_CONFIG, msg); } - + List incrementingColumnMapping = config.getIncrementingColumnMapping(); if (incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.INCREMENTING_COLUMN_MAPPING_CONFIG, msg); } - + return false; } - + /** * Validate that at least one configuration is provided. */ private boolean addConfigErrorsForNoConfigProvided() { - if (hasAnyQueryConfig()) { + if (config.getQuery().isPresent()) { return true; } String msg = "At least one table filtering configuration is required. " @@ -298,30 +286,30 @@ private boolean addConfigErrorsForNoConfigProvided() { + JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG + ", " + JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG + ", or " + JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG + "."; - + addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); return false; } - + /** * Validate new config requirements (when using new configs only). */ private boolean validateTableInclusionConfigs() { Set includeListSet = config.getTableIncludeListSet(); Set excludeListSet = config.getTableExcludeListSet(); - + // Validate that exclude list requires include list if (!excludeListSet.isEmpty() && includeListSet.isEmpty()) { - String msg = JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG + String msg = JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG + " cannot be used without " + JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG + ". Exclude list only applies to tables that match the include list."; addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); return false; } - + return true; } @@ -331,13 +319,15 @@ private boolean validateTableInclusionConfigs() { */ private boolean validateQueryConfigs() { String query = config.getString(JdbcSourceConnectorConfig.QUERY_CONFIG); - Password queryMasked = + String queryMaskedValue = null; + org.apache.kafka.common.config.types.Password queryMasked = config.getPassword(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG); + if (queryMasked != null && queryMasked.value() != null) { + queryMaskedValue = queryMasked.value(); + } boolean hasQuery = query != null && !query.isEmpty(); - boolean hasQueryMasked = queryMasked != null - && queryMasked.value() != null - && !queryMasked.value().isEmpty(); + boolean hasQueryMasked = queryMaskedValue != null && !queryMaskedValue.isEmpty(); if (hasQuery && hasQueryMasked) { String msg = "Both 'query' and 'query.masked' configs cannot be set at the same time. " @@ -350,11 +340,12 @@ private boolean validateQueryConfigs() { return false; } - if ((hasQuery || hasQueryMasked) && (isUsingLegacyConfigs() || isUsingNewConfigs())) { + if (config.getQuery().isPresent() && (isUsingLegacyConfigs() || isUsingNewConfigs())) { String msg = "Do not specify table filtering configs with 'query' or 'query.masked'. " + "Remove table.whitelist / table.blacklist / table.include.list / " - + "table.exclude.list."; + + "table.exclude.list when using query mode" + + " or 'query' / 'query.masked' when using table filtering mode."; addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); @@ -397,7 +388,7 @@ private boolean validateTsColProvidedWhenRequired() { List timestampColumnsMapping = config.getTimestampColumnMapping(); boolean hasNewTimestampConfig = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - + if (!hasNewTimestampConfig) { String msg = String.format( "Timestamp column configuration must be provided when using mode '%s' or '%s'. " @@ -420,10 +411,10 @@ private boolean validateTsColProvidedWhenRequired() { private boolean validateTsColNotProvidedWhenNotRequired() { if (!config.modeUsesTimestampColumn()) { List timestampColumnsMapping = config.getTimestampColumnMapping(); - + boolean hasNewTimestampConfig = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - + if (hasNewTimestampConfig) { String msg = String.format( "Timestamp column configurations should not be provided if mode is not '%s' or '%s'. " @@ -448,10 +439,10 @@ private boolean validateTsColNotProvidedWhenNotRequired() { private boolean validateIncrColProvidedWhenRequired() { if (config.modeUsesIncrementingColumn()) { List incrementingColumnMapping = config.getIncrementingColumnMapping(); - + boolean hasNewIncrementingConfig = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); - + if (!hasNewIncrementingConfig) { String msg = String.format( "Incrementing column configuration must be provided when using mode '%s' or '%s'. " @@ -474,10 +465,10 @@ private boolean validateIncrColProvidedWhenRequired() { private boolean validateIncrColumnNotProvidedWhenNotRequired() { if (!config.modeUsesIncrementingColumn()) { List incrementingColumnMapping = config.getIncrementingColumnMapping(); - + boolean hasNewIncrementingConfig = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); - + if (hasNewIncrementingConfig) { String msg = String.format( "Incrementing column configurations " @@ -504,4 +495,4 @@ protected void addConfigError(String configName, String errorMessage) { .ifPresent(cv -> cv.addErrorMessage(errorMessage)); } -} +} \ No newline at end of file diff --git a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java index c2ea1f4b0..f90877b25 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java @@ -465,8 +465,8 @@ public void testGetQueryStringReturnsQueryWhenOnlyQuerySet() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - String actualQuery = config.getQuery(); - assertEquals(expectedQuery, actualQuery); + assertTrue(config.getQuery().isPresent()); + assertEquals(expectedQuery, config.getQuery().get()); } @Test @@ -479,9 +479,8 @@ public void testGetQueryStringReturnsEmptyWhenNeitherSet() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - // getQueryString() should return empty string - String actualQuery = config.getQuery(); - assertEquals("", actualQuery); + // getQuery() should return empty Optional + assertFalse(config.getQuery().isPresent()); } @Test @@ -509,8 +508,8 @@ public void testQueryMaskedSupportsComplexQueryWithMultipleJoins() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - String retrievedQuery = config.getQuery(); - assertEquals(complexQuery, retrievedQuery); + assertTrue(config.getQuery().isPresent()); + assertEquals(complexQuery, config.getQuery().get()); } @Test @@ -532,8 +531,8 @@ public void testQueryMaskedSupportsSpecialCharacters() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); // Verify special characters are preserved - String retrievedQuery = config.getQuery(); - assertEquals(queryWithSpecialChars, retrievedQuery); + assertTrue(config.getQuery().isPresent()); + assertEquals(queryWithSpecialChars, config.getQuery().get()); } @Test @@ -547,7 +546,7 @@ public void testQueryMaskedWithEmptyStringBehavior() { JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); - String actualQuery = config.getQuery(); - assertEquals("", actualQuery); + // Empty string should return empty Optional + assertFalse(config.getQuery().isPresent()); } } From 48350cad7a3c27c3a3bf7c965ce5a383b868b826 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 21 Nov 2025 15:40:56 +0530 Subject: [PATCH 07/22] Minor checkstyle fixes --- .../connect/jdbc/source/JdbcSourceConnectorConfig.java | 10 +++++++++- .../confluent/connect/jdbc/source/JdbcSourceTask.java | 4 ++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 53feae5dc..53ff94717 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -19,7 +19,15 @@ import java.sql.Timestamp; import java.time.Duration; import java.time.ZoneId; -import java.util.*; +import java.util.Arrays; +import java.util.HashSet; +import java.util.HashMap; +import java.util.Map; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.Optional; +import java.util.Locale; import java.util.concurrent.atomic.AtomicReference; import com.microsoft.sqlserver.jdbc.SQLServerConnection; diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 093c6f7e4..0068ba4ec 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -146,8 +146,8 @@ public void start(Map properties) { ) ) ); - TableQuerier.QueryMode queryMode = config.getQuery().isPresent() ? TableQuerier.QueryMode.QUERY : - TableQuerier.QueryMode.TABLE; + TableQuerier.QueryMode queryMode = + config.getQuery().isPresent() ? TableQuerier.QueryMode.QUERY : TableQuerier.QueryMode.TABLE; final List tablesOrQuery = queryMode == TableQuerier.QueryMode.QUERY ? Collections.singletonList(config.getQuery().get()) : tables; From a202be678282988c09fe5b78a802c28d141e75bd Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 25 Nov 2025 16:38:16 +0530 Subject: [PATCH 08/22] Fixed Validation with query mode --- .../JdbcSourceConnectorValidation.java | 28 +++++++++++++++---- .../JdbcSourceConnectorValidationTest.java | 11 ++++++++ 2 files changed, 34 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index a99dd94fc..2770b3a47 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -340,7 +340,7 @@ private boolean validateQueryConfigs() { return false; } - if (config.getQuery().isPresent() && (isUsingLegacyConfigs() || isUsingNewConfigs())) { + if (config.getQuery().isPresent() && isUsingTableFilteringConfigs()) { String msg = "Do not specify table filtering configs with 'query' or 'query.masked'. " + "Remove table.whitelist / table.blacklist / table.include.list / " @@ -348,10 +348,18 @@ private boolean validateQueryConfigs() { + " or 'query' / 'query.masked' when using table filtering mode."; addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); - addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); - addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); - addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); - addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); + if (!config.getTableWhitelistSet().isEmpty()) { + addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); + } + if (!config.getTableBlacklistSet().isEmpty()) { + addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); + } + if (!config.getTableIncludeListSet().isEmpty()) { + addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); + } + if (!config.getTableExcludeListSet().isEmpty()) { + addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); + } return false; } @@ -495,4 +503,14 @@ protected void addConfigError(String configName, String errorMessage) { .ifPresent(cv -> cv.addErrorMessage(errorMessage)); } + /** + * Determine whether any table filtering configurations are in use. + */ + private boolean isUsingTableFilteringConfigs() { + return !config.getTableWhitelistSet().isEmpty() + || !config.getTableBlacklistSet().isEmpty() + || !config.getTableIncludeListSet().isEmpty() + || !config.getTableExcludeListSet().isEmpty(); + } + } \ No newline at end of file diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index 942200139..b5b98e2ba 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -816,6 +816,17 @@ public void validate_withOnlyQueryMasked_noErrors() { assertNoErrors(); } + @Test + public void validate_withQueryMaskedAndIncrementingColumn_noErrors() { + props.put(MODE_CONFIG, MODE_INCREMENTING); + props.put(QUERY_MASKED_CONFIG, "SELECT * FROM users"); + props.put(INCREMENTING_COLUMN_NAME_CONFIG, "id"); + + validate(); + + assertNoErrors(); + } + @Test public void validate_withQueryMaskedContainingComplexQuery_noErrors() { props.put(MODE_CONFIG, MODE_BULK); From 00815627632f980afa574eb70845601b5556c0e1 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 26 Nov 2025 13:05:29 +0530 Subject: [PATCH 09/22] Added functionality to fail tasks when non-retriable exceptions --- pom.xml | 5 + .../jdbc/source/TableQuerierProcessor.java | 19 ++- .../connect/jdbc/util/RetryUtils.java | 146 ++++++++++++++++++ .../connect/jdbc/util/RetryUtilsTest.java | 42 +++++ 4 files changed, 208 insertions(+), 4 deletions(-) create mode 100644 src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java create mode 100644 src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java diff --git a/pom.xml b/pom.xml index f0b9187f4..da16e6db8 100644 --- a/pom.xml +++ b/pom.xml @@ -110,6 +110,11 @@ com.google.re2j re2j 1.7 + + + org.apache.commons + commons-lang3 + 3.12.0 diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 3f5d2e97d..30af66bb2 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -7,6 +7,7 @@ import io.confluent.connect.jdbc.dialect.DatabaseDialect; import io.confluent.connect.jdbc.util.CachedConnectionProvider; import io.confluent.connect.jdbc.util.RecordDestination; +import io.confluent.connect.jdbc.util.RetryUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.SourceRecord; @@ -137,11 +138,21 @@ private void handleNonTransientException(RecordDestination destina destination.failWith(new ConnectException(sqle)); } - private void handleSqlException(RecordDestination destination, + private void handleSqlException(RecordDestination destination, TableQuerier querier, SQLException sqle) { + if (!RetryUtils.shouldRetry(sqle)) { + log.error( + "Non-retriable SQL exception while running query for table: {}. Failing task.", + querier, + sqle + ); + resetAndRequeueHead(querier, true); + destination.failWith(new ConnectException("Non-retriable SQL exception", sqle)); + return; + } + log.error( - "SQL exception while running query for table: {}." - + " Attempting retry {} of {} attempts.", + "SQL exception while running query for table: {}. Attempting retry {} of {} attempts.", querier, querier.getAttemptedRetryCount() + 1, maxRetriesPerQuerier, @@ -151,7 +162,7 @@ private void handleSqlException(RecordDestination destination, resetAndRequeueHead(querier, false); if (maxRetriesPerQuerier > 0 && querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) { - destination.failWith(new ConnectException("Failed to Query table after retries", sqle)); + destination.failWith(new ConnectException("Failed to query table after retries", sqle)); return; } querier.incrementRetryCount(); diff --git a/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java b/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java new file mode 100644 index 000000000..924cf6140 --- /dev/null +++ b/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java @@ -0,0 +1,146 @@ +/* + * Copyright 2024 Confluent Inc. + * + * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.connect.jdbc.util; + +import org.apache.commons.lang3.exception.ExceptionUtils; + +import java.sql.SQLException; +import java.sql.SQLNonTransientException; +import java.sql.SQLRecoverableException; +import java.sql.SQLTransientException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Locale; +import java.util.Set; + +/** + * Utility class for determining whether SQL exceptions should be retried. + */ +public final class RetryUtils { + + private static final String CONNECTION_CLOSED_MSG = "CONNECTION IS CLOSED"; + + private static final Set RETRIABLE_SQLSTATE_PREFIXES = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList( + "08", // connection exception + "40", // transaction rollback + "55" // object not available / lock not available (PG) + ))); + + private static final Set RETRIABLE_SQLSTATES = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList( + "40001", // serialization failure + "40P01", // deadlock detected (PostgreSQL) + "55P03", // lock not available (PostgreSQL) + "57014", // query canceled (PostgreSQL, DB2) + "57033" // connection ended (DB2) + ))); + + private static final Set RETRIABLE_MESSAGE_TOKENS = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList( + "ORA-01466", // Oracle definition changed + "ORA-01284", // Oracle file cannot be opened + "ORA-03113", // end-of-file communication channel + "ORA-03114", // not connected to ORACLE + "ORA-12541", // TNS no listener + "ORA-12545", // TNS host or object not known + "DEADLOCK DETECTED", + "LOCK REQUEST TIMEOUT", // SQL Server + "SQLCODE=-911" // DB2 deadlock/timeout + ))); + + private static final Set RETRIABLE_ERROR_CODES = + Collections.synchronizedSet(new HashSet<>(Arrays.asList( + 1466, 1284, 3113, 3114, 4068, // Oracle + 1205, 1213, // MySQL and SQL Server deadlock/timeouts + -911 // DB2 deadlock/timeout when reported via error code + ))); + + public static void addRetriableErrorCodes(Collection errorCodes) { + if (errorCodes == null || errorCodes.isEmpty()) { + return; + } + RETRIABLE_ERROR_CODES.addAll(errorCodes); + } + + public static boolean shouldRetry(Throwable throwable) { + if (throwable == null) { + return false; + } + for (Throwable t : ExceptionUtils.getThrowableList(throwable)) { + if (t instanceof SQLException) { + for (SQLException se = (SQLException) t; se != null; se = se.getNextException()) { + if (canRetry(se)) { + return true; + } + if (cannotRetry(se)) { + return false; + } + } + } + } + return false; + } + + private static boolean canRetry(SQLException se) { + return isRecoverableAndNotClosed(se) + || (se instanceof SQLTransientException) + || hasRetriableErrorCodeOrState(se) + || messageHasRetriableTokens(se); + } + + private static boolean hasRetriableErrorCodeOrState(SQLException se) { + return RETRIABLE_ERROR_CODES.contains(se.getErrorCode()) || hasRetriableSqlState(se); + } + + private static boolean isRecoverableAndNotClosed(SQLException se) { + return se instanceof SQLRecoverableException + && (se.getMessage() == null + || !se.getMessage().toUpperCase(Locale.ROOT).contains(CONNECTION_CLOSED_MSG)); + } + + private static boolean cannotRetry(SQLException se) { + return se instanceof SQLNonTransientException; + } + + private static boolean hasRetriableSqlState(SQLException se) { + String sqlState = se.getSQLState(); + return sqlState != null + && !sqlState.isEmpty() + && (hasExactSqlStateMatch(sqlState) || hasSqlStatePrefixMatch(sqlState)); + } + + private static boolean hasExactSqlStateMatch(String sqlState) { + return RETRIABLE_SQLSTATES.contains(sqlState.toUpperCase(Locale.ROOT)); + } + + private static boolean hasSqlStatePrefixMatch(String sqlState) { + return sqlState.length() >= 2 + && RETRIABLE_SQLSTATE_PREFIXES.contains( + sqlState.substring(0, 2).toUpperCase(Locale.ROOT)); + } + + private static boolean messageHasRetriableTokens(SQLException se) { + String message = se.getMessage(); + return message != null + && !message.isEmpty() + && RETRIABLE_MESSAGE_TOKENS.stream() + .anyMatch(token -> message.toUpperCase(Locale.ROOT).contains(token)); + } +} + diff --git a/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java b/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java new file mode 100644 index 000000000..187e6cb86 --- /dev/null +++ b/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java @@ -0,0 +1,42 @@ +/* + * Copyright 2024 Confluent Inc. + */ + +package io.confluent.connect.jdbc.util; + +import org.junit.Test; + +import java.sql.SQLException; +import java.sql.SQLNonTransientException; +import java.sql.SQLTransientException; +import java.util.Collections; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RetryUtilsTest { + + @Test + public void shouldRetryOnTransientException() { + SQLException exception = new SQLTransientException("temporary issue"); + + assertTrue("Transient exceptions should be retried", RetryUtils.shouldRetry(exception)); + } + + @Test + public void shouldNotRetryOnNonTransientException() { + SQLException exception = new SQLNonTransientException("permanent failure"); + + assertFalse("Non transient exceptions must not be retried", RetryUtils.shouldRetry(exception)); + } + + @Test + public void shouldRetryWhenErrorCodeRegistered() { + int customErrorCode = 987654; + RetryUtils.addRetriableErrorCodes(Collections.singleton(customErrorCode)); + SQLException exception = new SQLException("custom error", "S0001", customErrorCode); + + assertTrue("Registered error codes should be retried", RetryUtils.shouldRetry(exception)); + } +} + From 7eb279a6f4162f23eb5380eb518c4069ac7232b0 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 28 Nov 2025 09:24:41 +0530 Subject: [PATCH 10/22] Redact the sensitive values from logs and exceptions and add validation to allow only select queries in query configs --- .../connect/jdbc/source/BulkTableQuerier.java | 3 +- .../source/JdbcSourceConnectorConfig.java | 8 ++++ .../connect/jdbc/source/JdbcSourceTask.java | 24 +++++++---- .../connect/jdbc/source/TableQuerier.java | 16 ++++++- .../jdbc/source/TableQuerierProcessor.java | 42 +++++++++++++------ .../TimestampIncrementingTableQuerier.java | 22 ++++++---- .../jdbc/source/TimestampTableQuerier.java | 3 ++ .../confluent/connect/jdbc/util/LogUtil.java | 20 +++++++-- .../JdbcSourceConnectorValidation.java | 29 +++++++++++++ .../connect/jdbc/source/TableQuerierTest.java | 12 +++++- ...TimestampIncrementingTableQuerierTest.java | 3 ++ .../source/TimestampTableQuerierTest.java | 3 ++ .../connect/jdbc/util/LogUtilTest.java | 26 ++++++++---- .../JdbcSourceConnectorValidationTest.java | 12 ++++++ 14 files changed, 180 insertions(+), 43 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index 2f2d0613d..af0c2c936 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -40,13 +40,14 @@ public class BulkTableQuerier extends TableQuerier { private static final Logger log = LoggerFactory.getLogger(BulkTableQuerier.class); public BulkTableQuerier( + JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String name, String topicPrefix, String suffix ) { - super(dialect, mode, name, topicPrefix, suffix); + super(config, dialect, mode, name, topicPrefix, suffix); } @Override diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 53ff94717..78e9f5d8c 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -408,6 +408,9 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { "Number of times to retry SQL exceptions encountered when executing queries."; public static final String QUERY_RETRIES_DISPLAY = "Query Retry Attempts"; + public static final String TRIM_SENSITIVE_LOG_ENABLED = "trim.sensitive.log"; + private static final String TRIM_SENSITIVE_LOG_ENABLED_DEFAULT = "true"; + /** * The properties that begin with this prefix will be used to configure a class, specified by * {@code jdbc.credentials.provider.class} if it implements {@link Configurable}. @@ -1084,6 +1087,11 @@ private static final void addConnectorOptions(ConfigDef config) { Type.LONG, TABLE_MONITORING_STARTUP_POLLING_LIMIT_MS_DEFAULT, Importance.LOW + ).defineInternal( + TRIM_SENSITIVE_LOG_ENABLED, + ConfigDef.Type.BOOLEAN, + TRIM_SENSITIVE_LOG_ENABLED_DEFAULT, + ConfigDef.Importance.LOW ).define( TABLE_POLL_INTERVAL_MS_CONFIG, Type.LONG, diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 0068ba4ec..dee4ab709 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -15,6 +15,14 @@ package io.confluent.connect.jdbc.source; +import io.confluent.connect.jdbc.util.LogUtil; +import io.confluent.connect.jdbc.util.TableId; +import io.confluent.connect.jdbc.util.TableCollectionUtils; +import io.confluent.connect.jdbc.util.ColumnDefinition; +import io.confluent.connect.jdbc.util.ColumnId; +import io.confluent.connect.jdbc.util.CachedConnectionProvider; +import io.confluent.connect.jdbc.util.RecordQueue; +import io.confluent.connect.jdbc.util.Version; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; @@ -43,13 +51,6 @@ import io.confluent.connect.jdbc.dialect.DatabaseDialect; import io.confluent.connect.jdbc.dialect.DatabaseDialects; -import io.confluent.connect.jdbc.util.CachedConnectionProvider; -import io.confluent.connect.jdbc.util.ColumnDefinition; -import io.confluent.connect.jdbc.util.ColumnId; -import io.confluent.connect.jdbc.util.RecordQueue; -import io.confluent.connect.jdbc.util.TableCollectionUtils; -import io.confluent.connect.jdbc.util.TableId; -import io.confluent.connect.jdbc.util.Version; import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TransactionIsolationMode; /** @@ -287,6 +288,7 @@ mode, getIncrementingColumn(tables.get(0)), if (mode.equals(JdbcSourceTaskConfig.MODE_BULK)) { tableQueue.add( new BulkTableQuerier( + config, dialect, queryMode, tableOrQuery, @@ -297,6 +299,7 @@ mode, getIncrementingColumn(tables.get(0)), } else if (mode.equals(JdbcSourceTaskConfig.MODE_INCREMENTING)) { tableQueue.add( new TimestampIncrementingTableQuerier( + config, dialect, queryMode, tableOrQuery, @@ -313,6 +316,7 @@ mode, getIncrementingColumn(tables.get(0)), } else if (mode.equals(JdbcSourceTaskConfig.MODE_TIMESTAMP)) { tableQueue.add( new TimestampTableQuerier( + config, dialect, queryMode, tableOrQuery, @@ -328,6 +332,7 @@ mode, getIncrementingColumn(tables.get(0)), } else if (mode.endsWith(JdbcSourceTaskConfig.MODE_TIMESTAMP_INCREMENTING)) { tableQueue.add( new TimestampIncrementingTableQuerier( + config, dialect, queryMode, tableOrQuery, @@ -458,6 +463,11 @@ protected Map computeInitialOffset( String tableOrQuery, Map partitionOffset, ZoneId zoneId) { + Boolean shouldTrimSensitiveLogs = + config.getBoolean(JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); + if (shouldTrimSensitiveLogs) { + tableOrQuery = LogUtil.sensitiveLog(true, tableOrQuery); + } if (!(partitionOffset == null)) { log.info("Partition offset for '{}' is not null. Using existing offset.", tableOrQuery); return partitionOffset; diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 9f070c22b..5e1972cfd 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -15,6 +15,7 @@ package io.confluent.connect.jdbc.source; +import io.confluent.connect.jdbc.util.LogUtil; import org.apache.kafka.connect.source.SourceRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +42,7 @@ public enum QueryMode { private final Logger log = LoggerFactory.getLogger(TableQuerier.class); + protected final JdbcSourceTaskConfig config; protected final DatabaseDialect dialect; protected final QueryMode mode; protected final String query; @@ -56,16 +58,19 @@ public enum QueryMode { protected ResultSet resultSet; protected SchemaMapping schemaMapping; private String loggedQueryString; + private final Boolean shouldTrimSensitiveLogs; private int attemptedRetries; public TableQuerier( + JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String nameOrQuery, String topicPrefix, String suffix ) { + this.config = config; this.dialect = dialect; this.mode = mode; this.tableId = mode.equals(QueryMode.TABLE) ? dialect.parseTableIdentifier(nameOrQuery) : null; @@ -74,6 +79,8 @@ public TableQuerier( this.lastUpdate = 0; this.suffix = suffix; this.attemptedRetries = 0; + this.shouldTrimSensitiveLogs = config.getBoolean( + JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); } public long getLastUpdate() { @@ -178,12 +185,19 @@ protected void addSuffixIfPresent(ExpressionBuilder builder) { protected void recordQuery(String query) { if (query != null && !query.equals(loggedQueryString)) { + String querierLog = getQuerierLogString(query); // For usability, log the statement at INFO level only when it changes - log.info("Begin using SQL query: {}", query); + log.info("Begin using SQL query: {}", querierLog); loggedQueryString = query; } } + private String getQuerierLogString(String query) { + return shouldTrimSensitiveLogs + ? LogUtil.sensitiveLog(true, query) + : query; + } + @Override public int compareTo(TableQuerier other) { if (this.lastUpdate < other.lastUpdate) { diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 30af66bb2..fdaf2037e 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -6,6 +6,7 @@ import io.confluent.connect.jdbc.dialect.DatabaseDialect; import io.confluent.connect.jdbc.util.CachedConnectionProvider; +import io.confluent.connect.jdbc.util.LogUtil; import io.confluent.connect.jdbc.util.RecordDestination; import io.confluent.connect.jdbc.util.RetryUtils; import org.apache.kafka.common.utils.Time; @@ -26,6 +27,7 @@ public class TableQuerierProcessor { private final JdbcSourceTaskConfig config; private final Time time; private final PriorityQueue tableQueue; + private final Boolean shouldTrimSensitiveLogs; private CachedConnectionProvider cachedConnectionProvider; private final int maxRetriesPerQuerier; private final Duration timeout = Duration.ofSeconds(90); @@ -42,6 +44,8 @@ public TableQuerierProcessor( this.tableQueue = tableQueue; this.cachedConnectionProvider = cachedConnectionProvider; this.maxRetriesPerQuerier = config.getInt(JdbcSourceConnectorConfig.QUERY_RETRIES_CONFIG); + this.shouldTrimSensitiveLogs = config.getBoolean( + JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); } public long process(RecordDestination destination) { @@ -131,38 +135,44 @@ private void processQuerier(RecordDestination destination, TableQu private void handleNonTransientException(RecordDestination destination, TableQuerier querier, SQLNonTransientException sqle) { + SQLException trimmedException = shouldTrimSensitiveLogs + ? LogUtil.trimSensitiveData(sqle) : sqle; + String querierLog = getQuerierLogString(querier); log.error("Non-transient SQL exception while running query for table: {}", - querier, sqle); + querierLog, trimmedException); resetAndRequeueHead(querier, true); // This task has failed, report failure to destination - destination.failWith(new ConnectException(sqle)); + destination.failWith(new ConnectException(trimmedException)); } private void handleSqlException(RecordDestination destination, TableQuerier querier, SQLException sqle) { + SQLException trimmedException = shouldTrimSensitiveLogs + ? LogUtil.trimSensitiveData(sqle) : sqle; + String querierLog = getQuerierLogString(querier); if (!RetryUtils.shouldRetry(sqle)) { log.error( "Non-retriable SQL exception while running query for table: {}. Failing task.", - querier, - sqle + querierLog, + trimmedException ); resetAndRequeueHead(querier, true); - destination.failWith(new ConnectException("Non-retriable SQL exception", sqle)); + destination.failWith(new ConnectException("Non-retriable SQL exception", trimmedException)); return; } log.error( "SQL exception while running query for table: {}. Attempting retry {} of {} attempts.", - querier, + querierLog, querier.getAttemptedRetryCount() + 1, maxRetriesPerQuerier, - sqle + trimmedException ); resetAndRequeueHead(querier, false); - if (maxRetriesPerQuerier > 0 - && querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) { - destination.failWith(new ConnectException("Failed to query table after retries", sqle)); + if (maxRetriesPerQuerier > 0 && querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) { + destination.failWith( + new ConnectException("Failed to query table after retries", trimmedException)); return; } querier.incrementRetryCount(); @@ -176,7 +186,8 @@ private void handleInterruptedException(TableQuerier querier, InterruptedExcepti private void handleThrowable(RecordDestination destination, TableQuerier querier, Throwable t) { - log.error("Failed to run query for table: {}", querier, t); + String querierLog = getQuerierLogString(querier); + log.error("Failed to run query for table: {}", querierLog, t); resetAndRequeueHead(querier, true); // This task has failed, report failure to destination destination.failWith(new ConnectException("Error while processing table querier", t)); @@ -199,7 +210,8 @@ public void shutdown() { } private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean resetOffset) { - log.debug("Resetting querier {}", expectedHead.toString()); + String querierLog = getQuerierLogString(expectedHead); + log.debug("Resetting querier {}", querierLog); TableQuerier removedQuerier = tableQueue.poll(); assert removedQuerier == expectedHead; expectedHead.reset(time.milliseconds(), resetOffset); @@ -208,5 +220,9 @@ private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean } } - + private String getQuerierLogString(TableQuerier querier) { + return shouldTrimSensitiveLogs + ? LogUtil.sensitiveLog(true, querier.toString()) + : querier.toString(); + } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index f90dbf44b..89dd14d83 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -77,14 +77,20 @@ public class TimestampIncrementingTableQuerier extends TableQuerier implements C private final long timestampDelay; private final ZoneId zoneId; - public TimestampIncrementingTableQuerier(DatabaseDialect dialect, QueryMode mode, String name, - String topicPrefix, - List timestampColumnNames, - String incrementingColumnName, - Map offsetMap, Long timestampDelay, - ZoneId zoneId, String suffix, - TimestampGranularity timestampGranularity) { - super(dialect, mode, name, topicPrefix, suffix); + public TimestampIncrementingTableQuerier( + JdbcSourceTaskConfig config, + DatabaseDialect dialect, + QueryMode mode, + String name, + String topicPrefix, + List timestampColumnNames, + String incrementingColumnName, + Map offsetMap, + Long timestampDelay, + ZoneId zoneId, + String suffix, + TimestampGranularity timestampGranularity) { + super(config, dialect, mode, name, topicPrefix, suffix); this.incrementingColumnName = incrementingColumnName; this.timestampColumnNames = timestampColumnNames != null ? timestampColumnNames : Collections.emptyList(); diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index 3d7baef61..c050fb5e3 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -54,7 +54,9 @@ public class TimestampTableQuerier extends TimestampIncrementingTableQuerier { private PendingRecord nextRecord; private Timestamp latestCommittableTimestamp; + @SuppressWarnings("checkstyle:ParameterNumber") public TimestampTableQuerier( + JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String name, @@ -67,6 +69,7 @@ public TimestampTableQuerier( TimestampGranularity timestampGranularity ) { super( + config, dialect, mode, name, diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index 2bbeeb4a6..94f543535 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -23,8 +23,10 @@ * error information to investigate incidents while at the same time avoid logging sensitive data. */ public class LogUtil { + private static final String REDACTED_VALUE = ""; + public static SQLException trimSensitiveData(SQLException e) { - return (SQLException) trimSensitiveData((Throwable)e); + return (SQLException) trimSensitiveData((Throwable) e); } public static Throwable trimSensitiveData(Throwable t) { @@ -36,9 +38,14 @@ public static Throwable trimSensitiveData(Throwable t) { if (!(t instanceof BatchUpdateException)) { // t is a SQLException, but not BatchUpdateException. - SQLException oldSqe = (SQLException)t; - SQLException newSqe = new SQLException(oldSqe.getLocalizedMessage()); + SQLException oldSqe = (SQLException) t; + SQLException newSqe = new SQLException( + REDACTED_VALUE, + oldSqe.getSQLState(), + oldSqe.getErrorCode() + ); newSqe.setNextException(trimSensitiveData(oldSqe.getNextException())); + newSqe.setStackTrace(oldSqe.getStackTrace()); return newSqe; } @@ -79,4 +86,11 @@ private static String getNonSensitiveErrorMessage(String errMsg) { String msg2 = errMsg.substring(errorStartIdx, errorEndIdx); return msg1 + msg2; } + + public static String sensitiveLog(boolean trimSensitiveLogsEnabled, Object msg) { + if (trimSensitiveLogsEnabled) { + return REDACTED_VALUE; + } + return String.valueOf(msg); + } } diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 2770b3a47..4e20b1f15 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; /** * Validation class for JDBC Source Connector configurations. @@ -38,6 +39,8 @@ public class JdbcSourceConnectorValidation { private static final Logger log = LoggerFactory.getLogger(JdbcSourceConnectorValidation.class); + private static final Pattern SELECT_STATEMENT_PATTERN = + Pattern.compile("(?is)^SELECT\\b"); protected JdbcSourceConnectorConfig config; protected Config validationResult; private final Map connectorConfigs; @@ -363,6 +366,16 @@ private boolean validateQueryConfigs() { return false; } + if (hasQuery + && !validateSelectStatement(query, JdbcSourceConnectorConfig.QUERY_CONFIG)) { + return false; + } + if (hasQueryMasked + && !validateSelectStatement( + queryMaskedValue, JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG)) { + return false; + } + return true; } @@ -503,6 +516,22 @@ protected void addConfigError(String configName, String errorMessage) { .ifPresent(cv -> cv.addErrorMessage(errorMessage)); } + /** Validate that provided query strings start with a SELECT statement. */ + private boolean validateSelectStatement(String statement, String configKey) { + String trimmedStatement = statement.trim(); + if (!SELECT_STATEMENT_PATTERN.matcher(trimmedStatement).find()) { + String msg = + String.format( + "Only SELECT statements are supported for '%s'. Please provide " + + "a statement that starts with SELECT.", + configKey); + addConfigError(configKey, msg); + log.error(msg); + return false; + } + return true; + } + /** * Determine whether any table filtering configurations are in use. */ diff --git a/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java index dba07c082..c5a25b250 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java @@ -40,13 +40,16 @@ public class TableQuerierTest { private static final Long TIMESTAMP_DELAY = 0l; private static final String QUERY = "SELECT * FROM name"; + JdbcSourceTaskConfig taskConfigMock; DatabaseDialect databaseDialectMock; + Connection connectionMock; @Before public void init() { + taskConfigMock = mock(JdbcSourceTaskConfig.class); databaseDialectMock = mock(DatabaseDialect.class); when(databaseDialectMock.parseTableIdentifier(Matchers.anyString())) .thenReturn(new TableId(null,null,TABLE_NAME)); @@ -61,7 +64,8 @@ public void init() @Test public void testTimestampIncrementingTableQuerierInTableModeWithSuffix() throws SQLException { TimestampIncrementingTableQuerier querier = new TimestampIncrementingTableQuerier( - databaseDialectMock, + taskConfigMock, + databaseDialectMock, QueryMode.TABLE, TABLE_NAME, null, @@ -82,7 +86,8 @@ public void testTimestampIncrementingTableQuerierInTableModeWithSuffix() throws @Test public void testTimestampIncrementingTableQuerierInQueryModeWithSuffix() throws SQLException { TimestampIncrementingTableQuerier querier = new TimestampIncrementingTableQuerier( - databaseDialectMock, + taskConfigMock, + databaseDialectMock, QueryMode.QUERY, QUERY, null, @@ -103,6 +108,7 @@ public void testTimestampIncrementingTableQuerierInQueryModeWithSuffix() throws @Test public void testBulkTableQuerierInTableModeWithSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( + taskConfigMock, databaseDialectMock, QueryMode.TABLE, TABLE_NAME, @@ -118,6 +124,7 @@ public void testBulkTableQuerierInTableModeWithSuffix() throws SQLException { @Test public void testBulkTableQuerierInQueryModeWithSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( + taskConfigMock, databaseDialectMock, QueryMode.QUERY, QUERY, @@ -133,6 +140,7 @@ public void testBulkTableQuerierInQueryModeWithSuffix() throws SQLException { @Test public void testBulkTableQuerierInQueryModeWithoutSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( + taskConfigMock, databaseDialectMock, QueryMode.QUERY, QUERY, diff --git a/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java index a3f69d70e..7d6b89fbd 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java @@ -72,9 +72,11 @@ public class TimestampIncrementingTableQuerierTest { @Mock private SchemaMapping schemaMapping; private DatabaseDialect dialect; + private JdbcSourceTaskConfig taskConfigMock; @Before public void setUp() { + taskConfigMock = mock(JdbcSourceTaskConfig.class); dialect = mock(DatabaseDialect.class); mockStatic(SchemaMapping.class); } @@ -90,6 +92,7 @@ private TimestampIncrementingTableQuerier querier( replay(dialect); return new TimestampIncrementingTableQuerier( + taskConfigMock, dialect, TableQuerier.QueryMode.TABLE, tableName, diff --git a/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java index 852e13778..11bc1090e 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java @@ -73,9 +73,11 @@ public class TimestampTableQuerierTest { @Mock private SchemaMapping schemaMapping; private DatabaseDialect dialect; + private JdbcSourceTaskConfig config; @Before public void setUp() { + config = mock(JdbcSourceTaskConfig.class); dialect = mock(DatabaseDialect.class); mockStatic(SchemaMapping.class); } @@ -88,6 +90,7 @@ private TimestampIncrementingTableQuerier querier(Timestamp initialTimestampOffs replay(dialect); return new TimestampTableQuerier( + config, dialect, TableQuerier.QueryMode.TABLE, tableName, diff --git a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java index 9a18c2cb7..d43791c8e 100644 --- a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java +++ b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java @@ -26,6 +26,7 @@ import static org.junit.Assert.assertEquals; public class LogUtilTest { + private static final String REDACTED = ""; @Test public void testNonSqlThrowable() { @@ -37,7 +38,8 @@ public void testNonSqlThrowable() { public void testSqlExceptionNoNested() { SQLException e = new SQLException("e"); SQLException trimmed = LogUtil.trimSensitiveData(e); - assertEqualsSQLException(e, trimmed); + SQLException expectedTrimmed = new SQLException(REDACTED); + assertEqualsSQLException(expectedTrimmed, trimmed); } @Test @@ -47,7 +49,9 @@ public void testSqlExceptionOneLevelNestedNonBatchUpdate() { e1.setNextException(e2); SQLException trimmed = LogUtil.trimSensitiveData(e1); - assertEqualsSQLException(e1, trimmed); + SQLException expectedTrimmed = new SQLException(REDACTED); + expectedTrimmed.setNextException(new SQLException(REDACTED)); + assertEqualsSQLException(expectedTrimmed, trimmed); } @Test @@ -59,7 +63,11 @@ public void testSqlExceptionTwoLevelNestedNonBatchUpdate() { e2.setNextException(e3); SQLException trimmed = LogUtil.trimSensitiveData(e1); - assertEqualsSQLException(e1, trimmed); + SQLException expectedTrimmed = new SQLException(REDACTED); + SQLException expectedNext = new SQLException(REDACTED); + expectedNext.setNextException(new SQLException(REDACTED)); + expectedTrimmed.setNextException(expectedNext); + assertEqualsSQLException(expectedTrimmed, trimmed); } @Test @@ -91,7 +99,9 @@ public void testSecondLevelNestedBatchUpdateNoSensitive() { e1.setNextException(e2); SQLException trimmed = LogUtil.trimSensitiveData(e1); - assertEqualsSQLException(e1, trimmed); + SQLException expectedTrimmed = new SQLException(REDACTED); + expectedTrimmed.setNextException(e2); + assertEqualsSQLException(expectedTrimmed, trimmed); } @Test @@ -103,7 +113,7 @@ public void testSecondLevelNestedBatchUpdateSensitive() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException("e1"); + SQLException expectedTrimmed = new SQLException(REDACTED); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\"): " + "ERROR: null value in column \"c4\" violates not-null constraint", new int[0]); @@ -121,7 +131,7 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoError() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException("e1"); + SQLException expectedTrimmed = new SQLException(REDACTED); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); @@ -138,7 +148,7 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoDetails() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException("e1"); + SQLException expectedTrimmed = new SQLException(REDACTED); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); @@ -159,7 +169,7 @@ public void testBatchExceptionWithChild() { e2.setNextException(p1); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException("e1"); + SQLException expectedTrimmed = new SQLException(REDACTED); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index b5b98e2ba..a837cc7a8 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -816,6 +816,18 @@ public void validate_withOnlyQueryMasked_noErrors() { assertNoErrors(); } + @Test + public void validate_withQueryStartingWithUpdate_setsError() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(QUERY_CONFIG, "UPDATE users SET active = false"); + + validate(); + + assertErrors(1); + assertErrors(QUERY_CONFIG, 1); + assertErrorMatches(QUERY_CONFIG, ".*Only SELECT statements are supported for 'query'.*"); + } + @Test public void validate_withQueryMaskedAndIncrementingColumn_noErrors() { props.put(MODE_CONFIG, MODE_INCREMENTING); From 34a08079016ef9108504dc90b584000ff1017e5b Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Fri, 28 Nov 2025 10:09:16 +0530 Subject: [PATCH 11/22] Remove RetryUtils and add condition to redact values only when query.masked has been set --- .../source/JdbcSourceConnectorConfig.java | 9 +- .../connect/jdbc/source/JdbcSourceTask.java | 4 +- .../connect/jdbc/source/TableQuerier.java | 3 +- .../jdbc/source/TableQuerierProcessor.java | 20 +-- .../connect/jdbc/util/RetryUtils.java | 146 ------------------ .../source/JdbcSourceConnectorConfigTest.java | 25 +++ .../connect/jdbc/util/RetryUtilsTest.java | 42 ----- 7 files changed, 42 insertions(+), 207 deletions(-) delete mode 100644 src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java delete mode 100644 src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 78e9f5d8c..f6b7e6eb4 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -409,7 +409,7 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { public static final String QUERY_RETRIES_DISPLAY = "Query Retry Attempts"; public static final String TRIM_SENSITIVE_LOG_ENABLED = "trim.sensitive.log"; - private static final String TRIM_SENSITIVE_LOG_ENABLED_DEFAULT = "true"; + private static final String TRIM_SENSITIVE_LOG_ENABLED_DEFAULT = "false"; /** * The properties that begin with this prefix will be used to configure a class, specified by @@ -1515,6 +1515,13 @@ public Optional getQuery() { return Optional.empty(); } + public boolean isQueryMasked() { + Password maskedQuery = getPassword(QUERY_MASKED_CONFIG); + return maskedQuery != null + && maskedQuery.value() != null + && !maskedQuery.value().isEmpty(); + } + public boolean modeUsesTimestampColumn() { String mode = getString(MODE_CONFIG); return Arrays.asList(MODE_TIMESTAMP, MODE_TIMESTAMP_INCREMENTING).contains(mode); diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index dee4ab709..da7aa3d3c 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -463,9 +463,7 @@ protected Map computeInitialOffset( String tableOrQuery, Map partitionOffset, ZoneId zoneId) { - Boolean shouldTrimSensitiveLogs = - config.getBoolean(JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); - if (shouldTrimSensitiveLogs) { + if (config.isQueryMasked()) { tableOrQuery = LogUtil.sensitiveLog(true, tableOrQuery); } if (!(partitionOffset == null)) { diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 5e1972cfd..61a8447da 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -79,8 +79,7 @@ public TableQuerier( this.lastUpdate = 0; this.suffix = suffix; this.attemptedRetries = 0; - this.shouldTrimSensitiveLogs = config.getBoolean( - JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); + this.shouldTrimSensitiveLogs = config.isQueryMasked(); } public long getLastUpdate() { diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index fdaf2037e..204fb8ce7 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -8,7 +8,6 @@ import io.confluent.connect.jdbc.util.CachedConnectionProvider; import io.confluent.connect.jdbc.util.LogUtil; import io.confluent.connect.jdbc.util.RecordDestination; -import io.confluent.connect.jdbc.util.RetryUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.SourceRecord; @@ -44,8 +43,7 @@ public TableQuerierProcessor( this.tableQueue = tableQueue; this.cachedConnectionProvider = cachedConnectionProvider; this.maxRetriesPerQuerier = config.getInt(JdbcSourceConnectorConfig.QUERY_RETRIES_CONFIG); - this.shouldTrimSensitiveLogs = config.getBoolean( - JdbcSourceConnectorConfig.TRIM_SENSITIVE_LOG_ENABLED); + this.shouldTrimSensitiveLogs = config.isQueryMasked(); } public long process(RecordDestination destination) { @@ -150,16 +148,12 @@ private void handleSqlException(RecordDestination destination, SQLException trimmedException = shouldTrimSensitiveLogs ? LogUtil.trimSensitiveData(sqle) : sqle; String querierLog = getQuerierLogString(querier); - if (!RetryUtils.shouldRetry(sqle)) { - log.error( - "Non-retriable SQL exception while running query for table: {}. Failing task.", - querierLog, - trimmedException - ); - resetAndRequeueHead(querier, true); - destination.failWith(new ConnectException("Non-retriable SQL exception", trimmedException)); - return; - } + log.error( + "Non-retriable SQL exception while running query for table: {}. Failing task.", + querierLog, + trimmedException); + resetAndRequeueHead(querier, true); + destination.failWith(new ConnectException("Non-retriable SQL exception", trimmedException)); log.error( "SQL exception while running query for table: {}. Attempting retry {} of {} attempts.", diff --git a/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java b/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java deleted file mode 100644 index 924cf6140..000000000 --- a/src/main/java/io/confluent/connect/jdbc/util/RetryUtils.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright 2024 Confluent Inc. - * - * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.connect.jdbc.util; - -import org.apache.commons.lang3.exception.ExceptionUtils; - -import java.sql.SQLException; -import java.sql.SQLNonTransientException; -import java.sql.SQLRecoverableException; -import java.sql.SQLTransientException; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Locale; -import java.util.Set; - -/** - * Utility class for determining whether SQL exceptions should be retried. - */ -public final class RetryUtils { - - private static final String CONNECTION_CLOSED_MSG = "CONNECTION IS CLOSED"; - - private static final Set RETRIABLE_SQLSTATE_PREFIXES = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - "08", // connection exception - "40", // transaction rollback - "55" // object not available / lock not available (PG) - ))); - - private static final Set RETRIABLE_SQLSTATES = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - "40001", // serialization failure - "40P01", // deadlock detected (PostgreSQL) - "55P03", // lock not available (PostgreSQL) - "57014", // query canceled (PostgreSQL, DB2) - "57033" // connection ended (DB2) - ))); - - private static final Set RETRIABLE_MESSAGE_TOKENS = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList( - "ORA-01466", // Oracle definition changed - "ORA-01284", // Oracle file cannot be opened - "ORA-03113", // end-of-file communication channel - "ORA-03114", // not connected to ORACLE - "ORA-12541", // TNS no listener - "ORA-12545", // TNS host or object not known - "DEADLOCK DETECTED", - "LOCK REQUEST TIMEOUT", // SQL Server - "SQLCODE=-911" // DB2 deadlock/timeout - ))); - - private static final Set RETRIABLE_ERROR_CODES = - Collections.synchronizedSet(new HashSet<>(Arrays.asList( - 1466, 1284, 3113, 3114, 4068, // Oracle - 1205, 1213, // MySQL and SQL Server deadlock/timeouts - -911 // DB2 deadlock/timeout when reported via error code - ))); - - public static void addRetriableErrorCodes(Collection errorCodes) { - if (errorCodes == null || errorCodes.isEmpty()) { - return; - } - RETRIABLE_ERROR_CODES.addAll(errorCodes); - } - - public static boolean shouldRetry(Throwable throwable) { - if (throwable == null) { - return false; - } - for (Throwable t : ExceptionUtils.getThrowableList(throwable)) { - if (t instanceof SQLException) { - for (SQLException se = (SQLException) t; se != null; se = se.getNextException()) { - if (canRetry(se)) { - return true; - } - if (cannotRetry(se)) { - return false; - } - } - } - } - return false; - } - - private static boolean canRetry(SQLException se) { - return isRecoverableAndNotClosed(se) - || (se instanceof SQLTransientException) - || hasRetriableErrorCodeOrState(se) - || messageHasRetriableTokens(se); - } - - private static boolean hasRetriableErrorCodeOrState(SQLException se) { - return RETRIABLE_ERROR_CODES.contains(se.getErrorCode()) || hasRetriableSqlState(se); - } - - private static boolean isRecoverableAndNotClosed(SQLException se) { - return se instanceof SQLRecoverableException - && (se.getMessage() == null - || !se.getMessage().toUpperCase(Locale.ROOT).contains(CONNECTION_CLOSED_MSG)); - } - - private static boolean cannotRetry(SQLException se) { - return se instanceof SQLNonTransientException; - } - - private static boolean hasRetriableSqlState(SQLException se) { - String sqlState = se.getSQLState(); - return sqlState != null - && !sqlState.isEmpty() - && (hasExactSqlStateMatch(sqlState) || hasSqlStatePrefixMatch(sqlState)); - } - - private static boolean hasExactSqlStateMatch(String sqlState) { - return RETRIABLE_SQLSTATES.contains(sqlState.toUpperCase(Locale.ROOT)); - } - - private static boolean hasSqlStatePrefixMatch(String sqlState) { - return sqlState.length() >= 2 - && RETRIABLE_SQLSTATE_PREFIXES.contains( - sqlState.substring(0, 2).toUpperCase(Locale.ROOT)); - } - - private static boolean messageHasRetriableTokens(SQLException se) { - String message = se.getMessage(); - return message != null - && !message.isEmpty() - && RETRIABLE_MESSAGE_TOKENS.stream() - .anyMatch(token -> message.toUpperCase(Locale.ROOT).contains(token)); - } -} - diff --git a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java index f90877b25..8201efac6 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfigTest.java @@ -483,6 +483,31 @@ public void testGetQueryStringReturnsEmptyWhenNeitherSet() { assertFalse(config.getQuery().isPresent()); } + @Test + public void testIsQueryMaskedTrueWhenMaskedQueryPresent() { + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1"); + props.put(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, "SELECT * FROM sensitive_table"); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + assertTrue(config.isQueryMasked()); + } + + @Test + public void testIsQueryMaskedFalseWhenMaskedQueryMissing() { + Map props = new HashMap<>(); + props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:postgresql://localhost:5432/testdb"); + props.put(JdbcSourceConnectorConfig.CONNECTION_USER_CONFIG, "testUser"); + props.put(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, "table1"); + + JdbcSourceConnectorConfig config = new JdbcSourceConnectorConfig(props); + + assertFalse(config.isQueryMasked()); + } + @Test public void testQueryMaskedSupportsComplexQueryWithMultipleJoins() { // Test that complex queries work fine with query.masked diff --git a/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java b/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java deleted file mode 100644 index 187e6cb86..000000000 --- a/src/test/java/io/confluent/connect/jdbc/util/RetryUtilsTest.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright 2024 Confluent Inc. - */ - -package io.confluent.connect.jdbc.util; - -import org.junit.Test; - -import java.sql.SQLException; -import java.sql.SQLNonTransientException; -import java.sql.SQLTransientException; -import java.util.Collections; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class RetryUtilsTest { - - @Test - public void shouldRetryOnTransientException() { - SQLException exception = new SQLTransientException("temporary issue"); - - assertTrue("Transient exceptions should be retried", RetryUtils.shouldRetry(exception)); - } - - @Test - public void shouldNotRetryOnNonTransientException() { - SQLException exception = new SQLNonTransientException("permanent failure"); - - assertFalse("Non transient exceptions must not be retried", RetryUtils.shouldRetry(exception)); - } - - @Test - public void shouldRetryWhenErrorCodeRegistered() { - int customErrorCode = 987654; - RetryUtils.addRetriableErrorCodes(Collections.singleton(customErrorCode)); - SQLException exception = new SQLException("custom error", "S0001", customErrorCode); - - assertTrue("Registered error codes should be retried", RetryUtils.shouldRetry(exception)); - } -} - From d1dfc62d3e1711f3688099df55daf876db189dc1 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Mon, 1 Dec 2025 13:17:19 +0530 Subject: [PATCH 12/22] Minor changes in comments and code --- .../connect/jdbc/source/JdbcSourceConnectorConfig.java | 6 +++++- .../connect/jdbc/source/TableQuerierProcessor.java | 4 +--- src/main/java/io/confluent/connect/jdbc/util/LogUtil.java | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index f6b7e6eb4..6acb04af3 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -431,6 +431,11 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { private static final EnumRecommender QUOTE_METHOD_RECOMMENDER = EnumRecommender.in(QuoteMethod.values()); + /** + * A recommender that hides configuration parameters from being displayed in config list + * This is used for sensitive or internal configurations that should not be exposed + * to users through standard configuration interfaces. + */ private static final ConfigDef.Recommender HIDDEN_RECOMMENDER = new ConfigDef.Recommender() { @Override @@ -1497,7 +1502,6 @@ public List incrementingColMappingRegexes() { /** * Get the query string from either query or query.masked config. - * Prioritizes query.masked over query if both are set (though validation should prevent this). * * @return Optional containing the query string if present, empty Optional otherwise. */ diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 204fb8ce7..6acf66213 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -215,8 +215,6 @@ private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean } private String getQuerierLogString(TableQuerier querier) { - return shouldTrimSensitiveLogs - ? LogUtil.sensitiveLog(true, querier.toString()) - : querier.toString(); + return LogUtil.sensitiveLog(shouldTrimSensitiveLogs, querier.toString()); } } diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index 94f543535..8d3878bfd 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -87,7 +87,7 @@ private static String getNonSensitiveErrorMessage(String errMsg) { return msg1 + msg2; } - public static String sensitiveLog(boolean trimSensitiveLogsEnabled, Object msg) { + public static String sensitiveLog(boolean trimSensitiveLogsEnabled, String msg) { if (trimSensitiveLogsEnabled) { return REDACTED_VALUE; } From 226373d525aef826320b92e4985cd85a04e22203 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Mon, 1 Dec 2025 15:13:39 +0530 Subject: [PATCH 13/22] Removed config object in Tablequerier and masked only query value in querier object --- .../connect/jdbc/source/BulkTableQuerier.java | 12 ++++---- .../connect/jdbc/source/JdbcSourceTask.java | 17 +++++------ .../connect/jdbc/source/TableQuerier.java | 12 ++++---- .../jdbc/source/TableQuerierProcessor.java | 28 ++++--------------- .../TimestampIncrementingTableQuerier.java | 17 +++++------ .../jdbc/source/TimestampTableQuerier.java | 11 ++++---- .../connect/jdbc/source/TableQuerierTest.java | 22 +++++++-------- ...TimestampIncrementingTableQuerierTest.java | 6 ++-- .../source/TimestampTableQuerierTest.java | 6 ++-- 9 files changed, 56 insertions(+), 75 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index af0c2c936..260fefc8c 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -40,14 +40,14 @@ public class BulkTableQuerier extends TableQuerier { private static final Logger log = LoggerFactory.getLogger(BulkTableQuerier.class); public BulkTableQuerier( - JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String name, String topicPrefix, - String suffix + String suffix, + Boolean isQueryMasked ) { - super(config, dialect, mode, name, topicPrefix, suffix); + super(dialect, mode, name, topicPrefix, suffix, isQueryMasked); } @Override @@ -69,9 +69,10 @@ protected void createPreparedStatement(Connection db) throws SQLException { addSuffixIfPresent(builder); String queryStr = builder.toString(); + String queryForLog = getQuerierLogString(queryStr); recordQuery(queryStr); - log.trace("{} prepared SQL query: {}", this, queryStr); + log.trace("{} prepared SQL query: {}", this, queryForLog); stmt = dialect.createPreparedStatement(db, queryStr); } @@ -117,7 +118,8 @@ public SourceRecord extractRecord() throws SQLException { @Override public String toString() { - return "BulkTableQuerier{" + "table='" + tableId + '\'' + ", query='" + query + '\'' + String queryForLog = getQuerierLogString(query); + return "BulkTableQuerier{" + "table='" + tableId + '\'' + ", query='" + queryForLog + '\'' + ", topicPrefix='" + topicPrefix + '\'' + '}'; } diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index da7aa3d3c..435fc0eba 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -225,6 +225,7 @@ public void start(Map properties) { = config.getBoolean(JdbcSourceTaskConfig.VALIDATE_NON_NULL_CONFIG); ZoneId zoneId = config.zoneId(); String suffix = config.getString(JdbcSourceTaskConfig.QUERY_SUFFIX_CONFIG).trim(); + Boolean queryMasked = config.isQueryMasked(); if (queryMode.equals(TableQuerier.QueryMode.TABLE)) { validateColumnsExist( @@ -288,18 +289,17 @@ mode, getIncrementingColumn(tables.get(0)), if (mode.equals(JdbcSourceTaskConfig.MODE_BULK)) { tableQueue.add( new BulkTableQuerier( - config, dialect, queryMode, tableOrQuery, topicPrefix, - suffix + suffix, + queryMasked ) ); } else if (mode.equals(JdbcSourceTaskConfig.MODE_INCREMENTING)) { tableQueue.add( new TimestampIncrementingTableQuerier( - config, dialect, queryMode, tableOrQuery, @@ -310,13 +310,13 @@ mode, getIncrementingColumn(tables.get(0)), timestampDelayInterval, zoneId, suffix, - timestampGranularity + timestampGranularity, + queryMasked ) ); } else if (mode.equals(JdbcSourceTaskConfig.MODE_TIMESTAMP)) { tableQueue.add( new TimestampTableQuerier( - config, dialect, queryMode, tableOrQuery, @@ -326,13 +326,13 @@ mode, getIncrementingColumn(tables.get(0)), timestampDelayInterval, zoneId, suffix, - timestampGranularity + timestampGranularity, + queryMasked ) ); } else if (mode.endsWith(JdbcSourceTaskConfig.MODE_TIMESTAMP_INCREMENTING)) { tableQueue.add( new TimestampIncrementingTableQuerier( - config, dialect, queryMode, tableOrQuery, @@ -343,7 +343,8 @@ mode, getIncrementingColumn(tables.get(0)), timestampDelayInterval, zoneId, suffix, - timestampGranularity + timestampGranularity, + queryMasked ) ); } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 61a8447da..28e386f2a 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -42,7 +42,6 @@ public enum QueryMode { private final Logger log = LoggerFactory.getLogger(TableQuerier.class); - protected final JdbcSourceTaskConfig config; protected final DatabaseDialect dialect; protected final QueryMode mode; protected final String query; @@ -63,14 +62,13 @@ public enum QueryMode { private int attemptedRetries; public TableQuerier( - JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String nameOrQuery, String topicPrefix, - String suffix + String suffix, + Boolean isQueryMasked ) { - this.config = config; this.dialect = dialect; this.mode = mode; this.tableId = mode.equals(QueryMode.TABLE) ? dialect.parseTableIdentifier(nameOrQuery) : null; @@ -79,7 +77,7 @@ public TableQuerier( this.lastUpdate = 0; this.suffix = suffix; this.attemptedRetries = 0; - this.shouldTrimSensitiveLogs = config.isQueryMasked(); + this.shouldTrimSensitiveLogs = isQueryMasked; } public long getLastUpdate() { @@ -191,9 +189,9 @@ protected void recordQuery(String query) { } } - private String getQuerierLogString(String query) { + protected String getQuerierLogString(String query) { return shouldTrimSensitiveLogs - ? LogUtil.sensitiveLog(true, query) + ? LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) : query; } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 6acf66213..b0f385d10 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -135,9 +135,8 @@ private void handleNonTransientException(RecordDestination destina TableQuerier querier, SQLNonTransientException sqle) { SQLException trimmedException = shouldTrimSensitiveLogs ? LogUtil.trimSensitiveData(sqle) : sqle; - String querierLog = getQuerierLogString(querier); log.error("Non-transient SQL exception while running query for table: {}", - querierLog, trimmedException); + querier, trimmedException); resetAndRequeueHead(querier, true); // This task has failed, report failure to destination destination.failWith(new ConnectException(trimmedException)); @@ -147,21 +146,12 @@ private void handleSqlException(RecordDestination destination, TableQuerier querier, SQLException sqle) { SQLException trimmedException = shouldTrimSensitiveLogs ? LogUtil.trimSensitiveData(sqle) : sqle; - String querierLog = getQuerierLogString(querier); log.error( - "Non-retriable SQL exception while running query for table: {}. Failing task.", - querierLog, - trimmedException); - resetAndRequeueHead(querier, true); - destination.failWith(new ConnectException("Non-retriable SQL exception", trimmedException)); - - log.error( - "SQL exception while running query for table: {}. Attempting retry {} of {} attempts.", - querierLog, + "SQL exception while running query for table: {}." + " Attempting retry {} of {} attempts.", + querier, querier.getAttemptedRetryCount() + 1, maxRetriesPerQuerier, - trimmedException - ); + trimmedException); resetAndRequeueHead(querier, false); if (maxRetriesPerQuerier > 0 && querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) { @@ -180,8 +170,7 @@ private void handleInterruptedException(TableQuerier querier, InterruptedExcepti private void handleThrowable(RecordDestination destination, TableQuerier querier, Throwable t) { - String querierLog = getQuerierLogString(querier); - log.error("Failed to run query for table: {}", querierLog, t); + log.error("Failed to run query for table: {}", querier, t); resetAndRequeueHead(querier, true); // This task has failed, report failure to destination destination.failWith(new ConnectException("Error while processing table querier", t)); @@ -204,8 +193,7 @@ public void shutdown() { } private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean resetOffset) { - String querierLog = getQuerierLogString(expectedHead); - log.debug("Resetting querier {}", querierLog); + log.debug("Resetting querier {}", expectedHead); TableQuerier removedQuerier = tableQueue.poll(); assert removedQuerier == expectedHead; expectedHead.reset(time.milliseconds(), resetOffset); @@ -213,8 +201,4 @@ private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean tableQueue.add(expectedHead); } } - - private String getQuerierLogString(TableQuerier querier) { - return LogUtil.sensitiveLog(shouldTrimSensitiveLogs, querier.toString()); - } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 89dd14d83..95028db08 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -15,6 +15,10 @@ package io.confluent.connect.jdbc.source; +import io.confluent.connect.jdbc.util.DateTimeUtils; +import io.confluent.connect.jdbc.util.ExpressionBuilder; +import io.confluent.connect.jdbc.util.ColumnDefinition; +import io.confluent.connect.jdbc.util.ColumnId; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; @@ -38,10 +42,6 @@ import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TimestampGranularity; import io.confluent.connect.jdbc.source.SchemaMapping.FieldSetter; import io.confluent.connect.jdbc.source.TimestampIncrementingCriteria.CriteriaValues; -import io.confluent.connect.jdbc.util.ColumnDefinition; -import io.confluent.connect.jdbc.util.ColumnId; -import io.confluent.connect.jdbc.util.DateTimeUtils; -import io.confluent.connect.jdbc.util.ExpressionBuilder; /** *

@@ -78,7 +78,6 @@ public class TimestampIncrementingTableQuerier extends TableQuerier implements C private final ZoneId zoneId; public TimestampIncrementingTableQuerier( - JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String name, @@ -89,8 +88,9 @@ public TimestampIncrementingTableQuerier( Long timestampDelay, ZoneId zoneId, String suffix, - TimestampGranularity timestampGranularity) { - super(config, dialect, mode, name, topicPrefix, suffix); + TimestampGranularity timestampGranularity, + Boolean isQueryMasked) { + super(dialect, mode, name, topicPrefix, suffix, isQueryMasked); this.incrementingColumnName = incrementingColumnName; this.timestampColumnNames = timestampColumnNames != null ? timestampColumnNames : Collections.emptyList(); @@ -275,9 +275,10 @@ public Long lastIncrementedValue() { @Override public String toString() { + String queryForLog = getQuerierLogString(query); return "TimestampIncrementingTableQuerier{" + "table=" + tableId - + ", query='" + query + '\'' + + ", query='" + queryForLog + '\'' + ", topicPrefix='" + topicPrefix + '\'' + ", incrementingColumn='" + (incrementingColumnName != null ? incrementingColumnName diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index c050fb5e3..d77cde59a 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -56,7 +56,6 @@ public class TimestampTableQuerier extends TimestampIncrementingTableQuerier { @SuppressWarnings("checkstyle:ParameterNumber") public TimestampTableQuerier( - JdbcSourceTaskConfig config, DatabaseDialect dialect, QueryMode mode, String name, @@ -66,10 +65,10 @@ public TimestampTableQuerier( Long timestampDelay, ZoneId zoneId, String suffix, - TimestampGranularity timestampGranularity + TimestampGranularity timestampGranularity, + Boolean isQueryMasked ) { super( - config, dialect, mode, name, @@ -80,7 +79,8 @@ public TimestampTableQuerier( timestampDelay, zoneId, suffix, - timestampGranularity + timestampGranularity, + isQueryMasked ); this.latestCommittableTimestamp = this.offset.getTimestampOffset(); @@ -179,9 +179,10 @@ public void reset(long now, boolean resetOffset) { @Override public String toString() { + String queryForLog = getQuerierLogString(query); return "TimestampTableQuerier{" + "table=" + tableId - + ", query='" + query + '\'' + + ", query='" + queryForLog + '\'' + ", topicPrefix='" + topicPrefix + '\'' + ", timestampColumns=" + timestampColumnNames + '}'; diff --git a/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java index c5a25b250..94af95cef 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TableQuerierTest.java @@ -40,7 +40,6 @@ public class TableQuerierTest { private static final Long TIMESTAMP_DELAY = 0l; private static final String QUERY = "SELECT * FROM name"; - JdbcSourceTaskConfig taskConfigMock; DatabaseDialect databaseDialectMock; @@ -49,7 +48,6 @@ public class TableQuerierTest { @Before public void init() { - taskConfigMock = mock(JdbcSourceTaskConfig.class); databaseDialectMock = mock(DatabaseDialect.class); when(databaseDialectMock.parseTableIdentifier(Matchers.anyString())) .thenReturn(new TableId(null,null,TABLE_NAME)); @@ -64,7 +62,6 @@ public void init() @Test public void testTimestampIncrementingTableQuerierInTableModeWithSuffix() throws SQLException { TimestampIncrementingTableQuerier querier = new TimestampIncrementingTableQuerier( - taskConfigMock, databaseDialectMock, QueryMode.TABLE, TABLE_NAME, @@ -75,7 +72,8 @@ public void testTimestampIncrementingTableQuerierInTableModeWithSuffix() throws TIMESTAMP_DELAY, null, SUFFIX, - JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL + JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL, + false ); querier.createPreparedStatement(connectionMock); @@ -86,7 +84,6 @@ public void testTimestampIncrementingTableQuerierInTableModeWithSuffix() throws @Test public void testTimestampIncrementingTableQuerierInQueryModeWithSuffix() throws SQLException { TimestampIncrementingTableQuerier querier = new TimestampIncrementingTableQuerier( - taskConfigMock, databaseDialectMock, QueryMode.QUERY, QUERY, @@ -97,7 +94,8 @@ public void testTimestampIncrementingTableQuerierInQueryModeWithSuffix() throws TIMESTAMP_DELAY, null, SUFFIX, - JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL + JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL, + false ); querier.createPreparedStatement(connectionMock); @@ -108,12 +106,12 @@ public void testTimestampIncrementingTableQuerierInQueryModeWithSuffix() throws @Test public void testBulkTableQuerierInTableModeWithSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( - taskConfigMock, databaseDialectMock, QueryMode.TABLE, TABLE_NAME, null, - SUFFIX + SUFFIX, + false ); querier.createPreparedStatement(connectionMock); @@ -124,12 +122,12 @@ public void testBulkTableQuerierInTableModeWithSuffix() throws SQLException { @Test public void testBulkTableQuerierInQueryModeWithSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( - taskConfigMock, databaseDialectMock, QueryMode.QUERY, QUERY, null, - SUFFIX + SUFFIX, + false ); querier.createPreparedStatement(connectionMock); @@ -140,12 +138,12 @@ public void testBulkTableQuerierInQueryModeWithSuffix() throws SQLException { @Test public void testBulkTableQuerierInQueryModeWithoutSuffix() throws SQLException { BulkTableQuerier querier = new BulkTableQuerier( - taskConfigMock, databaseDialectMock, QueryMode.QUERY, QUERY, null, - "" /* default value */ + "", /* default value */ + false ); querier.createPreparedStatement(connectionMock); diff --git a/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java index 7d6b89fbd..efe842cd3 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerierTest.java @@ -72,11 +72,9 @@ public class TimestampIncrementingTableQuerierTest { @Mock private SchemaMapping schemaMapping; private DatabaseDialect dialect; - private JdbcSourceTaskConfig taskConfigMock; @Before public void setUp() { - taskConfigMock = mock(JdbcSourceTaskConfig.class); dialect = mock(DatabaseDialect.class); mockStatic(SchemaMapping.class); } @@ -92,7 +90,6 @@ private TimestampIncrementingTableQuerier querier( replay(dialect); return new TimestampIncrementingTableQuerier( - taskConfigMock, dialect, TableQuerier.QueryMode.TABLE, tableName, @@ -103,7 +100,8 @@ private TimestampIncrementingTableQuerier querier( 10211197100L, // Timestamp delay ZoneId.of("UTC"), "", - JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL + JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL, + false ); } diff --git a/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java b/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java index 11bc1090e..a26f0374c 100644 --- a/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java +++ b/src/test/java/io/confluent/connect/jdbc/source/TimestampTableQuerierTest.java @@ -73,11 +73,9 @@ public class TimestampTableQuerierTest { @Mock private SchemaMapping schemaMapping; private DatabaseDialect dialect; - private JdbcSourceTaskConfig config; @Before public void setUp() { - config = mock(JdbcSourceTaskConfig.class); dialect = mock(DatabaseDialect.class); mockStatic(SchemaMapping.class); } @@ -90,7 +88,6 @@ private TimestampIncrementingTableQuerier querier(Timestamp initialTimestampOffs replay(dialect); return new TimestampTableQuerier( - config, dialect, TableQuerier.QueryMode.TABLE, tableName, @@ -100,7 +97,8 @@ private TimestampIncrementingTableQuerier querier(Timestamp initialTimestampOffs 10211197100L, // Timestamp delay ZoneId.of("UTC"), "", - JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL + JdbcSourceConnectorConfig.TimestampGranularity.CONNECT_LOGICAL, + false ); } From ad7df13c189411263d893e54412ed119f5fb1ce2 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 2 Dec 2025 09:58:30 +0530 Subject: [PATCH 14/22] Refactored code and added few tests for coverage --- pom.xml | 5 --- .../connect/jdbc/source/BulkTableQuerier.java | 5 +-- .../connect/jdbc/source/TableQuerier.java | 10 ++---- .../TimestampIncrementingTableQuerier.java | 7 ++-- .../jdbc/source/TimestampTableQuerier.java | 3 +- .../connect/jdbc/util/LogUtilTest.java | 14 ++++++++ .../JdbcSourceConnectorValidationTest.java | 33 +++++++++++++++++++ 7 files changed, 56 insertions(+), 21 deletions(-) diff --git a/pom.xml b/pom.xml index da16e6db8..f0b9187f4 100644 --- a/pom.xml +++ b/pom.xml @@ -110,11 +110,6 @@ com.google.re2j re2j 1.7 - - - org.apache.commons - commons-lang3 - 3.12.0 diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index 260fefc8c..71911901b 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -15,6 +15,7 @@ package io.confluent.connect.jdbc.source; +import io.confluent.connect.jdbc.util.LogUtil; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; @@ -69,7 +70,7 @@ protected void createPreparedStatement(Connection db) throws SQLException { addSuffixIfPresent(builder); String queryStr = builder.toString(); - String queryForLog = getQuerierLogString(queryStr); + String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); recordQuery(queryStr); log.trace("{} prepared SQL query: {}", this, queryForLog); @@ -118,7 +119,7 @@ public SourceRecord extractRecord() throws SQLException { @Override public String toString() { - String queryForLog = getQuerierLogString(query); + String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); return "BulkTableQuerier{" + "table='" + tableId + '\'' + ", query='" + queryForLog + '\'' + ", topicPrefix='" + topicPrefix + '\'' + '}'; } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 28e386f2a..6138355a5 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -57,7 +57,7 @@ public enum QueryMode { protected ResultSet resultSet; protected SchemaMapping schemaMapping; private String loggedQueryString; - private final Boolean shouldTrimSensitiveLogs; + protected final Boolean shouldTrimSensitiveLogs; private int attemptedRetries; @@ -182,19 +182,13 @@ protected void addSuffixIfPresent(ExpressionBuilder builder) { protected void recordQuery(String query) { if (query != null && !query.equals(loggedQueryString)) { - String querierLog = getQuerierLogString(query); + String querierLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); // For usability, log the statement at INFO level only when it changes log.info("Begin using SQL query: {}", querierLog); loggedQueryString = query; } } - protected String getQuerierLogString(String query) { - return shouldTrimSensitiveLogs - ? LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) - : query; - } - @Override public int compareTo(TableQuerier other) { if (this.lastUpdate < other.lastUpdate) { diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 95028db08..215c44846 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -15,10 +15,7 @@ package io.confluent.connect.jdbc.source; -import io.confluent.connect.jdbc.util.DateTimeUtils; -import io.confluent.connect.jdbc.util.ExpressionBuilder; -import io.confluent.connect.jdbc.util.ColumnDefinition; -import io.confluent.connect.jdbc.util.ColumnId; +import io.confluent.connect.jdbc.util.*; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; @@ -275,7 +272,7 @@ public Long lastIncrementedValue() { @Override public String toString() { - String queryForLog = getQuerierLogString(query); + String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); return "TimestampIncrementingTableQuerier{" + "table=" + tableId + ", query='" + queryForLog + '\'' diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index d77cde59a..d8197660a 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -17,6 +17,7 @@ import java.time.ZoneId; +import io.confluent.connect.jdbc.util.LogUtil; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; @@ -179,7 +180,7 @@ public void reset(long now, boolean resetOffset) { @Override public String toString() { - String queryForLog = getQuerierLogString(query); + String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); return "TimestampTableQuerier{" + "table=" + tableId + ", query='" + queryForLog + '\'' diff --git a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java index d43791c8e..8e792d842 100644 --- a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java +++ b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java @@ -140,6 +140,20 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoError() { assertEqualsSQLException(expectedTrimmed, actualTrimmed); } + @Test + public void testSensitiveLogWithTrimEnabled() { + String sensitiveMessage = "SELECT * FROM users WHERE password='secret123'"; + String result = LogUtil.sensitiveLog(true, sensitiveMessage); + assertEquals(REDACTED, result); + } + + @Test + public void testSensitiveLogWithTrimDisabled() { + String message = "SELECT * FROM users WHERE id=1"; + String result = LogUtil.sensitiveLog(false, message); + assertEquals(message, result); + } + @Test public void testSecondLevelNestedBatchUpdateSensitiveNoDetails() { SQLException e1 = new SQLException("e1"); diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index a837cc7a8..6802c2a7d 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -868,5 +868,38 @@ public void validate_withBothQueryAndQueryMaskedEmpty_noErrors() { assertNoErrors(); } + + @Test + public void validate_withQueryAndTableFilteringConfigs_setsError() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_INCLUDE_LIST_CONFIG, "database.schema.table.*"); + props.put(QUERY_CONFIG, "SELECT * FROM users"); + + validate(); + + assertErrors(3); + assertErrors(QUERY_CONFIG, 1); + assertErrors(QUERY_MASKED_CONFIG, 1); + assertErrors(TABLE_INCLUDE_LIST_CONFIG, 1); + assertErrorMatches( + QUERY_CONFIG, + "Do not specify table filtering configs with 'query'" + ); + } + + @Test + public void validate_withQueryMaskedStartingWithUpdate_setsError() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(QUERY_MASKED_CONFIG, "UPDATE users SET active = false"); + + validate(); + + assertErrors(1); + assertErrors(QUERY_MASKED_CONFIG, 1); + assertErrorMatches( + QUERY_MASKED_CONFIG, + "Only SELECT statements are supported for 'query.masked'" + ); + } } From aed3f9010ced1a246de6c553d02792e59179b84c Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 2 Dec 2025 10:38:45 +0530 Subject: [PATCH 15/22] Fix checkstyle errors --- .../jdbc/source/JdbcSourceConnectorConfig.java | 6 +++--- .../connect/jdbc/source/JdbcSourceTask.java | 16 ++++++++-------- .../TimestampIncrementingTableQuerier.java | 6 +++++- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 6acb04af3..32090fdcd 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -20,14 +20,14 @@ import java.time.Duration; import java.time.ZoneId; import java.util.Arrays; -import java.util.HashSet; import java.util.HashMap; -import java.util.Map; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.Locale; +import java.util.Map; import java.util.Set; import java.util.Optional; -import java.util.Locale; import java.util.concurrent.atomic.AtomicReference; import com.microsoft.sqlserver.jdbc.SQLServerConnection; diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 435fc0eba..731218aa3 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -15,14 +15,6 @@ package io.confluent.connect.jdbc.source; -import io.confluent.connect.jdbc.util.LogUtil; -import io.confluent.connect.jdbc.util.TableId; -import io.confluent.connect.jdbc.util.TableCollectionUtils; -import io.confluent.connect.jdbc.util.ColumnDefinition; -import io.confluent.connect.jdbc.util.ColumnId; -import io.confluent.connect.jdbc.util.CachedConnectionProvider; -import io.confluent.connect.jdbc.util.RecordQueue; -import io.confluent.connect.jdbc.util.Version; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; @@ -51,6 +43,14 @@ import io.confluent.connect.jdbc.dialect.DatabaseDialect; import io.confluent.connect.jdbc.dialect.DatabaseDialects; +import io.confluent.connect.jdbc.util.CachedConnectionProvider; +import io.confluent.connect.jdbc.util.ColumnDefinition; +import io.confluent.connect.jdbc.util.ColumnId; +import io.confluent.connect.jdbc.util.RecordQueue; +import io.confluent.connect.jdbc.util.TableCollectionUtils; +import io.confluent.connect.jdbc.util.TableId; +import io.confluent.connect.jdbc.util.Version; +import io.confluent.connect.jdbc.util.LogUtil; import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TransactionIsolationMode; /** diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 215c44846..2744ed6b7 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -15,7 +15,6 @@ package io.confluent.connect.jdbc.source; -import io.confluent.connect.jdbc.util.*; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; @@ -39,6 +38,11 @@ import io.confluent.connect.jdbc.source.JdbcSourceConnectorConfig.TimestampGranularity; import io.confluent.connect.jdbc.source.SchemaMapping.FieldSetter; import io.confluent.connect.jdbc.source.TimestampIncrementingCriteria.CriteriaValues; +import io.confluent.connect.jdbc.util.ColumnDefinition; +import io.confluent.connect.jdbc.util.ColumnId; +import io.confluent.connect.jdbc.util.DateTimeUtils; +import io.confluent.connect.jdbc.util.ExpressionBuilder; +import io.confluent.connect.jdbc.util.LogUtil; /** *

From c0922d332f7fc7821f485864f1dbd5f42c55f29d Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 2 Dec 2025 15:56:19 +0530 Subject: [PATCH 16/22] Extended redactSensitiveData logging util and Minor code changes --- .../connect/jdbc/source/BulkTableQuerier.java | 18 ++++-- .../connect/jdbc/source/TableQuerier.java | 3 +- .../TimestampIncrementingTableQuerier.java | 23 +++++--- .../jdbc/source/TimestampTableQuerier.java | 15 +++-- .../confluent/connect/jdbc/util/LogUtil.java | 43 +++++++++++--- .../JdbcSourceConnectorValidation.java | 7 +-- .../connect/jdbc/util/LogUtilTest.java | 57 ++++++++++++++---- .../JdbcSourceConnectorValidationTest.java | 58 +++++++++---------- 8 files changed, 151 insertions(+), 73 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index 71911901b..d28ac5491 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -70,10 +70,10 @@ protected void createPreparedStatement(Connection db) throws SQLException { addSuffixIfPresent(builder); String queryStr = builder.toString(); - String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); recordQuery(queryStr); - log.trace("{} prepared SQL query: {}", this, queryForLog); + log.trace( + "{} prepared SQL query: {}", this, LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query)); stmt = dialect.createPreparedStatement(db, queryStr); } @@ -119,9 +119,17 @@ public SourceRecord extractRecord() throws SQLException { @Override public String toString() { - String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); - return "BulkTableQuerier{" + "table='" + tableId + '\'' + ", query='" + queryForLog + '\'' - + ", topicPrefix='" + topicPrefix + '\'' + '}'; + return "BulkTableQuerier{" + + "table='" + + tableId + + '\'' + + ", query='" + + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + '\'' + + ", topicPrefix='" + + topicPrefix + + '\'' + + '}'; } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 6138355a5..508fe1d5d 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -182,9 +182,8 @@ protected void addSuffixIfPresent(ExpressionBuilder builder) { protected void recordQuery(String query) { if (query != null && !query.equals(loggedQueryString)) { - String querierLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); // For usability, log the statement at INFO level only when it changes - log.info("Begin using SQL query: {}", querierLog); + log.info("Begin using SQL query: {}", LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query)); loggedQueryString = query; } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 2744ed6b7..752720f83 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -276,15 +276,20 @@ public Long lastIncrementedValue() { @Override public String toString() { - String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); return "TimestampIncrementingTableQuerier{" - + "table=" + tableId - + ", query='" + queryForLog + '\'' - + ", topicPrefix='" + topicPrefix + '\'' - + ", incrementingColumn='" + (incrementingColumnName != null - ? incrementingColumnName - : "") + '\'' - + ", timestampColumns=" + timestampColumnNames - + '}'; + + "table=" + + tableId + + ", query='" + + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + '\'' + + ", topicPrefix='" + + topicPrefix + + '\'' + + ", incrementingColumn='" + + (incrementingColumnName != null ? incrementingColumnName : "") + + '\'' + + ", timestampColumns=" + + timestampColumnNames + + '}'; } } \ No newline at end of file diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index d8197660a..1cdf09171 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -180,12 +180,17 @@ public void reset(long now, boolean resetOffset) { @Override public String toString() { - String queryForLog = LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query); return "TimestampTableQuerier{" - + "table=" + tableId - + ", query='" + queryForLog + '\'' - + ", topicPrefix='" + topicPrefix + '\'' - + ", timestampColumns=" + timestampColumnNames + + "table=" + + tableId + + ", query='" + + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + '\'' + + ", topicPrefix='" + + topicPrefix + + '\'' + + ", timestampColumns=" + + timestampColumnNames + '}'; } diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index 8d3878bfd..55553058f 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -38,14 +38,9 @@ public static Throwable trimSensitiveData(Throwable t) { if (!(t instanceof BatchUpdateException)) { // t is a SQLException, but not BatchUpdateException. - SQLException oldSqe = (SQLException) t; - SQLException newSqe = new SQLException( - REDACTED_VALUE, - oldSqe.getSQLState(), - oldSqe.getErrorCode() - ); + SQLException oldSqe = (SQLException)t; + SQLException newSqe = new SQLException(oldSqe.getLocalizedMessage()); newSqe.setNextException(trimSensitiveData(oldSqe.getNextException())); - newSqe.setStackTrace(oldSqe.getStackTrace()); return newSqe; } @@ -55,6 +50,40 @@ public static Throwable trimSensitiveData(Throwable t) { e.getUpdateCounts()); } + public static SQLException redactSensitiveData(SQLException e) { + return (SQLException) redactSensitiveData((Throwable) e); + } + + public static Throwable redactSensitiveData(Throwable t) { + if (!(t instanceof SQLException)) { + return t; + } + + if (!(t instanceof BatchUpdateException)) { + // t is a SQLException, but not BatchUpdateException. + SQLException oldSqlException = (SQLException) t; + SQLException newSqlException = + new SQLException( + REDACTED_VALUE, oldSqlException.getSQLState(), oldSqlException.getErrorCode()); + newSqlException.setNextException(redactSensitiveData(oldSqlException.getNextException())); + newSqlException.setStackTrace(oldSqlException.getStackTrace()); + return newSqlException; + } + + // At this point t is BatchUpdateException; redact its message too. + BatchUpdateException oldBatchUpdateException = (BatchUpdateException) t; + BatchUpdateException newBatchUpdateException = + new BatchUpdateException( + REDACTED_VALUE, + oldBatchUpdateException.getSQLState(), + oldBatchUpdateException.getErrorCode(), + oldBatchUpdateException.getUpdateCounts()); + newBatchUpdateException.setNextException( + redactSensitiveData(oldBatchUpdateException.getNextException())); + newBatchUpdateException.setStackTrace(oldBatchUpdateException.getStackTrace()); + return newBatchUpdateException; + } + // This implementation assumes it to be Postgres, see toString() of ServerErrorMessage.java // as well as the constructor of PSQLException.java with "boolean detail" flag in // https://github.com/pgjdbc/pgjdbc/blob/master/pgjdbc/src/main/java/org/postgresql/util/ diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 4e20b1f15..8ffee7586 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -281,9 +281,6 @@ private boolean addConfigErrorsForLegacyAndNewConfigConflict() { * Validate that at least one configuration is provided. */ private boolean addConfigErrorsForNoConfigProvided() { - if (config.getQuery().isPresent()) { - return true; - } String msg = "At least one table filtering configuration is required. " + "Provide one of: " + JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG + ", " + JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG + ", " @@ -345,10 +342,10 @@ private boolean validateQueryConfigs() { if (config.getQuery().isPresent() && isUsingTableFilteringConfigs()) { String msg = - "Do not specify table filtering configs with 'query' or 'query.masked'. " + "Do not specify table filtering configs with 'query'. " + "Remove table.whitelist / table.blacklist / table.include.list / " + "table.exclude.list when using query mode" - + " or 'query' / 'query.masked' when using table filtering mode."; + + " or 'query' when using table filtering mode."; addConfigError(JdbcSourceConnectorConfig.QUERY_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.QUERY_MASKED_CONFIG, msg); if (!config.getTableWhitelistSet().isEmpty()) { diff --git a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java index 8e792d842..ccb33a790 100644 --- a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java +++ b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java @@ -38,8 +38,7 @@ public void testNonSqlThrowable() { public void testSqlExceptionNoNested() { SQLException e = new SQLException("e"); SQLException trimmed = LogUtil.trimSensitiveData(e); - SQLException expectedTrimmed = new SQLException(REDACTED); - assertEqualsSQLException(expectedTrimmed, trimmed); + assertEqualsSQLException(e, trimmed); } @Test @@ -49,9 +48,7 @@ public void testSqlExceptionOneLevelNestedNonBatchUpdate() { e1.setNextException(e2); SQLException trimmed = LogUtil.trimSensitiveData(e1); - SQLException expectedTrimmed = new SQLException(REDACTED); - expectedTrimmed.setNextException(new SQLException(REDACTED)); - assertEqualsSQLException(expectedTrimmed, trimmed); + assertEqualsSQLException(e1, trimmed); } @Test @@ -99,9 +96,7 @@ public void testSecondLevelNestedBatchUpdateNoSensitive() { e1.setNextException(e2); SQLException trimmed = LogUtil.trimSensitiveData(e1); - SQLException expectedTrimmed = new SQLException(REDACTED); - expectedTrimmed.setNextException(e2); - assertEqualsSQLException(expectedTrimmed, trimmed); + assertEqualsSQLException(e1, trimmed); } @Test @@ -113,7 +108,7 @@ public void testSecondLevelNestedBatchUpdateSensitive() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException(REDACTED); + SQLException expectedTrimmed = new SQLException("e1"); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\"): " + "ERROR: null value in column \"c4\" violates not-null constraint", new int[0]); @@ -131,7 +126,7 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoError() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException(REDACTED); + SQLException expectedTrimmed = new SQLException("e1"); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); @@ -154,6 +149,48 @@ public void testSensitiveLogWithTrimDisabled() { assertEquals(message, result); } + @Test + public void testRedactSensitiveDataWithNonSqlThrowable() { + Throwable t = new RuntimeException("secret"); + Assert.assertSame(t, LogUtil.redactSensitiveData(t)); + } + + @Test + public void testRedactSensitiveDataWithSqlExceptionChain() { + SQLException e1 = new SQLException("sensitive-message-e1", "42000", 10); + SQLException e2 = new SQLException("sensitive-message-e2", "42001", 20); + e1.setNextException(e2); + + SQLException expected = new SQLException(REDACTED, "42000", 10); + SQLException expectedChild = new SQLException(REDACTED, "42001", 20); + expected.setNextException(expectedChild); + + SQLException redacted = LogUtil.redactSensitiveData(e1); + + assertEqualsSQLException(expected, redacted); + } + + @Test + public void testRedactSensitiveDataWithBatchUpdateException() { + BatchUpdateException e1 = + new BatchUpdateException("sensitive message-e1", "42002", 30, new int[0]); + + SQLException e2 = new SQLException("sensitive message-e2", "42003", 40); + e1.setNextException(e2); + + BatchUpdateException expected = + new BatchUpdateException(REDACTED, "42002", 30, new int[0]); + SQLException expectedChild = new SQLException(REDACTED, "42003", 40); + expected.setNextException(expectedChild); + + SQLException actual = LogUtil.redactSensitiveData(e1); + Assert.assertTrue(actual instanceof BatchUpdateException); + Assert.assertArrayEquals( + expected.getUpdateCounts(), ((BatchUpdateException) actual).getUpdateCounts()); + + assertEqualsSQLException(expected, actual); + } + @Test public void testSecondLevelNestedBatchUpdateSensitiveNoDetails() { SQLException e1 = new SQLException("e1"); diff --git a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java index 6802c2a7d..5922d7865 100644 --- a/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java +++ b/src/test/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidationTest.java @@ -778,8 +778,6 @@ public void validate_withModeBulkWithNewIncrementingMapping_setsError() { assertErrorMatches(MODE_CONFIG, ".*Incrementing column configurations should not be provided.*"); } - // ========== Query and Query.Masked Config Tests ========== - @Test public void validate_withBothQueryAndQueryMasked_setsError() { props.put(MODE_CONFIG, MODE_BULK); @@ -816,6 +814,19 @@ public void validate_withOnlyQueryMasked_noErrors() { assertNoErrors(); } + @Test + public void validate_withBothQueryAndQueryMaskedEmpty_noErrors() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); + // Both empty should be fine as it's equivalent to neither being set + props.put(QUERY_CONFIG, ""); + props.put(QUERY_MASKED_CONFIG, ""); + + validate(); + + assertNoErrors(); + } + @Test public void validate_withQueryStartingWithUpdate_setsError() { props.put(MODE_CONFIG, MODE_BULK); @@ -828,6 +839,21 @@ public void validate_withQueryStartingWithUpdate_setsError() { assertErrorMatches(QUERY_CONFIG, ".*Only SELECT statements are supported for 'query'.*"); } + @Test + public void validate_withQueryMaskedStartingWithUpdate_setsError() { + props.put(MODE_CONFIG, MODE_BULK); + props.put(QUERY_MASKED_CONFIG, "UPDATE users SET active = false"); + + validate(); + + assertErrors(1); + assertErrors(QUERY_MASKED_CONFIG, 1); + assertErrorMatches( + QUERY_MASKED_CONFIG, + "Only SELECT statements are supported for 'query.masked'" + ); + } + @Test public void validate_withQueryMaskedAndIncrementingColumn_noErrors() { props.put(MODE_CONFIG, MODE_INCREMENTING); @@ -856,19 +882,6 @@ public void validate_withQueryMaskedContainingComplexQuery_noErrors() { assertNoErrors(); } - @Test - public void validate_withBothQueryAndQueryMaskedEmpty_noErrors() { - props.put(MODE_CONFIG, MODE_BULK); - props.put(TABLE_WHITELIST_CONFIG, "table1,table2"); - // Both empty should be fine as it's equivalent to neither being set - props.put(QUERY_CONFIG, ""); - props.put(QUERY_MASKED_CONFIG, ""); - - validate(); - - assertNoErrors(); - } - @Test public void validate_withQueryAndTableFilteringConfigs_setsError() { props.put(MODE_CONFIG, MODE_BULK); @@ -886,20 +899,5 @@ public void validate_withQueryAndTableFilteringConfigs_setsError() { "Do not specify table filtering configs with 'query'" ); } - - @Test - public void validate_withQueryMaskedStartingWithUpdate_setsError() { - props.put(MODE_CONFIG, MODE_BULK); - props.put(QUERY_MASKED_CONFIG, "UPDATE users SET active = false"); - - validate(); - - assertErrors(1); - assertErrors(QUERY_MASKED_CONFIG, 1); - assertErrorMatches( - QUERY_MASKED_CONFIG, - "Only SELECT statements are supported for 'query.masked'" - ); - } } From a74fd74627c1af48affe60dafba34ad240e97c19 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 2 Dec 2025 16:02:40 +0530 Subject: [PATCH 17/22] Minor nits --- src/main/java/io/confluent/connect/jdbc/util/LogUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index 55553058f..8ad95a26a 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -26,7 +26,7 @@ public class LogUtil { private static final String REDACTED_VALUE = ""; public static SQLException trimSensitiveData(SQLException e) { - return (SQLException) trimSensitiveData((Throwable) e); + return (SQLException) trimSensitiveData((Throwable)e); } public static Throwable trimSensitiveData(Throwable t) { From 6bc41955702582860f0f461b79e99f0671dac844 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Tue, 2 Dec 2025 17:26:27 +0530 Subject: [PATCH 18/22] Fix tests --- .../jdbc/source/TableQuerierProcessor.java | 4 ++- .../JdbcSourceConnectorValidation.java | 26 ------------------- .../connect/jdbc/util/LogUtilTest.java | 10 +++---- 3 files changed, 6 insertions(+), 34 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index b0f385d10..7ad3d9103 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -193,7 +193,7 @@ public void shutdown() { } private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean resetOffset) { - log.debug("Resetting querier {}", expectedHead); + log.debug("Resetting querier {}", expectedHead.toString()); TableQuerier removedQuerier = tableQueue.poll(); assert removedQuerier == expectedHead; expectedHead.reset(time.milliseconds(), resetOffset); @@ -201,4 +201,6 @@ private synchronized void resetAndRequeueHead(TableQuerier expectedHead, boolean tableQueue.add(expectedHead); } } + + } diff --git a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java index 8ffee7586..f32401430 100644 --- a/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java +++ b/src/main/java/io/confluent/connect/jdbc/validation/JdbcSourceConnectorValidation.java @@ -92,17 +92,14 @@ && validateLegacyNewConfigCompatibility() } validationResult = validationResult && validatePluginSpecificNeeds(); - if (!validationResult) { log.info("Validation failed"); } else { log.info("Validation succeeded"); } - } catch (Exception e) { log.error("Error during validation", e); } - return this.validationResult; } @@ -187,7 +184,6 @@ private boolean isUsingLegacyConfigs() { Set blacklistSet = config.getTableBlacklistSet(); String incrementingColumnName = config.getIncrementingColumnName(); List timestampColumnName = config.getTimestampColumnName(); - boolean hasWhitelist = !whitelistSet.isEmpty(); boolean hasBlacklist = !blacklistSet.isEmpty(); boolean hasLegacyIncrementing = incrementingColumnName != null @@ -195,7 +191,6 @@ private boolean isUsingLegacyConfigs() { boolean hasLegacyTimestamp = timestampColumnName != null && !timestampColumnName.isEmpty() && !timestampColumnName.get(0).trim().isEmpty(); - return hasWhitelist || hasBlacklist || hasLegacyIncrementing || hasLegacyTimestamp; } @@ -209,14 +204,12 @@ private boolean isUsingNewConfigs() { Set excludeListSet = config.getTableExcludeListSet(); List incrementingColumnMapping = config.getIncrementingColumnMapping(); List timestampColumnsMapping = config.getTimestampColumnMapping(); - boolean hasIncludeList = !includeListSet.isEmpty(); boolean hasExcludeList = !excludeListSet.isEmpty(); boolean hasNewIncrementing = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); boolean hasNewTimestamp = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - return hasIncludeList || hasExcludeList || hasNewIncrementing || hasNewTimestamp; } @@ -231,49 +224,40 @@ private boolean addConfigErrorsForLegacyAndNewConfigConflict() { + "(table.include.list, table.exclude.list, timestamp.columns.mapping, " + "incrementing.column.mapping). Please choose one approach: either use all legacy " + "configurations or all new configurations."; - // Only add errors to configs that are actually present and non-empty Set whitelistSet = config.getTableWhitelistSet(); if (!whitelistSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); } - Set blacklistSet = config.getTableBlacklistSet(); if (!blacklistSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); } - Set includeListSet = config.getTableIncludeListSet(); if (!includeListSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); } - Set excludeListSet = config.getTableExcludeListSet(); if (!excludeListSet.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); } - List timestampColumnName = config.getTimestampColumnName(); if (timestampColumnName != null && !timestampColumnName.isEmpty() && !timestampColumnName.get(0).trim().isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TIMESTAMP_COLUMN_NAME_CONFIG, msg); } - List timestampColumnsMapping = config.getTimestampColumnMapping(); if (timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.TIMESTAMP_COLUMN_MAPPING_CONFIG, msg); } - String incrementingColumnName = config.getIncrementingColumnName(); if (incrementingColumnName != null && !incrementingColumnName.trim().isEmpty()) { addConfigError(JdbcSourceConnectorConfig.INCREMENTING_COLUMN_NAME_CONFIG, msg); } - List incrementingColumnMapping = config.getIncrementingColumnMapping(); if (incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty()) { addConfigError(JdbcSourceConnectorConfig.INCREMENTING_COLUMN_MAPPING_CONFIG, msg); } - return false; } @@ -286,7 +270,6 @@ private boolean addConfigErrorsForNoConfigProvided() { + JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG + ", " + JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG + ", or " + JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG + "."; - addConfigError(JdbcSourceConnectorConfig.TABLE_WHITELIST_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_BLACKLIST_CONFIG, msg); addConfigError(JdbcSourceConnectorConfig.TABLE_INCLUDE_LIST_CONFIG, msg); @@ -300,7 +283,6 @@ private boolean addConfigErrorsForNoConfigProvided() { private boolean validateTableInclusionConfigs() { Set includeListSet = config.getTableIncludeListSet(); Set excludeListSet = config.getTableExcludeListSet(); - // Validate that exclude list requires include list if (!excludeListSet.isEmpty() && includeListSet.isEmpty()) { String msg = JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG @@ -309,7 +291,6 @@ private boolean validateTableInclusionConfigs() { addConfigError(JdbcSourceConnectorConfig.TABLE_EXCLUDE_LIST_CONFIG, msg); return false; } - return true; } @@ -406,7 +387,6 @@ private boolean validateTsColProvidedWhenRequired() { List timestampColumnsMapping = config.getTimestampColumnMapping(); boolean hasNewTimestampConfig = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - if (!hasNewTimestampConfig) { String msg = String.format( "Timestamp column configuration must be provided when using mode '%s' or '%s'. " @@ -429,10 +409,8 @@ private boolean validateTsColProvidedWhenRequired() { private boolean validateTsColNotProvidedWhenNotRequired() { if (!config.modeUsesTimestampColumn()) { List timestampColumnsMapping = config.getTimestampColumnMapping(); - boolean hasNewTimestampConfig = timestampColumnsMapping != null && !timestampColumnsMapping.isEmpty(); - if (hasNewTimestampConfig) { String msg = String.format( "Timestamp column configurations should not be provided if mode is not '%s' or '%s'. " @@ -457,10 +435,8 @@ private boolean validateTsColNotProvidedWhenNotRequired() { private boolean validateIncrColProvidedWhenRequired() { if (config.modeUsesIncrementingColumn()) { List incrementingColumnMapping = config.getIncrementingColumnMapping(); - boolean hasNewIncrementingConfig = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); - if (!hasNewIncrementingConfig) { String msg = String.format( "Incrementing column configuration must be provided when using mode '%s' or '%s'. " @@ -483,10 +459,8 @@ private boolean validateIncrColProvidedWhenRequired() { private boolean validateIncrColumnNotProvidedWhenNotRequired() { if (!config.modeUsesIncrementingColumn()) { List incrementingColumnMapping = config.getIncrementingColumnMapping(); - boolean hasNewIncrementingConfig = incrementingColumnMapping != null && !incrementingColumnMapping.isEmpty(); - if (hasNewIncrementingConfig) { String msg = String.format( "Incrementing column configurations " diff --git a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java index ccb33a790..7bd003279 100644 --- a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java +++ b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java @@ -60,11 +60,7 @@ public void testSqlExceptionTwoLevelNestedNonBatchUpdate() { e2.setNextException(e3); SQLException trimmed = LogUtil.trimSensitiveData(e1); - SQLException expectedTrimmed = new SQLException(REDACTED); - SQLException expectedNext = new SQLException(REDACTED); - expectedNext.setNextException(new SQLException(REDACTED)); - expectedTrimmed.setNextException(expectedNext); - assertEqualsSQLException(expectedTrimmed, trimmed); + assertEqualsSQLException(e1, trimmed); } @Test @@ -199,7 +195,7 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoDetails() { new int[0]); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException(REDACTED); + SQLException expectedTrimmed = new SQLException("e1"); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); @@ -220,7 +216,7 @@ public void testBatchExceptionWithChild() { e2.setNextException(p1); e1.setNextException(e2); - SQLException expectedTrimmed = new SQLException(REDACTED); + SQLException expectedTrimmed = new SQLException("e1"); BatchUpdateException e3 = new BatchUpdateException("Batch entry 0 INSERT INTO \"abc\" (\"c1\",\"c2\",\"c3\",\"c4\")", new int[0]); expectedTrimmed.setNextException(e3); From 9c654e9a20e7f0efd86a2489dd78f6e26d04c782 Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 3 Dec 2025 10:16:13 +0530 Subject: [PATCH 19/22] Remove trim.sensitive.log config in source connector config --- .../connect/jdbc/source/JdbcSourceConnectorConfig.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java index 32090fdcd..651aef7d3 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceConnectorConfig.java @@ -408,9 +408,6 @@ public class JdbcSourceConnectorConfig extends AbstractConfig { "Number of times to retry SQL exceptions encountered when executing queries."; public static final String QUERY_RETRIES_DISPLAY = "Query Retry Attempts"; - public static final String TRIM_SENSITIVE_LOG_ENABLED = "trim.sensitive.log"; - private static final String TRIM_SENSITIVE_LOG_ENABLED_DEFAULT = "false"; - /** * The properties that begin with this prefix will be used to configure a class, specified by * {@code jdbc.credentials.provider.class} if it implements {@link Configurable}. @@ -1092,11 +1089,6 @@ private static final void addConnectorOptions(ConfigDef config) { Type.LONG, TABLE_MONITORING_STARTUP_POLLING_LIMIT_MS_DEFAULT, Importance.LOW - ).defineInternal( - TRIM_SENSITIVE_LOG_ENABLED, - ConfigDef.Type.BOOLEAN, - TRIM_SENSITIVE_LOG_ENABLED_DEFAULT, - ConfigDef.Importance.LOW ).define( TABLE_POLL_INTERVAL_MS_CONFIG, Type.LONG, From 552d19d33284320f10fb0a571a2d69fb36bdbfae Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 3 Dec 2025 10:23:25 +0530 Subject: [PATCH 20/22] Renamed sensitiveLog methodName --- .../io/confluent/connect/jdbc/source/BulkTableQuerier.java | 4 ++-- .../java/io/confluent/connect/jdbc/source/JdbcSourceTask.java | 2 +- .../java/io/confluent/connect/jdbc/source/TableQuerier.java | 2 +- .../jdbc/source/TimestampIncrementingTableQuerier.java | 2 +- .../confluent/connect/jdbc/source/TimestampTableQuerier.java | 2 +- src/main/java/io/confluent/connect/jdbc/util/LogUtil.java | 2 +- src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java | 4 ++-- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index d28ac5491..281b7784f 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -73,7 +73,7 @@ protected void createPreparedStatement(Connection db) throws SQLException { recordQuery(queryStr); log.trace( - "{} prepared SQL query: {}", this, LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query)); + "{} prepared SQL query: {}", this, LogUtil.maybeRedact(shouldTrimSensitiveLogs, query)); stmt = dialect.createPreparedStatement(db, queryStr); } @@ -124,7 +124,7 @@ public String toString() { + tableId + '\'' + ", query='" - + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java index 731218aa3..64d49b895 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java +++ b/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java @@ -465,7 +465,7 @@ protected Map computeInitialOffset( Map partitionOffset, ZoneId zoneId) { if (config.isQueryMasked()) { - tableOrQuery = LogUtil.sensitiveLog(true, tableOrQuery); + tableOrQuery = LogUtil.maybeRedact(true, tableOrQuery); } if (!(partitionOffset == null)) { log.info("Partition offset for '{}' is not null. Using existing offset.", tableOrQuery); diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 508fe1d5d..14e4e0dfd 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -183,7 +183,7 @@ protected void addSuffixIfPresent(ExpressionBuilder builder) { protected void recordQuery(String query) { if (query != null && !query.equals(loggedQueryString)) { // For usability, log the statement at INFO level only when it changes - log.info("Begin using SQL query: {}", LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query)); + log.info("Begin using SQL query: {}", LogUtil.maybeRedact(shouldTrimSensitiveLogs, query)); loggedQueryString = query; } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 752720f83..9bf064c2b 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -280,7 +280,7 @@ public String toString() { + "table=" + tableId + ", query='" - + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index 1cdf09171..e1188a86a 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -184,7 +184,7 @@ public String toString() { + "table=" + tableId + ", query='" - + LogUtil.sensitiveLog(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index 8ad95a26a..a64a0b18b 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -116,7 +116,7 @@ private static String getNonSensitiveErrorMessage(String errMsg) { return msg1 + msg2; } - public static String sensitiveLog(boolean trimSensitiveLogsEnabled, String msg) { + public static String maybeRedact(boolean trimSensitiveLogsEnabled, String msg) { if (trimSensitiveLogsEnabled) { return REDACTED_VALUE; } diff --git a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java index 7bd003279..465699656 100644 --- a/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java +++ b/src/test/java/io/confluent/connect/jdbc/util/LogUtilTest.java @@ -134,14 +134,14 @@ public void testSecondLevelNestedBatchUpdateSensitiveNoError() { @Test public void testSensitiveLogWithTrimEnabled() { String sensitiveMessage = "SELECT * FROM users WHERE password='secret123'"; - String result = LogUtil.sensitiveLog(true, sensitiveMessage); + String result = LogUtil.maybeRedact(true, sensitiveMessage); assertEquals(REDACTED, result); } @Test public void testSensitiveLogWithTrimDisabled() { String message = "SELECT * FROM users WHERE id=1"; - String result = LogUtil.sensitiveLog(false, message); + String result = LogUtil.maybeRedact(false, message); assertEquals(message, result); } From 6b8b0be068b3e702502cd66e7741882306de1ddb Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 3 Dec 2025 13:54:19 +0530 Subject: [PATCH 21/22] Minor nits --- .../connect/jdbc/source/BulkTableQuerier.java | 4 ++-- .../confluent/connect/jdbc/source/TableQuerier.java | 6 +++--- .../connect/jdbc/source/TableQuerierProcessor.java | 12 ++++++------ .../source/TimestampIncrementingTableQuerier.java | 2 +- .../connect/jdbc/source/TimestampTableQuerier.java | 2 +- .../java/io/confluent/connect/jdbc/util/LogUtil.java | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java index 281b7784f..f46dd55ae 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/BulkTableQuerier.java @@ -73,7 +73,7 @@ protected void createPreparedStatement(Connection db) throws SQLException { recordQuery(queryStr); log.trace( - "{} prepared SQL query: {}", this, LogUtil.maybeRedact(shouldTrimSensitiveLogs, query)); + "{} prepared SQL query: {}", this, LogUtil.maybeRedact(shouldRedactSensitiveLogs, query)); stmt = dialect.createPreparedStatement(db, queryStr); } @@ -124,7 +124,7 @@ public String toString() { + tableId + '\'' + ", query='" - + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldRedactSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java index 14e4e0dfd..b679b6500 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerier.java @@ -57,7 +57,7 @@ public enum QueryMode { protected ResultSet resultSet; protected SchemaMapping schemaMapping; private String loggedQueryString; - protected final Boolean shouldTrimSensitiveLogs; + protected final Boolean shouldRedactSensitiveLogs; private int attemptedRetries; @@ -77,7 +77,7 @@ public TableQuerier( this.lastUpdate = 0; this.suffix = suffix; this.attemptedRetries = 0; - this.shouldTrimSensitiveLogs = isQueryMasked; + this.shouldRedactSensitiveLogs = isQueryMasked; } public long getLastUpdate() { @@ -183,7 +183,7 @@ protected void addSuffixIfPresent(ExpressionBuilder builder) { protected void recordQuery(String query) { if (query != null && !query.equals(loggedQueryString)) { // For usability, log the statement at INFO level only when it changes - log.info("Begin using SQL query: {}", LogUtil.maybeRedact(shouldTrimSensitiveLogs, query)); + log.info("Begin using SQL query: {}", LogUtil.maybeRedact(shouldRedactSensitiveLogs, query)); loggedQueryString = query; } } diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index 7ad3d9103..d2b788a2e 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -26,7 +26,7 @@ public class TableQuerierProcessor { private final JdbcSourceTaskConfig config; private final Time time; private final PriorityQueue tableQueue; - private final Boolean shouldTrimSensitiveLogs; + private final Boolean shouldRedactSensitiveLogs; private CachedConnectionProvider cachedConnectionProvider; private final int maxRetriesPerQuerier; private final Duration timeout = Duration.ofSeconds(90); @@ -43,7 +43,7 @@ public TableQuerierProcessor( this.tableQueue = tableQueue; this.cachedConnectionProvider = cachedConnectionProvider; this.maxRetriesPerQuerier = config.getInt(JdbcSourceConnectorConfig.QUERY_RETRIES_CONFIG); - this.shouldTrimSensitiveLogs = config.isQueryMasked(); + this.shouldRedactSensitiveLogs = config.isQueryMasked(); } public long process(RecordDestination destination) { @@ -133,8 +133,8 @@ private void processQuerier(RecordDestination destination, TableQu private void handleNonTransientException(RecordDestination destination, TableQuerier querier, SQLNonTransientException sqle) { - SQLException trimmedException = shouldTrimSensitiveLogs - ? LogUtil.trimSensitiveData(sqle) : sqle; + SQLException trimmedException = shouldRedactSensitiveLogs + ? LogUtil.redactSensitiveData(sqle) : sqle; log.error("Non-transient SQL exception while running query for table: {}", querier, trimmedException); resetAndRequeueHead(querier, true); @@ -144,8 +144,8 @@ private void handleNonTransientException(RecordDestination destina private void handleSqlException(RecordDestination destination, TableQuerier querier, SQLException sqle) { - SQLException trimmedException = shouldTrimSensitiveLogs - ? LogUtil.trimSensitiveData(sqle) : sqle; + SQLException trimmedException = shouldRedactSensitiveLogs + ? LogUtil.redactSensitiveData(sqle) : sqle; log.error( "SQL exception while running query for table: {}." + " Attempting retry {} of {} attempts.", querier, diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java index 9bf064c2b..c727d88c6 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampIncrementingTableQuerier.java @@ -280,7 +280,7 @@ public String toString() { + "table=" + tableId + ", query='" - + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldRedactSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java index e1188a86a..eb1d143b6 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TimestampTableQuerier.java @@ -184,7 +184,7 @@ public String toString() { + "table=" + tableId + ", query='" - + LogUtil.maybeRedact(shouldTrimSensitiveLogs, query) + + LogUtil.maybeRedact(shouldRedactSensitiveLogs, query) + '\'' + ", topicPrefix='" + topicPrefix diff --git a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java index a64a0b18b..acb75f2e6 100644 --- a/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java +++ b/src/main/java/io/confluent/connect/jdbc/util/LogUtil.java @@ -116,8 +116,8 @@ private static String getNonSensitiveErrorMessage(String errMsg) { return msg1 + msg2; } - public static String maybeRedact(boolean trimSensitiveLogsEnabled, String msg) { - if (trimSensitiveLogsEnabled) { + public static String maybeRedact(boolean shouldRedactSensitiveLogs, String msg) { + if (shouldRedactSensitiveLogs) { return REDACTED_VALUE; } return String.valueOf(msg); From 5e9c1be03d86017005cf97bf01fc4e8b96cef63a Mon Sep 17 00:00:00 2001 From: KGaneshDatta Date: Wed, 3 Dec 2025 14:00:37 +0530 Subject: [PATCH 22/22] Minor nits+ --- .../connect/jdbc/source/TableQuerierProcessor.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java index d2b788a2e..2d9462113 100644 --- a/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java +++ b/src/main/java/io/confluent/connect/jdbc/source/TableQuerierProcessor.java @@ -133,30 +133,30 @@ private void processQuerier(RecordDestination destination, TableQu private void handleNonTransientException(RecordDestination destination, TableQuerier querier, SQLNonTransientException sqle) { - SQLException trimmedException = shouldRedactSensitiveLogs + SQLException redactedException = shouldRedactSensitiveLogs ? LogUtil.redactSensitiveData(sqle) : sqle; log.error("Non-transient SQL exception while running query for table: {}", - querier, trimmedException); + querier, redactedException); resetAndRequeueHead(querier, true); // This task has failed, report failure to destination - destination.failWith(new ConnectException(trimmedException)); + destination.failWith(new ConnectException(redactedException)); } private void handleSqlException(RecordDestination destination, TableQuerier querier, SQLException sqle) { - SQLException trimmedException = shouldRedactSensitiveLogs + SQLException redactedException = shouldRedactSensitiveLogs ? LogUtil.redactSensitiveData(sqle) : sqle; log.error( "SQL exception while running query for table: {}." + " Attempting retry {} of {} attempts.", querier, querier.getAttemptedRetryCount() + 1, maxRetriesPerQuerier, - trimmedException); + redactedException); resetAndRequeueHead(querier, false); if (maxRetriesPerQuerier > 0 && querier.getAttemptedRetryCount() >= maxRetriesPerQuerier) { destination.failWith( - new ConnectException("Failed to query table after retries", trimmedException)); + new ConnectException("Failed to query table after retries", redactedException)); return; } querier.incrementRetryCount();