diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml
index 3456907d014ed..224c11dc8b2f8 100644
--- a/presto-blackhole/pom.xml
+++ b/presto-blackhole/pom.xml
@@ -23,6 +23,21 @@
guava
+
+ com.facebook.airlift
+ configuration
+
+
+
+ com.facebook.airlift
+ json
+
+
+
+ com.facebook.airlift
+ bootstrap
+
+
com.facebook.airlift
concurrent
@@ -92,9 +107,8 @@
- com.facebook.airlift
- log-manager
- test
+ com.google.inject
+ guice
diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleClientConfig.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleClientConfig.java
new file mode 100644
index 0000000000000..d098af4308f4b
--- /dev/null
+++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleClientConfig.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.plugin.blackhole;
+
+import com.facebook.airlift.configuration.Config;
+import com.facebook.airlift.configuration.ConfigDescription;
+
+public class BlackHoleClientConfig
+{
+ private boolean caseSensitiveNameMatchingEnabled;
+
+ public boolean isCaseSensitiveNameMatching()
+ {
+ return caseSensitiveNameMatchingEnabled;
+ }
+
+ @Config("case-sensitive-name-matching")
+ @ConfigDescription("Enable case-sensitive matching of schema, table names across the connector. " +
+ "When disabled, names are matched case-insensitively using lowercase normalization.")
+ public BlackHoleClientConfig setCaseSensitiveNameMatching(boolean caseSensitiveNameMatchingEnabled)
+ {
+ this.caseSensitiveNameMatchingEnabled = caseSensitiveNameMatchingEnabled;
+ return this;
+ }
+}
diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnector.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnector.java
index eb9bd54b12548..13046450d81ed 100644
--- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnector.java
+++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnector.java
@@ -55,6 +55,7 @@ public class BlackHoleConnector
private final BlackHoleNodePartitioningProvider partitioningProvider;
private final TypeManager typeManager;
private final ExecutorService executorService;
+ private final boolean caseSensitiveNameMatching;
public BlackHoleConnector(
BlackHoleMetadata metadata,
@@ -63,7 +64,8 @@ public BlackHoleConnector(
BlackHolePageSinkProvider pageSinkProvider,
BlackHoleNodePartitioningProvider partitioningProvider,
TypeManager typeManager,
- ExecutorService executorService)
+ ExecutorService executorService,
+ BlackHoleClientConfig config)
{
this.metadata = metadata;
this.splitManager = splitManager;
@@ -72,6 +74,12 @@ public BlackHoleConnector(
this.partitioningProvider = partitioningProvider;
this.typeManager = typeManager;
this.executorService = executorService;
+ this.caseSensitiveNameMatching = config.isCaseSensitiveNameMatching();
+ }
+
+ private String normalizeColumnName(String name)
+ {
+ return caseSensitiveNameMatching ? name : name.toLowerCase(ENGLISH);
}
@Override
@@ -143,7 +151,7 @@ public List> getTableProperties()
ImmutableList.of(),
false,
value -> ImmutableList.copyOf(((List) value).stream()
- .map(name -> name.toLowerCase(ENGLISH))
+ .map(this::normalizeColumnName)
.collect(toList())),
List.class::cast),
new PropertyMetadata<>(
diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java
index 1c3c0c3c9efc9..1e2b55df6a2a1 100644
--- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java
+++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java
@@ -13,15 +13,19 @@
*/
package com.facebook.presto.plugin.blackhole;
+import com.facebook.airlift.bootstrap.Bootstrap;
+import com.facebook.airlift.json.JsonModule;
import com.facebook.presto.spi.ConnectorHandleResolver;
import com.facebook.presto.spi.connector.Connector;
import com.facebook.presto.spi.connector.ConnectorContext;
import com.facebook.presto.spi.connector.ConnectorFactory;
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
+import com.google.inject.Injector;
import java.util.Map;
import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed;
+import static com.facebook.airlift.configuration.ConfigBinder.configBinder;
import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator;
import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
@@ -43,14 +47,33 @@ public ConnectorHandleResolver getHandleResolver()
@Override
public Connector create(String catalogName, Map requiredConfig, ConnectorContext context)
{
- ListeningScheduledExecutorService executorService = listeningDecorator(newSingleThreadScheduledExecutor(daemonThreadsNamed("blackhole")));
- return new BlackHoleConnector(
- new BlackHoleMetadata(),
- new BlackHoleSplitManager(),
- new BlackHolePageSourceProvider(executorService),
- new BlackHolePageSinkProvider(executorService),
- new BlackHoleNodePartitioningProvider(context.getNodeManager()),
- context.getTypeManager(),
- executorService);
+ try {
+ Bootstrap app = new Bootstrap(
+ new JsonModule(),
+ binder -> {
+ configBinder(binder).bindConfig(BlackHoleClientConfig.class);
+ });
+
+ Injector injector = app
+ .doNotInitializeLogging()
+ .setRequiredConfigurationProperties(requiredConfig)
+ .initialize();
+
+ BlackHoleClientConfig config = injector.getInstance(BlackHoleClientConfig.class);
+
+ ListeningScheduledExecutorService executorService = listeningDecorator(newSingleThreadScheduledExecutor(daemonThreadsNamed("blackhole")));
+ return new BlackHoleConnector(
+ new BlackHoleMetadata(),
+ new BlackHoleSplitManager(),
+ new BlackHolePageSourceProvider(executorService),
+ new BlackHolePageSinkProvider(executorService),
+ new BlackHoleNodePartitioningProvider(context.getNodeManager()),
+ context.getTypeManager(),
+ executorService,
+ config);
+ }
+ catch (Exception e) {
+ throw new RuntimeException(e);
+ }
}
}
diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleMetadata.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleMetadata.java
index 4c0495ca4a5c1..5f6713a5f51c6 100644
--- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleMetadata.java
+++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleMetadata.java
@@ -56,6 +56,8 @@
import static com.facebook.presto.spi.StandardErrorCode.INVALID_TABLE_PROPERTY;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.lang.String.format;
+import static java.util.Locale.ENGLISH;
+import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toMap;
import static java.util.stream.Collectors.toSet;
@@ -67,10 +69,23 @@ public class BlackHoleMetadata
private final List schemas = new ArrayList<>();
private final Map tables = new ConcurrentHashMap<>();
+ private boolean caseSensitiveNameMatchingEnabled;
public BlackHoleMetadata()
+ {
+ this(new BlackHoleClientConfig());
+ }
+
+ public BlackHoleMetadata(BlackHoleClientConfig config)
{
schemas.add(SCHEMA_NAME);
+ this.caseSensitiveNameMatchingEnabled = requireNonNull(config, "config is null").isCaseSensitiveNameMatching();
+ }
+
+ @Override
+ public String normalizeIdentifier(ConnectorSession session, String identifier)
+ {
+ return caseSensitiveNameMatchingEnabled ? identifier : identifier.toLowerCase(ENGLISH);
}
@Override
diff --git a/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleCaseSensitivity.java b/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleCaseSensitivity.java
new file mode 100644
index 0000000000000..972bdc34fa397
--- /dev/null
+++ b/presto-blackhole/src/test/java/com/facebook/presto/plugin/blackhole/TestBlackHoleCaseSensitivity.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.facebook.presto.plugin.blackhole;
+
+import com.facebook.presto.spi.ConnectorSession;
+import com.facebook.presto.testing.TestingConnectorSession;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+public class TestBlackHoleCaseSensitivity
+{
+ private static final ConnectorSession SESSION = TestingConnectorSession.SESSION;
+
+ @Test
+ public void testBlackHoleClientConfigDefaults()
+ {
+ BlackHoleClientConfig config = new BlackHoleClientConfig();
+ assertFalse(config.isCaseSensitiveNameMatching(),
+ "Default should be case-insensitive");
+ }
+
+ @Test
+ public void testBlackHoleClientConfigCaseSensitive()
+ {
+ BlackHoleClientConfig config = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(true);
+
+ assertTrue(config.isCaseSensitiveNameMatching(),
+ "Should be case-sensitive when enabled");
+ }
+
+ @Test
+ public void testBlackHoleClientConfigCaseInsensitive()
+ {
+ BlackHoleClientConfig config = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(false);
+
+ assertFalse(config.isCaseSensitiveNameMatching(),
+ "Should be case-insensitive when disabled");
+ }
+
+ @Test
+ public void testBlackHoleMetadataNormalizeIdentifierCaseSensitive()
+ {
+ BlackHoleClientConfig config = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(true);
+ BlackHoleMetadata metadata = new BlackHoleMetadata(config);
+
+ // Case-sensitive: identifiers should remain unchanged
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MyTable"), "MyTable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "myTable"), "myTable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MYTABLE"), "MYTABLE");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MySchema"), "MySchema");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "Test_Table"), "Test_Table");
+ }
+
+ @Test
+ public void testBlackHoleMetadataNormalizeIdentifierCaseInsensitive()
+ {
+ BlackHoleClientConfig config = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(false);
+ BlackHoleMetadata metadata = new BlackHoleMetadata(config);
+
+ // Case-insensitive: identifiers should be lowercased
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MyTable"), "mytable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "myTable"), "mytable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MYTABLE"), "mytable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MySchema"), "myschema");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "Test_Table"), "test_table");
+ }
+
+ @Test
+ public void testBlackHoleMetadataDefaultBehavior()
+ {
+ // Default metadata should be case-insensitive
+ BlackHoleMetadata metadata = new BlackHoleMetadata();
+
+ assertEquals(metadata.normalizeIdentifier(SESSION, "MyTable"), "mytable");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "TestSchema"), "testschema");
+ assertEquals(metadata.normalizeIdentifier(SESSION, "UPPERCASE"), "uppercase");
+ }
+
+ @Test
+ public void testConfigIntegrationWithMetadata()
+ {
+ // Test that config changes are properly reflected in metadata behavior
+
+ // Case-sensitive config
+ BlackHoleClientConfig caseSensitiveConfig = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(true);
+ BlackHoleMetadata caseSensitiveMetadata = new BlackHoleMetadata(caseSensitiveConfig);
+
+ // Case-insensitive config
+ BlackHoleClientConfig caseInsensitiveConfig = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(false);
+ BlackHoleMetadata caseInsensitiveMetadata = new BlackHoleMetadata(caseInsensitiveConfig);
+
+ String testIdentifier = "MyTestTable";
+
+ assertEquals(caseSensitiveMetadata.normalizeIdentifier(SESSION, testIdentifier), "MyTestTable");
+ assertEquals(caseInsensitiveMetadata.normalizeIdentifier(SESSION, testIdentifier), "mytesttable");
+ }
+
+ @Test
+ public void testMultipleConfigInstances()
+ {
+ // Test that multiple config instances work independently
+ BlackHoleClientConfig config1 = new BlackHoleClientConfig().setCaseSensitiveNameMatching(true);
+ BlackHoleClientConfig config2 = new BlackHoleClientConfig().setCaseSensitiveNameMatching(false);
+
+ assertTrue(config1.isCaseSensitiveNameMatching());
+ assertFalse(config2.isCaseSensitiveNameMatching());
+
+ // Test that they don't interfere with each other
+ BlackHoleMetadata metadata1 = new BlackHoleMetadata(config1);
+ BlackHoleMetadata metadata2 = new BlackHoleMetadata(config2);
+
+ assertEquals(metadata1.normalizeIdentifier(SESSION, "TestCase"), "TestCase");
+ assertEquals(metadata2.normalizeIdentifier(SESSION, "TestCase"), "testcase");
+ }
+
+ @Test
+ public void testConfigChaining()
+ {
+ // Test that config setter returns the config instance for chaining
+ BlackHoleClientConfig config = new BlackHoleClientConfig()
+ .setCaseSensitiveNameMatching(true)
+ .setCaseSensitiveNameMatching(false)
+ .setCaseSensitiveNameMatching(true);
+
+ assertTrue(config.isCaseSensitiveNameMatching(),
+ "Final value should be true after chaining");
+ }
+}