From a68ed8118a8b07ebb7f8c0cf83770dc90b3da1ef Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Nov 2024 17:04:41 +0100 Subject: [PATCH] Remove airlift's EventClient usage --- plugin/trino-delta-lake/pom.xml | 5 - .../deltalake/DeltaLakeConnectorFactory.java | 2 - plugin/trino-hive/pom.xml | 5 - .../plugin/hive/HiveConnectorFactory.java | 2 - .../io/trino/plugin/hive/HiveEventClient.java | 39 ---- .../java/io/trino/plugin/hive/HiveModule.java | 3 - .../plugin/hive/HivePageSinkProvider.java | 14 -- .../java/io/trino/plugin/hive/HiveWriter.java | 8 +- .../trino/plugin/hive/HiveWriterFactory.java | 59 ------ .../plugin/hive/WriteCompletedEvent.java | 171 ------------------ .../trino/plugin/hive/TestHivePageSink.java | 5 - plugin/trino-hudi/pom.xml | 5 - .../plugin/hudi/HudiConnectorFactory.java | 2 - plugin/trino-iceberg/pom.xml | 5 - .../iceberg/IcebergConnectorFactory.java | 2 - 15 files changed, 1 insertion(+), 326 deletions(-) delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveEventClient.java delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/WriteCompletedEvent.java diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index 4c531b6748087..c4382bf4ff362 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -65,11 +65,6 @@ configuration - - io.airlift - event - - io.airlift json diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConnectorFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConnectorFactory.java index 8dcde95ece248..93f11c4016749 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConnectorFactory.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConnectorFactory.java @@ -19,7 +19,6 @@ import com.google.inject.Module; import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; @@ -91,7 +90,6 @@ public static Connector createConnector( ClassLoader classLoader = DeltaLakeConnectorFactory.class.getClassLoader(); try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { Bootstrap app = new Bootstrap( - new EventModule(), new MBeanModule(), new ConnectorObjectNameGeneratorModule("io.trino.plugin.deltalake", "trino.plugin.deltalake"), new JsonModule(), diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index ec589147d9bf1..df1aa05e6ae91 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -79,11 +79,6 @@ configuration - - io.airlift - event - - io.airlift json diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnectorFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnectorFactory.java index 4fcc498fa561b..2b1eba0220f2d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnectorFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConnectorFactory.java @@ -19,7 +19,6 @@ import com.google.inject.Scopes; import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; @@ -96,7 +95,6 @@ public static Connector createConnector( try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { Bootstrap app = new Bootstrap( new CatalogNameModule(catalogName), - new EventModule(), new MBeanModule(), new ConnectorObjectNameGeneratorModule("io.trino.plugin.hive", "trino.plugin.hive"), new JsonModule(), diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveEventClient.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveEventClient.java deleted file mode 100644 index c5e3b4f11df51..0000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveEventClient.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.hive; - -import io.airlift.event.client.AbstractEventClient; -import io.airlift.log.Logger; - -public class HiveEventClient - extends AbstractEventClient -{ - private static final Logger log = Logger.get(HiveEventClient.class); - - @Override - public void postEvent(T event) - { - if (!(event instanceof WriteCompletedEvent writeCompletedEvent)) { - return; - } - log.debug("File created: query: %s, schema: %s, table: %s, partition: '%s', format: %s, size: %s, path: %s", - writeCompletedEvent.getQueryId(), - writeCompletedEvent.getSchemaName(), - writeCompletedEvent.getTableName(), - writeCompletedEvent.getPartitionName(), - writeCompletedEvent.getStorageFormat(), - writeCompletedEvent.getBytes(), - writeCompletedEvent.getPath()); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java index af24f09cc1094..3ad1ad2fb1280 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveModule.java @@ -21,7 +21,6 @@ import com.google.inject.Scopes; import com.google.inject.Singleton; import com.google.inject.multibindings.Multibinder; -import io.airlift.event.client.EventClient; import io.trino.plugin.hive.avro.AvroFileWriterFactory; import io.trino.plugin.hive.avro.AvroPageSourceFactory; import io.trino.plugin.hive.fs.CachingDirectoryLister; @@ -95,8 +94,6 @@ public void configure(Binder binder) binder.bind(HiveWriterStats.class).in(Scopes.SINGLETON); newExporter(binder).export(HiveWriterStats.class).withGeneratedName(); - - newSetBinder(binder, EventClient.class).addBinding().to(HiveEventClient.class).in(Scopes.SINGLETON); binder.bind(HivePartitionManager.class).in(Scopes.SINGLETON); binder.bind(LocationService.class).to(HiveLocationService.class).in(Scopes.SINGLETON); Multibinder systemTableProviders = newSetBinder(binder, SystemTableProvider.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java index b65499bb824c6..e8e987b1c7285 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSinkProvider.java @@ -18,7 +18,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; -import io.airlift.event.client.EventClient; import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; import io.trino.filesystem.TrinoFileSystemFactory; @@ -26,7 +25,6 @@ import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.HivePageSinkMetadataProvider; import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; -import io.trino.spi.NodeManager; import io.trino.spi.PageIndexerFactory; import io.trino.spi.PageSorter; import io.trino.spi.connector.ConnectorInsertTableHandle; @@ -69,9 +67,6 @@ public class HivePageSinkProvider private final LocationService locationService; private final ListeningExecutorService writeVerificationExecutor; private final JsonCodec partitionUpdateCodec; - private final NodeManager nodeManager; - private final EventClient eventClient; - private final HiveSessionProperties hiveSessionProperties; private final HiveWriterStats hiveWriterStats; private final long perTransactionMetastoreCacheMaximumSize; private final boolean temporaryStagingDirectoryEnabled; @@ -89,9 +84,6 @@ public HivePageSinkProvider( SortingFileWriterConfig sortingFileWriterConfig, LocationService locationService, JsonCodec partitionUpdateCodec, - NodeManager nodeManager, - EventClient eventClient, - HiveSessionProperties hiveSessionProperties, HiveWriterStats hiveWriterStats) { this.fileWriterFactories = ImmutableSet.copyOf(requireNonNull(fileWriterFactories, "fileWriterFactories is null")); @@ -106,9 +98,6 @@ public HivePageSinkProvider( this.locationService = requireNonNull(locationService, "locationService is null"); this.writeVerificationExecutor = listeningDecorator(newFixedThreadPool(config.getWriteValidationThreads(), daemonThreadsNamed("hive-write-validation-%s"))); this.partitionUpdateCodec = requireNonNull(partitionUpdateCodec, "partitionUpdateCodec is null"); - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); - this.eventClient = requireNonNull(eventClient, "eventClient is null"); - this.hiveSessionProperties = requireNonNull(hiveSessionProperties, "hiveSessionProperties is null"); this.hiveWriterStats = requireNonNull(hiveWriterStats, "hiveWriterStats is null"); this.perTransactionMetastoreCacheMaximumSize = config.getPerTransactionMetastoreCacheMaximumSize(); this.temporaryStagingDirectoryEnabled = config.isTemporaryStagingDirectoryEnabled(); @@ -178,9 +167,6 @@ private HivePageSink createPageSink(HiveWritableTableHandle handle, boolean isCr writerSortBufferSize, maxOpenSortFiles, session, - nodeManager, - eventClient, - hiveSessionProperties, hiveWriterStats, temporaryStagingDirectoryEnabled, temporaryStagingDirectoryPath); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriter.java index f8d7b66b2a1b9..d45ed32e261a2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriter.java @@ -19,7 +19,6 @@ import java.io.Closeable; import java.util.Optional; -import java.util.function.Consumer; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; @@ -32,7 +31,6 @@ public class HiveWriter private final String fileName; private final String writePath; private final String targetPath; - private final Consumer onCommit; private final HiveWriterStats hiveWriterStats; private long rowCount; @@ -45,7 +43,6 @@ public HiveWriter( String fileName, String writePath, String targetPath, - Consumer onCommit, HiveWriterStats hiveWriterStats) { this.fileWriter = requireNonNull(fileWriter, "fileWriter is null"); @@ -54,7 +51,6 @@ public HiveWriter( this.fileName = requireNonNull(fileName, "fileName is null"); this.writePath = requireNonNull(writePath, "writePath is null"); this.targetPath = requireNonNull(targetPath, "targetPath is null"); - this.onCommit = requireNonNull(onCommit, "onCommit is null"); this.hiveWriterStats = requireNonNull(hiveWriterStats, "hiveWriterStats is null"); } @@ -89,9 +85,7 @@ public void append(Page dataPage) public Closeable commit() { - Closeable rollbackAction = fileWriter.commit(); - onCommit.accept(this); - return rollbackAction; + return fileWriter.commit(); } long getValidationCpuNanos() diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java index 656f90843f3e6..28d60c873c1ac 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java @@ -19,7 +19,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; -import io.airlift.event.client.EventClient; import io.airlift.units.DataSize; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; @@ -39,7 +38,6 @@ import io.trino.plugin.hive.metastore.HivePageSinkMetadataProvider; import io.trino.plugin.hive.orc.OrcFileWriterFactory; import io.trino.plugin.hive.util.HiveWriteUtils; -import io.trino.spi.NodeManager; import io.trino.spi.Page; import io.trino.spi.PageSorter; import io.trino.spi.TrinoException; @@ -50,25 +48,20 @@ import io.trino.spi.type.TypeManager; import java.io.IOException; -import java.security.Principal; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalInt; import java.util.Set; import java.util.UUID; -import java.util.function.Consumer; import java.util.regex.Matcher; import java.util.regex.Pattern; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static com.google.common.collect.Maps.immutableEntry; import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.hive.formats.HiveClassNames.HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS; import static io.trino.metastore.AcidOperation.CREATE_TABLE; @@ -147,11 +140,6 @@ public class HiveWriterFactory private final ConnectorSession session; private final OptionalInt bucketCount; private final List sortedBy; - - private final NodeManager nodeManager; - private final EventClient eventClient; - private final Map sessionProperties; - private final HiveWriterStats hiveWriterStats; private final Optional rowType; private final Optional hiveRowtype; @@ -178,9 +166,6 @@ public HiveWriterFactory( DataSize sortBufferSize, int maxOpenSortFiles, ConnectorSession session, - NodeManager nodeManager, - EventClient eventClient, - HiveSessionProperties hiveSessionProperties, HiveWriterStats hiveWriterStats, boolean sortedWritingTempStagingPathEnabled, String sortedWritingTempStagingPath) @@ -256,20 +241,7 @@ public HiveWriterFactory( } this.sortedBy = ImmutableList.copyOf(requireNonNull(sortedBy, "sortedBy is null")); - this.session = requireNonNull(session, "session is null"); - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); - this.eventClient = requireNonNull(eventClient, "eventClient is null"); - - requireNonNull(hiveSessionProperties, "hiveSessionProperties is null"); - this.sessionProperties = hiveSessionProperties.getSessionProperties().stream() - .map(propertyMetadata -> immutableEntry( - propertyMetadata.getName(), - session.getProperty(propertyMetadata.getName(), propertyMetadata.getJavaType()))) - // The session properties collected here are used for events only. Filter out nulls to avoid problems with downstream consumers - .filter(entry -> entry.getValue() != null) - .collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().toString())); - this.hiveWriterStats = requireNonNull(hiveWriterStats, "hiveWriterStats is null"); } @@ -505,36 +477,6 @@ public HiveWriter createWriter(Page partitionColumns, int position, OptionalInt throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, "Writing not supported for " + outputStorageFormat); } - String writePath = path.toString(); - String writerImplementation = hiveFileWriter.getClass().getName(); - - Consumer onCommit = hiveWriter -> { - Optional size; - try { - size = Optional.of(hiveWriter.getWrittenBytes()); - } - catch (RuntimeException e) { - // Do not fail the query if file system is not available - size = Optional.empty(); - } - - eventClient.post(new WriteCompletedEvent( - session.getQueryId(), - writePath, - schemaName, - tableName, - partitionName.orElse(null), - outputStorageFormat.getOutputFormat(), - writerImplementation, - nodeManager.getCurrentNode().getVersion(), - nodeManager.getCurrentNode().getHost(), - session.getIdentity().getPrincipal().map(Principal::getName).orElse(null), - nodeManager.getEnvironment(), - sessionProperties, - size.orElse(null), - hiveWriter.getRowCount())); - }; - if (!sortedBy.isEmpty()) { Location tempFilePath; if (sortedWritingTempStagingPathEnabled) { @@ -587,7 +529,6 @@ public HiveWriter createWriter(Page partitionColumns, int position, OptionalInt path.fileName(), writeInfo.writePath().toString(), writeInfo.targetPath().toString(), - onCommit, hiveWriterStats); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/WriteCompletedEvent.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/WriteCompletedEvent.java deleted file mode 100644 index 1c3d7df154946..0000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/WriteCompletedEvent.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.hive; - -import com.google.errorprone.annotations.Immutable; -import io.airlift.event.client.EventField; -import io.airlift.event.client.EventField.EventFieldMapping; -import io.airlift.event.client.EventType; -import jakarta.annotation.Nullable; - -import java.time.Instant; -import java.util.Map; - -import static java.util.Objects.requireNonNull; - -@Immutable -@EventType("WriteCompletedEvent") -public class WriteCompletedEvent -{ - private final String queryId; - private final String path; - private final String schemaName; - private final String tableName; - private final String partitionName; - private final String storageFormat; - private final String writerImplementation; - private final String trinoVersion; - private final String host; - private final String principal; - private final String environment; - private final Map sessionProperties; - private final Long bytes; - private final long rows; - private final Instant timestamp = Instant.now(); - - public WriteCompletedEvent( - String queryId, - String path, - String schemaName, - String tableName, - @Nullable String partitionName, - String storageFormat, - String writerImplementation, - String trinoVersion, - String serverAddress, - @Nullable String principal, - String environment, - Map sessionProperties, - @Nullable Long bytes, - long rows) - { - this.queryId = requireNonNull(queryId, "queryId is null"); - this.path = requireNonNull(path, "path is null"); - this.schemaName = requireNonNull(schemaName, "schemaName is null"); - this.tableName = requireNonNull(tableName, "tableName is null"); - this.partitionName = partitionName; - this.storageFormat = requireNonNull(storageFormat, "storageFormat is null"); - this.writerImplementation = requireNonNull(writerImplementation, "writerImplementation is null"); - this.trinoVersion = requireNonNull(trinoVersion, "trinoVersion is null"); - this.host = requireNonNull(serverAddress, "serverAddress is null"); - this.principal = principal; - this.environment = requireNonNull(environment, "environment is null"); - this.sessionProperties = requireNonNull(sessionProperties, "sessionProperties is null"); - this.bytes = bytes; - this.rows = rows; - } - - @EventField - public String getQueryId() - { - return queryId; - } - - @EventField - public String getPath() - { - return path; - } - - @EventField - public String getSchemaName() - { - return schemaName; - } - - @EventField - public String getTableName() - { - return tableName; - } - - @Nullable - @EventField - public String getPartitionName() - { - return partitionName; - } - - @EventField - public String getStorageFormat() - { - return storageFormat; - } - - @EventField - public String getWriterImplementation() - { - return writerImplementation; - } - - @EventField - public String getTrinoVersion() - { - return trinoVersion; - } - - @EventField(fieldMapping = EventFieldMapping.HOST) - public String getHost() - { - return host; - } - - @Nullable - @EventField - public String getPrincipal() - { - return principal; - } - - @EventField - public String getEnvironment() - { - return environment; - } - - @EventField - public Map getSessionProperties() - { - return sessionProperties; - } - - @Nullable - @EventField - public Long getBytes() - { - return bytes; - } - - @EventField - public long getRows() - { - return rows; - } - - @EventField(fieldMapping = EventFieldMapping.TIMESTAMP) - public Instant getTimestamp() - { - return timestamp; - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java index 5143e36fad1f3..b0bb0e9854ffd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java @@ -43,7 +43,6 @@ import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; import io.trino.testing.MaterializedResult; -import io.trino.testing.TestingNodeManager; import io.trino.tpch.LineItem; import io.trino.tpch.LineItemColumn; import io.trino.tpch.LineItemGenerator; @@ -77,7 +76,6 @@ import static io.trino.plugin.hive.HiveTestUtils.getDefaultHiveFileWriterFactories; import static io.trino.plugin.hive.HiveTestUtils.getDefaultHivePageSourceFactories; import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; -import static io.trino.plugin.hive.HiveTestUtils.getHiveSessionProperties; import static io.trino.plugin.hive.LocationHandle.WriteMode.DIRECT_TO_TARGET_NEW_DIRECTORY; import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; @@ -394,9 +392,6 @@ private static ConnectorPageSink createPageSink( sortingFileWriterConfig, new HiveLocationService(HDFS_FILE_SYSTEM_FACTORY, config), partitionUpdateCodec, - new TestingNodeManager("fake-environment"), - new HiveEventClient(), - getHiveSessionProperties(config), stats); return provider.createPageSink(transaction, getHiveSession(config), handle, TESTING_PAGE_SINK_ID); } diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index b2097c4829195..01092a73f4380 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -55,11 +55,6 @@ configuration - - io.airlift - event - - io.airlift json diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java index dc98f38f4c2d4..f7dd96d767991 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java @@ -19,7 +19,6 @@ import com.google.inject.Module; import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; @@ -75,7 +74,6 @@ public static Connector createConnector( ClassLoader classLoader = HudiConnectorFactory.class.getClassLoader(); try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { Bootstrap app = new Bootstrap( - new EventModule(), new MBeanModule(), new JsonModule(), new HudiModule(), diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 593419fee464a..5f93805d455ef 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -75,11 +75,6 @@ configuration - - io.airlift - event - - io.airlift json diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConnectorFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConnectorFactory.java index b554bafbd16d8..b5489aca7ae78 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConnectorFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConnectorFactory.java @@ -20,7 +20,6 @@ import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; @@ -91,7 +90,6 @@ public static Connector createConnector( ClassLoader classLoader = IcebergConnectorFactory.class.getClassLoader(); try (ThreadContextClassLoader _ = new ThreadContextClassLoader(classLoader)) { Bootstrap app = new Bootstrap( - new EventModule(), new MBeanModule(), new ConnectorObjectNameGeneratorModule("io.trino.plugin.iceberg", "trino.plugin.iceberg"), new JsonModule(),