diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java index 4fb49bf8f19d..a08efb5641ac 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java @@ -68,7 +68,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; -import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isNonTransactionalInsert; +import static io.trino.plugin.jdbc.JdbcWriteSessionProperties.isNonTransactionalInsert; import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN; import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java index 5bcecf38b375..963f5ea624ff 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java @@ -17,13 +17,10 @@ import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.LegacyConfig; -import javax.validation.constraints.Max; import javax.validation.constraints.Min; public class JdbcMetadataConfig { - static final int MAX_ALLOWED_WRITE_BATCH_SIZE = 1_000_000; - private boolean allowDropTable; /* * Join pushdown is disabled by default as this is the safer option. @@ -43,12 +40,6 @@ public class JdbcMetadataConfig // between performance and pushdown capabilities private int domainCompactionThreshold = 32; - private int writeBatchSize = 1000; - - // Do not create temporary table during insert. - // This means that the write operation can fail and leave the table in an inconsistent state. - private boolean nonTransactionalInsert; - public boolean isAllowDropTable() { return allowDropTable; @@ -116,33 +107,4 @@ public JdbcMetadataConfig setDomainCompactionThreshold(int domainCompactionThres this.domainCompactionThreshold = domainCompactionThreshold; return this; } - - @Min(1) - @Max(MAX_ALLOWED_WRITE_BATCH_SIZE) - public int getWriteBatchSize() - { - return writeBatchSize; - } - - @Config("write.batch-size") - @ConfigDescription("Maximum number of rows to write in a single batch") - public JdbcMetadataConfig setWriteBatchSize(int writeBatchSize) - { - this.writeBatchSize = writeBatchSize; - return this; - } - - public boolean isNonTransactionalInsert() - { - return nonTransactionalInsert; - } - - @Config("insert.non-transactional-insert.enabled") - @ConfigDescription("Do not create temporary table during insert. " + - "This means that the write operation can fail and leave the table in an inconsistent state.") - public JdbcMetadataConfig setNonTransactionalInsert(boolean nonTransactionalInsert) - { - this.nonTransactionalInsert = nonTransactionalInsert; - return this; - } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java index f4cb013f2483..983022eaac03 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Optional; -import static io.trino.plugin.jdbc.JdbcMetadataConfig.MAX_ALLOWED_WRITE_BATCH_SIZE; import static io.trino.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; import static io.trino.spi.session.PropertyMetadata.booleanProperty; import static io.trino.spi.session.PropertyMetadata.integerProperty; @@ -37,8 +36,6 @@ public class JdbcMetadataSessionProperties public static final String AGGREGATION_PUSHDOWN_ENABLED = "aggregation_pushdown_enabled"; public static final String TOPN_PUSHDOWN_ENABLED = "topn_pushdown_enabled"; public static final String DOMAIN_COMPACTION_THRESHOLD = "domain_compaction_threshold"; - public static final String WRITE_BATCH_SIZE = "write_batch_size"; - public static final String NON_TRANSACTIONAL_INSERT = "non_transactional_insert"; private final List> properties; @@ -68,17 +65,6 @@ public JdbcMetadataSessionProperties(JdbcMetadataConfig jdbcMetadataConfig, @Max "Enable TopN pushdown", jdbcMetadataConfig.isTopNPushdownEnabled(), false)) - .add(integerProperty( - WRITE_BATCH_SIZE, - "Maximum number of rows to write in a single batch", - jdbcMetadataConfig.getWriteBatchSize(), - JdbcMetadataSessionProperties::validateWriteBatchSize, - false)) - .add(booleanProperty( - NON_TRANSACTIONAL_INSERT, - "Do not use temporary table on insert to table", - jdbcMetadataConfig.isNonTransactionalInsert(), - false)) .build(); } @@ -108,16 +94,6 @@ public static int getDomainCompactionThreshold(ConnectorSession session) return session.getProperty(DOMAIN_COMPACTION_THRESHOLD, Integer.class); } - public static int getWriteBatchSize(ConnectorSession session) - { - return session.getProperty(WRITE_BATCH_SIZE, Integer.class); - } - - public static boolean isNonTransactionalInsert(ConnectorSession session) - { - return session.getProperty(NON_TRANSACTIONAL_INSERT, Boolean.class); - } - private static void validateDomainCompactionThreshold(int domainCompactionThreshold, Optional maxDomainCompactionThreshold) { if (domainCompactionThreshold < 1) { @@ -130,14 +106,4 @@ private static void validateDomainCompactionThreshold(int domainCompactionThresh } }); } - - private static void validateWriteBatchSize(int maxBatchSize) - { - if (maxBatchSize < 1) { - throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s must be greater than 0: %s", WRITE_BATCH_SIZE, maxBatchSize)); - } - if (maxBatchSize > MAX_ALLOWED_WRITE_BATCH_SIZE) { - throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s cannot exceed %s: %s", WRITE_BATCH_SIZE, MAX_ALLOWED_WRITE_BATCH_SIZE, maxBatchSize)); - } - } } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java index f2dd1973a3b9..a7402ad2f082 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcModule.java @@ -66,11 +66,13 @@ public void setup(Binder binder) newOptionalBinder(binder, ConnectorPageSinkProvider.class).setDefault().to(JdbcPageSinkProvider.class).in(Scopes.SINGLETON); binder.bind(JdbcConnector.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(JdbcMetadataConfig.class); + configBinder(binder).bindConfig(JdbcWriteConfig.class); configBinder(binder).bindConfig(BaseJdbcConfig.class); configBinder(binder).bindConfig(TypeHandlingJdbcConfig.class); bindSessionPropertiesProvider(binder, TypeHandlingJdbcSessionProperties.class); bindSessionPropertiesProvider(binder, JdbcMetadataSessionProperties.class); + bindSessionPropertiesProvider(binder, JdbcWriteSessionProperties.class); binder.bind(CachingJdbcClient.class).in(Scopes.SINGLETON); binder.bind(JdbcClient.class).to(Key.get(CachingJdbcClient.class)).in(Scopes.SINGLETON); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcPageSink.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcPageSink.java index 2ef6a25383f2..f6e1abe5eea5 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcPageSink.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcPageSink.java @@ -35,7 +35,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_NON_TRANSIENT_ERROR; -import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.getWriteBatchSize; +import static io.trino.plugin.jdbc.JdbcWriteSessionProperties.getWriteBatchSize; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static java.util.concurrent.CompletableFuture.completedFuture; diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteConfig.java new file mode 100644 index 000000000000..957781f05eeb --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteConfig.java @@ -0,0 +1,60 @@ +/* + * 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 io.trino.plugin.jdbc; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; + +public class JdbcWriteConfig +{ + static final int MAX_ALLOWED_WRITE_BATCH_SIZE = 1_000_000; + + private int writeBatchSize = 1000; + + // Do not create temporary table during insert. + // This means that the write operation can fail and leave the table in an inconsistent state. + private boolean nonTransactionalInsert; + + @Min(1) + @Max(MAX_ALLOWED_WRITE_BATCH_SIZE) + public int getWriteBatchSize() + { + return writeBatchSize; + } + + @Config("write.batch-size") + @ConfigDescription("Maximum number of rows to write in a single batch") + public JdbcWriteConfig setWriteBatchSize(int writeBatchSize) + { + this.writeBatchSize = writeBatchSize; + return this; + } + + public boolean isNonTransactionalInsert() + { + return nonTransactionalInsert; + } + + @Config("insert.non-transactional-insert.enabled") + @ConfigDescription("Do not create temporary table during insert. " + + "This means that the write operation can fail and leave the table in an inconsistent state.") + public JdbcWriteConfig setNonTransactionalInsert(boolean nonTransactionalInsert) + { + this.nonTransactionalInsert = nonTransactionalInsert; + return this; + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteSessionProperties.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteSessionProperties.java new file mode 100644 index 000000000000..038998229b7d --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcWriteSessionProperties.java @@ -0,0 +1,83 @@ +/* + * 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 io.trino.plugin.jdbc; + +import com.google.common.collect.ImmutableList; +import io.trino.plugin.base.session.SessionPropertiesProvider; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.session.PropertyMetadata; + +import javax.inject.Inject; + +import java.util.List; + +import static io.trino.plugin.jdbc.JdbcWriteConfig.MAX_ALLOWED_WRITE_BATCH_SIZE; +import static io.trino.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; +import static io.trino.spi.session.PropertyMetadata.booleanProperty; +import static io.trino.spi.session.PropertyMetadata.integerProperty; +import static java.lang.String.format; + +public class JdbcWriteSessionProperties + implements SessionPropertiesProvider +{ + public static final String WRITE_BATCH_SIZE = "write_batch_size"; + public static final String NON_TRANSACTIONAL_INSERT = "non_transactional_insert"; + + private final List> properties; + + @Inject + public JdbcWriteSessionProperties(JdbcWriteConfig writeConfig) + { + properties = ImmutableList.>builder() + .add(integerProperty( + WRITE_BATCH_SIZE, + "Maximum number of rows to write in a single batch", + writeConfig.getWriteBatchSize(), + JdbcWriteSessionProperties::validateWriteBatchSize, + false)) + .add(booleanProperty( + NON_TRANSACTIONAL_INSERT, + "Do not use temporary table on insert to table", + writeConfig.isNonTransactionalInsert(), + false)) + .build(); + } + + @Override + public List> getSessionProperties() + { + return properties; + } + + public static int getWriteBatchSize(ConnectorSession session) + { + return session.getProperty(WRITE_BATCH_SIZE, Integer.class); + } + + public static boolean isNonTransactionalInsert(ConnectorSession session) + { + return session.getProperty(NON_TRANSACTIONAL_INSERT, Boolean.class); + } + + private static void validateWriteBatchSize(int maxBatchSize) + { + if (maxBatchSize < 1) { + throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s must be greater than 0: %s", WRITE_BATCH_SIZE, maxBatchSize)); + } + if (maxBatchSize > MAX_ALLOWED_WRITE_BATCH_SIZE) { + throw new TrinoException(INVALID_SESSION_PROPERTY, format("%s cannot exceed %s: %s", WRITE_BATCH_SIZE, MAX_ALLOWED_WRITE_BATCH_SIZE, maxBatchSize)); + } + } +} diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java index b676e39bb65f..693136be110d 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java @@ -14,7 +14,6 @@ package io.trino.plugin.jdbc; import com.google.common.collect.ImmutableMap; -import io.airlift.configuration.ConfigurationFactory; import org.testng.annotations.Test; import java.util.Map; @@ -22,8 +21,6 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; -import static org.assertj.core.api.Assertions.assertThatCode; -import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestJdbcMetadataConfig { @@ -35,9 +32,7 @@ public void testDefaults() .setJoinPushdownEnabled(false) .setAggregationPushdownEnabled(true) .setTopNPushdownEnabled(true) - .setDomainCompactionThreshold(32) - .setWriteBatchSize(1000) - .setNonTransactionalInsert(false)); + .setDomainCompactionThreshold(32)); } @Test @@ -49,8 +44,6 @@ public void testExplicitPropertyMappings() .put("aggregation-pushdown.enabled", "false") .put("domain-compaction-threshold", "42") .put("topn-pushdown.enabled", "false") - .put("write.batch-size", "24") - .put("insert.non-transactional-insert.enabled", "true") .build(); JdbcMetadataConfig expected = new JdbcMetadataConfig() @@ -58,31 +51,8 @@ public void testExplicitPropertyMappings() .setJoinPushdownEnabled(true) .setAggregationPushdownEnabled(false) .setTopNPushdownEnabled(false) - .setDomainCompactionThreshold(42) - .setWriteBatchSize(24) - .setNonTransactionalInsert(true); + .setDomainCompactionThreshold(42); assertFullMapping(properties, expected); } - - @Test - public void testWriteBatchSizeValidation() - { - assertThatThrownBy(() -> makeConfig(ImmutableMap.of("write.batch-size", "-42"))) - .hasMessageContaining("write.batch-size: must be greater than or equal to 1"); - - assertThatThrownBy(() -> makeConfig(ImmutableMap.of("write.batch-size", "0"))) - .hasMessageContaining("write.batch-size: must be greater than or equal to 1"); - - assertThatCode(() -> makeConfig(ImmutableMap.of("write.batch-size", "1"))) - .doesNotThrowAnyException(); - - assertThatCode(() -> makeConfig(ImmutableMap.of("write.batch-size", "42"))) - .doesNotThrowAnyException(); - } - - private static JdbcMetadataConfig makeConfig(Map props) - { - return new ConfigurationFactory(props).build(JdbcMetadataConfig.class); - } } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcWriteConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcWriteConfig.java new file mode 100644 index 000000000000..f8ba41336be0 --- /dev/null +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcWriteConfig.java @@ -0,0 +1,73 @@ +/* + * 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 io.trino.plugin.jdbc; + +import com.google.common.collect.ImmutableMap; +import io.airlift.configuration.ConfigurationFactory; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestJdbcWriteConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(JdbcWriteConfig.class) + .setWriteBatchSize(1000) + .setNonTransactionalInsert(false)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("write.batch-size", "24") + .put("insert.non-transactional-insert.enabled", "true") + .build(); + + JdbcWriteConfig expected = new JdbcWriteConfig() + .setWriteBatchSize(24) + .setNonTransactionalInsert(true); + + assertFullMapping(properties, expected); + } + + @Test + public void testWriteBatchSizeValidation() + { + assertThatThrownBy(() -> makeConfig(ImmutableMap.of("write.batch-size", "-42"))) + .hasMessageContaining("write.batch-size: must be greater than or equal to 1"); + + assertThatThrownBy(() -> makeConfig(ImmutableMap.of("write.batch-size", "0"))) + .hasMessageContaining("write.batch-size: must be greater than or equal to 1"); + + assertThatCode(() -> makeConfig(ImmutableMap.of("write.batch-size", "1"))) + .doesNotThrowAnyException(); + + assertThatCode(() -> makeConfig(ImmutableMap.of("write.batch-size", "42"))) + .doesNotThrowAnyException(); + } + + private static JdbcWriteConfig makeConfig(Map props) + { + return new ConfigurationFactory(props).build(JdbcWriteConfig.class); + } +} diff --git a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java index 8a5bb35029b7..ccdd3d5eec6d 100644 --- a/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java +++ b/plugin/trino-phoenix/src/main/java/io/trino/plugin/phoenix/PhoenixClientModule.java @@ -35,6 +35,8 @@ import io.trino.plugin.jdbc.JdbcMetadataSessionProperties; import io.trino.plugin.jdbc.JdbcPageSinkProvider; import io.trino.plugin.jdbc.JdbcRecordSetProvider; +import io.trino.plugin.jdbc.JdbcWriteConfig; +import io.trino.plugin.jdbc.JdbcWriteSessionProperties; import io.trino.plugin.jdbc.LazyConnectionFactory; import io.trino.plugin.jdbc.MaxDomainCompactionThreshold; import io.trino.plugin.jdbc.StatsCollecting; @@ -83,8 +85,10 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(TypeHandlingJdbcConfig.class); bindSessionPropertiesProvider(binder, TypeHandlingJdbcSessionProperties.class); bindSessionPropertiesProvider(binder, JdbcMetadataSessionProperties.class); + bindSessionPropertiesProvider(binder, JdbcWriteSessionProperties.class); configBinder(binder).bindConfig(JdbcMetadataConfig.class); + configBinder(binder).bindConfig(JdbcWriteConfig.class); configBinder(binder).bindConfigDefaults(JdbcMetadataConfig.class, config -> config.setAllowDropTable(true)); binder.bind(PhoenixClient.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java index a967e3ad9a26..9c6198f8f03f 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClientModule.java @@ -35,6 +35,8 @@ import io.trino.plugin.jdbc.JdbcMetadataSessionProperties; import io.trino.plugin.jdbc.JdbcPageSinkProvider; import io.trino.plugin.jdbc.JdbcRecordSetProvider; +import io.trino.plugin.jdbc.JdbcWriteConfig; +import io.trino.plugin.jdbc.JdbcWriteSessionProperties; import io.trino.plugin.jdbc.LazyConnectionFactory; import io.trino.plugin.jdbc.MaxDomainCompactionThreshold; import io.trino.plugin.jdbc.StatsCollecting; @@ -83,8 +85,10 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(TypeHandlingJdbcConfig.class); bindSessionPropertiesProvider(binder, TypeHandlingJdbcSessionProperties.class); bindSessionPropertiesProvider(binder, JdbcMetadataSessionProperties.class); + bindSessionPropertiesProvider(binder, JdbcWriteSessionProperties.class); configBinder(binder).bindConfig(JdbcMetadataConfig.class); + configBinder(binder).bindConfig(JdbcWriteConfig.class); configBinder(binder).bindConfigDefaults(JdbcMetadataConfig.class, config -> config.setAllowDropTable(true)); binder.bind(PhoenixClient.class).in(Scopes.SINGLETON);