diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index b9eba5275bb2..b9febc8be3d6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -64,6 +64,8 @@ public class HiveConfig { private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); + private boolean singleStatementWritesOnly; + private DataSize maxSplitSize = DataSize.of(64, MEGABYTE); private int maxPartitionsPerScan = 100_000; private int maxOutstandingSplits = 1_000; @@ -164,6 +166,19 @@ public class HiveConfig private boolean sizeBasedSplitWeightsEnabled = true; private double minimumAssignedSplitWeight = 0.05; + public boolean isSingleStatementWritesOnly() + { + return singleStatementWritesOnly; + } + + @Config("hive.single-statement-writes") + @ConfigDescription("Require transaction to be in auto-commit mode for writes") + public HiveConfig setSingleStatementWritesOnly(boolean singleStatementWritesOnly) + { + this.singleStatementWritesOnly = singleStatementWritesOnly; + return this; + } + public int getMaxInitialSplits() { return maxInitialSplits; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java index 32a62eb3953b..0b1d10943025 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnector.java @@ -66,6 +66,7 @@ public class HiveConnector private final ClassLoader classLoader; private final HiveTransactionManager transactionManager; + private final boolean singleStatementWritesOnly; public HiveConnector( LifeCycleManager lifeCycleManager, @@ -84,6 +85,7 @@ public HiveConnector( List> analyzeProperties, List> materializedViewProperties, Optional accessControl, + boolean singleStatementWritesOnly, ClassLoader classLoader) { this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); @@ -104,6 +106,7 @@ public HiveConnector( this.analyzeProperties = ImmutableList.copyOf(requireNonNull(analyzeProperties, "analyzeProperties is null")); this.materializedViewProperties = requireNonNull(materializedViewProperties, "materializedViewProperties is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.singleStatementWritesOnly = singleStatementWritesOnly; this.classLoader = requireNonNull(classLoader, "classLoader is null"); } @@ -196,7 +199,7 @@ public ConnectorAccessControl getAccessControl() @Override public boolean isSingleStatementWritesOnly() { - return false; + return singleStatementWritesOnly; } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java index aebdc56a4088..18398eecb6c6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java @@ -161,6 +161,7 @@ public static Connector createConnector(String catalogName, Map hiveAnalyzeProperties.getAnalyzeProperties(), hiveMaterializedViewPropertiesProvider.getMaterializedViewProperties(), hiveAccessControl, + injector.getInstance(HiveConfig.class).isSingleStatementWritesOnly(), classLoader); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java index 2921d26f7911..1ee9564b365e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java @@ -36,6 +36,7 @@ public class TestHiveConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(HiveConfig.class) + .setSingleStatementWritesOnly(false) .setMaxSplitSize(DataSize.of(64, Unit.MEGABYTE)) .setMaxPartitionsPerScan(100_000) .setMaxOutstandingSplits(1_000) @@ -115,6 +116,7 @@ public void testDefaults() public void testExplicitPropertyMappings() { Map properties = new ImmutableMap.Builder() + .put("hive.single-statement-writes", "true") .put("hive.max-split-size", "256MB") .put("hive.max-partitions-per-scan", "123") .put("hive.max-outstanding-splits", "10") @@ -191,6 +193,7 @@ public void testExplicitPropertyMappings() .buildOrThrow(); HiveConfig expected = new HiveConfig() + .setSingleStatementWritesOnly(true) .setMaxSplitSize(DataSize.of(256, Unit.MEGABYTE)) .setMaxPartitionsPerScan(123) .setMaxOutstandingSplits(10)