From 41932c63f398a262c53c6aae6ef21320017d159a Mon Sep 17 00:00:00 2001 From: avv Date: Tue, 26 Nov 2024 12:36:27 +0500 Subject: [PATCH] ADH-5241 - implemented trino-adb plugin (for reading) - done refactoring --- .../arenadata/coordinator/etc/log.properties | 2 +- .../arenadata/worker/etc/log.properties | 2 +- .../io/trino/plugin/adb/AdbPluginConfig.java | 36 +- .../java/io/trino/plugin/adb/TypeUtil.java | 4 +- .../plugin/adb/connector/AdbClientModule.java | 13 +- .../adb/connector/AdbSessionProperties.java | 12 +- .../plugin/adb/connector/AdbSqlClient.java | 1138 +---------------- .../datatype/mapper/DataTypeMapper.java | 39 + .../datatype/mapper/DataTypeMapperImpl.java | 920 +++++++++++++ .../adb/connector/decode/ColumnValue.java | 18 + .../adb/connector/decode/RowDecoder.java | 24 + .../connector/decode/RowDecoderFactory.java | 24 + .../connector/decode/csv/CsvRowDecoder.java | 403 ++++++ .../decode/csv/CsvRowDecoderFactory.java | 31 + .../connector/encode/AbstractRowEncoder.java | 10 +- .../{EncoderFormat.java => DataFormat.java} | 2 +- ...coderConfig.java => DataFormatConfig.java} | 4 +- ...coderModule.java => DataFormatModule.java} | 12 +- .../adb/connector/encode/RowEncoder.java | 2 +- ...ncoderConfig.java => CsvFormatConfig.java} | 30 +- .../connector/encode/csv/CsvRowEncoder.java | 10 +- .../encode/csv/CsvRowEncoderFactory.java | 16 +- .../gpfdist/AbstractContextManager.java | 48 + .../AbstractDataTransferQueryExecutor.java | 82 ++ .../AbstractExternalTableQueryFactory.java | 47 + .../protocol/gpfdist/ConnectorRow.java | 25 + .../connector/protocol/gpfdist/Context.java | 23 + .../protocol/gpfdist/ContextManager.java | 27 + .../CreateExternalTableQueryFactory.java | 1 + .../protocol/gpfdist/GpfdistModule.java | 44 +- .../protocol/gpfdist/GpfdistUtil.java | 26 + .../gpfdist/InsertDataQueryFactory.java | 7 +- ...eateReadableExternalTableQueryFactory.java | 56 - .../gpfdist/load/GpfdistPacketBuilder.java | 5 + .../gpfdist/load/context/WriteContext.java | 27 +- .../load/context/WriteContextManager.java | 12 +- .../load/context/WriteContextManagerImpl.java | 56 - .../GpfdistPageProcessor.java | 24 +- .../GpfdistPageSerializer.java | 8 +- .../GpfdistPageSink.java | 34 +- .../GpfdistPageSinkProvider.java | 60 +- .../GpfdistDataTransferQueryExecutor.java | 113 -- ...ateReadableExternalTableQueryFactory.java} | 14 +- .../GpfdistLoadDataTransferQueryExecutor.java | 67 + ...sertDataFromExternalTableQueryFactory.java | 17 +- .../ExternalTableFormatConfig.java | 6 +- .../ExternalTableFormatConfigFactory.java | 2 - .../ExternalTableFormatConfigFactoryImpl.java | 27 +- .../{ => metadata}/ExternalTableType.java | 2 +- .../gpfdist/metadata/ExternalTableUtil.java | 29 + .../gpfdist/metadata/GpfdistLoadMetadata.java | 20 +- .../metadata/GpfdistLoadMetadataFactory.java | 79 +- .../GpfdistLoadMetadataFactoryImpl.java | 80 ++ .../metadata/GpfdistLocationFactory.java | 19 + .../metadata/GpfdistLocationFactoryImpl.java | 62 + .../gpfdist/metadata/GpfdistMetadata.java | 41 +- .../metadata/GpfdistUnloadMetadata.java | 59 + ...java => GpfdistUnloadMetadataFactory.java} | 18 +- .../GpfdistUnloadMetadataFactoryImpl.java | 94 ++ .../gpfdist/server/GpfdistResource.java | 163 ++- .../{ => request}/GpfdistReadableRequest.java | 34 +- .../{ => request}/GpfdistRequestHeader.java | 5 +- .../request/GpfdistWritableRequest.java | 181 +++ .../gpfdist/unload/AdbRecordSetProvider.java | 48 - ...yFactory.java => GpfdistConnectorRow.java} | 17 +- .../gpfdist/unload/InputDataProcessor.java | 23 + .../unload/InputDataProcessorFactory.java | 21 + .../gpfdist/unload/ProcessingDataResult.java | 20 + .../gpfdist/unload/RowProcessingService.java | 27 + .../gpfdist/unload/SegmentRequestStatus.java | 22 + .../gpfdist/unload/WithEstimatedSize.java | 19 + .../gpfdist/unload/context/ReadContext.java | 104 ++ .../unload/context/ReadContextManager.java | 21 + .../GpfdistBufferedRowProcessingService.java | 104 ++ .../process/GpfdistCsvDataProcessor.java | 97 ++ .../GpfdistInputDataProcessorFactory.java | 45 + .../unload/process/GpfdistRecordCursor.java | 232 ++++ .../unload/process/GpfdistRecordSet.java | 57 + .../process/GpfdistRecordSetProvider.java | 141 ++ .../GpfdistSegmentRequestProcessor.java | 79 ++ ...eateWritableExternalTableQueryFactory.java | 56 + ...pfdistUnloadDataTransferQueryExecutor.java | 85 ++ ...InsertDataToExternalTableQueryFactory.java | 74 ++ .../connector/table/SplitSourceManager.java | 23 + .../table/SplitSourceManagerImpl.java | 135 ++ .../connector/table/StatisticsManager.java | 26 + .../table/StatisticsManagerImpl.java | 255 ++++ .../trino-server-dev/etc/config.properties | 1 + testing/trino-server-dev/etc/log.properties | 2 +- 89 files changed, 4547 insertions(+), 1683 deletions(-) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapper.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapperImpl.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/ColumnValue.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoder.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoderFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoder.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoderFactory.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/{EncoderFormat.java => DataFormat.java} (95%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/{EncoderConfig.java => DataFormatConfig.java} (89%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/{EncoderModule.java => DataFormatModule.java} (67%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/{CsvEncoderConfig.java => CsvFormatConfig.java} (63%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractContextManager.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractDataTransferQueryExecutor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractExternalTableQueryFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ConnectorRow.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/Context.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ContextManager.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistUtil.java delete mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/CreateReadableExternalTableQueryFactory.java delete mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManagerImpl.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/{processing => process}/GpfdistPageProcessor.java (84%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/{processing => process}/GpfdistPageSerializer.java (89%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/{processing => process}/GpfdistPageSink.java (88%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/{ => process}/GpfdistPageSinkProvider.java (70%) delete mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistDataTransferQueryExecutor.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/{unload/CreateWritableExternalTableQueryFactory.java => load/query/CreateReadableExternalTableQueryFactory.java} (64%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/GpfdistLoadDataTransferQueryExecutor.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/{ => query}/InsertDataFromExternalTableQueryFactory.java (75%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/{ => metadata}/ExternalTableFormatConfig.java (78%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/{ => metadata}/ExternalTableType.java (90%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableUtil.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactoryImpl.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactoryImpl.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadata.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/{GpfdistMetadataFactory.java => GpfdistUnloadMetadataFactory.java} (60%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactoryImpl.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/{ => request}/GpfdistReadableRequest.java (90%) rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/{ => request}/GpfdistRequestHeader.java (86%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistWritableRequest.java delete mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/AdbRecordSetProvider.java rename plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/{InsertDataToExternalTableQueryFactory.java => GpfdistConnectorRow.java} (55%) create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessorFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/ProcessingDataResult.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/RowProcessingService.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/SegmentRequestStatus.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/WithEstimatedSize.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContext.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContextManager.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistBufferedRowProcessingService.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistCsvDataProcessor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistInputDataProcessorFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordCursor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSet.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSetProvider.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistSegmentRequestProcessor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/CreateWritableExternalTableQueryFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/GpfdistUnloadDataTransferQueryExecutor.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/InsertDataToExternalTableQueryFactory.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManager.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManagerImpl.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManager.java create mode 100644 plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManagerImpl.java diff --git a/core/docker/arenadata/coordinator/etc/log.properties b/core/docker/arenadata/coordinator/etc/log.properties index abee45ebcde7..9b0788d5a139 100644 --- a/core/docker/arenadata/coordinator/etc/log.properties +++ b/core/docker/arenadata/coordinator/etc/log.properties @@ -1,2 +1,2 @@ # Enable verbose logging from Trino -#io.trino=DEBUG +io.trino=DEBUG diff --git a/core/docker/arenadata/worker/etc/log.properties b/core/docker/arenadata/worker/etc/log.properties index abee45ebcde7..9b0788d5a139 100644 --- a/core/docker/arenadata/worker/etc/log.properties +++ b/core/docker/arenadata/worker/etc/log.properties @@ -1,2 +1,2 @@ # Enable verbose logging from Trino -#io.trino=DEBUG +io.trino=DEBUG diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/AdbPluginConfig.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/AdbPluginConfig.java index f3a6488eed53..2dc11487d2ef 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/AdbPluginConfig.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/AdbPluginConfig.java @@ -16,6 +16,7 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import io.airlift.units.DataSize; +import io.airlift.units.Duration; import io.airlift.units.MinDataSize; import io.trino.plugin.adb.connector.protocol.TransferDataProtocol; import jakarta.validation.constraints.Min; @@ -23,11 +24,14 @@ public class AdbPluginConfig { + public static final String IDENTIFIER_QUOTE = "\""; private AdbPluginConfig.ArrayMapping arrayMapping = AdbPluginConfig.ArrayMapping.DISABLED; private int maxScanParallelism = 1; private boolean includeSystemTables; - private DataSize writeBufferSize = DataSize.of(16L, DataSize.Unit.MEGABYTE); + private DataSize writeBufferSize = DataSize.of(64L, DataSize.Unit.MEGABYTE); + private DataSize readBufferSize = DataSize.of(64L, DataSize.Unit.MEGABYTE); private final TransferDataProtocol dataProtocol = TransferDataProtocol.GPFDIST; + private Duration gpfdistRetryTimeout; public TransferDataProtocol getDataProtocol() { @@ -81,13 +85,41 @@ public DataSize getWriteBufferSize() } @Config("adb.connector.write-buffer-size") - @ConfigDescription("Maximum amount of memory that could be allocated per sink when executing write queries. Defaults to 16MB") + @ConfigDescription("Maximum amount of memory that could be allocated per sink when executing write queries. Defaults to 64MB") public AdbPluginConfig setWriteBufferSize(DataSize writeBufferSize) { this.writeBufferSize = writeBufferSize; return this; } + @MinDataSize("1kB") + @NotNull + public DataSize getReadBufferSize() + { + return readBufferSize; + } + + @Config("adb.connector.read-buffer-size") + @ConfigDescription("Maximum amount of memory that could be allocated per record cursor when executing read queries. Defaults to 64MB") + public AdbPluginConfig setReadBufferSize(DataSize readBufferSize) + { + this.readBufferSize = readBufferSize; + return this; + } + + public Duration getGpfdistRetryTimeout() + { + return this.gpfdistRetryTimeout; + } + + @Config("adb.gpfdist.retry-timeout") + @ConfigDescription("Value of adb gpfdist_retry_timeout property. Defaults to null (use adb defaults)") + public AdbPluginConfig setGpfdistRetryTimeout(Duration gpfdistRetryTimeout) + { + this.gpfdistRetryTimeout = gpfdistRetryTimeout; + return this; + } + public static enum ArrayMapping { DISABLED, diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/TypeUtil.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/TypeUtil.java index 18843d3670c9..006a4a23e69c 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/TypeUtil.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/TypeUtil.java @@ -16,7 +16,7 @@ import com.google.common.primitives.Shorts; import com.google.common.primitives.SignedBytes; import io.airlift.slice.Slice; -import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.datatype.mapper.DataTypeMapper; import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import io.trino.spi.connector.ConnectorSession; @@ -131,7 +131,7 @@ public static Object[] getJdbcObjectArray(ConnectorSession session, Type element return valuesArray; } - public static String getArrayElementPgTypeName(ConnectorSession session, AdbSqlClient client, Type elementType) + public static String getArrayElementPgTypeName(ConnectorSession session, DataTypeMapper client, Type elementType) { if (DOUBLE.equals(elementType)) { return "float8"; diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbClientModule.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbClientModule.java index e08d1a0b19d0..6b50552ea919 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbClientModule.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbClientModule.java @@ -20,13 +20,19 @@ import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.configuration.ConfigBinder; import io.trino.plugin.adb.AdbPluginConfig; -import io.trino.plugin.adb.connector.encode.EncoderModule; +import io.trino.plugin.adb.connector.datatype.mapper.DataTypeMapper; +import io.trino.plugin.adb.connector.datatype.mapper.DataTypeMapperImpl; +import io.trino.plugin.adb.connector.encode.DataFormatModule; import io.trino.plugin.adb.connector.metadata.AdbMetadataDao; import io.trino.plugin.adb.connector.metadata.impl.AdbMetadataDaoImpl; import io.trino.plugin.adb.connector.protocol.TransferDataProtocol; import io.trino.plugin.adb.connector.protocol.gpfdist.GpfdistModule; import io.trino.plugin.adb.connector.table.AdbCreateTableStorageConfig; import io.trino.plugin.adb.connector.table.AdbTableProperties; +import io.trino.plugin.adb.connector.table.SplitSourceManager; +import io.trino.plugin.adb.connector.table.SplitSourceManagerImpl; +import io.trino.plugin.adb.connector.table.StatisticsManager; +import io.trino.plugin.adb.connector.table.StatisticsManagerImpl; import io.trino.plugin.jdbc.DecimalModule; import io.trino.plugin.jdbc.ForBaseJdbc; import io.trino.plugin.jdbc.JdbcClient; @@ -44,8 +50,11 @@ public class AdbClientModule @Override protected void setup(Binder binder) { - install(new EncoderModule()); + install(new DataFormatModule()); binder.bind(AdbMetadataDao.class).to(AdbMetadataDaoImpl.class).in(Scopes.SINGLETON); + binder.bind(DataTypeMapper.class).to(DataTypeMapperImpl.class).in(Scopes.SINGLETON); + binder.bind(StatisticsManager.class).to(StatisticsManagerImpl.class).in(Scopes.SINGLETON); + binder.bind(SplitSourceManager.class).to(SplitSourceManagerImpl.class).in(Scopes.SINGLETON); binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(AdbSqlClient.class).in(Scopes.SINGLETON); ConfigBinder.configBinder(binder).bindConfig(AdbCreateTableStorageConfig.class); ConfigBinder.configBinder(binder).bindConfig(JdbcStatisticsConfig.class); diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSessionProperties.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSessionProperties.java index 66dd0b03462c..2660b80833ba 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSessionProperties.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSessionProperties.java @@ -15,12 +15,15 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Inject; +import io.airlift.units.Duration; import io.trino.plugin.adb.AdbPluginConfig; +import io.trino.plugin.base.session.PropertyMetadataUtil; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.session.PropertyMetadata; import java.util.List; +import java.util.Optional; public class AdbSessionProperties implements SessionPropertiesProvider @@ -33,7 +36,9 @@ public AdbSessionProperties(AdbPluginConfig config) this.sessionProperties = ImmutableList.of( PropertyMetadata.enumProperty("array_mapping", "Handling of PostgreSql arrays", AdbPluginConfig.ArrayMapping.class, config.getArrayMapping(), false), PropertyMetadata.integerProperty( - "max_scan_parallelism", "Maximum degree of parallelism when scanning tables. Defaults to 1.", config.getMaxScanParallelism(), false)); + "max_scan_parallelism", "Maximum degree of parallelism when scanning tables. Defaults to 1.", config.getMaxScanParallelism(), false), + PropertyMetadataUtil.durationProperty( + "gpfdist_retry_timeout", "Value of adb gpfdist_retry_timeout property", config.getGpfdistRetryTimeout(), false)); } @Override @@ -56,4 +61,9 @@ public static int getMaxScanParallelism(ConnectorSession session) { return session.getProperty("max_scan_parallelism", Integer.class); } + + public static Optional getGpfdistRetryTimeout(ConnectorSession session) + { + return Optional.ofNullable(session.getProperty("gpfdist_retry_timeout", Duration.class)); + } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSqlClient.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSqlClient.java index 3e8936c46af5..510290997927 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSqlClient.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/AdbSqlClient.java @@ -16,39 +16,18 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.math.LongMath; import com.google.inject.Inject; import io.airlift.log.Logger; -import io.airlift.slice.Slice; import io.trino.plugin.adb.AdbPluginConfig; -import io.trino.plugin.adb.connector.datatype.BigintDataType; -import io.trino.plugin.adb.connector.datatype.BitDataType; -import io.trino.plugin.adb.connector.datatype.BooleanDataType; -import io.trino.plugin.adb.connector.datatype.ByteaDataType; -import io.trino.plugin.adb.connector.datatype.CharDataType; import io.trino.plugin.adb.connector.datatype.ColumnDataType; -import io.trino.plugin.adb.connector.datatype.ConnectorDataType; -import io.trino.plugin.adb.connector.datatype.DateDataType; -import io.trino.plugin.adb.connector.datatype.DecimalLongDataType; -import io.trino.plugin.adb.connector.datatype.DecimalShortDataType; -import io.trino.plugin.adb.connector.datatype.DoubleDataType; -import io.trino.plugin.adb.connector.datatype.IntegerDataType; -import io.trino.plugin.adb.connector.datatype.JsonbDataType; -import io.trino.plugin.adb.connector.datatype.MoneyDataType; -import io.trino.plugin.adb.connector.datatype.RealDataType; -import io.trino.plugin.adb.connector.datatype.SmallintDataType; -import io.trino.plugin.adb.connector.datatype.TimeDataType; -import io.trino.plugin.adb.connector.datatype.TimestampLongWithTimeZoneDataType; -import io.trino.plugin.adb.connector.datatype.TimestampShortWithTimeZoneDataType; -import io.trino.plugin.adb.connector.datatype.TimestampWithoutTimeZoneDataType; -import io.trino.plugin.adb.connector.datatype.UnknownDataType; -import io.trino.plugin.adb.connector.datatype.UuidDataType; -import io.trino.plugin.adb.connector.datatype.VarcharDataType; +import io.trino.plugin.adb.connector.datatype.mapper.DataTypeMapper; import io.trino.plugin.adb.connector.metadata.AdbMetadataDao; import io.trino.plugin.adb.connector.table.AdbTableDistributed; import io.trino.plugin.adb.connector.table.AdbTableProperties; import io.trino.plugin.adb.connector.table.AdbTableStorageCompressType; import io.trino.plugin.adb.connector.table.AdbTableStorageOrientation; +import io.trino.plugin.adb.connector.table.SplitSourceManager; +import io.trino.plugin.adb.connector.table.StatisticsManager; import io.trino.plugin.base.aggregation.AggregateFunctionRewriter; import io.trino.plugin.base.aggregation.AggregateFunctionRule; import io.trino.plugin.base.expression.ConnectorExpressionRewriter; @@ -56,32 +35,20 @@ import io.trino.plugin.base.mapping.RemoteIdentifiers; import io.trino.plugin.jdbc.BaseJdbcClient; import io.trino.plugin.jdbc.BaseJdbcConfig; -import io.trino.plugin.jdbc.BooleanReadFunction; import io.trino.plugin.jdbc.ColumnMapping; import io.trino.plugin.jdbc.ConnectionFactory; -import io.trino.plugin.jdbc.DoubleReadFunction; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcErrorCode; import io.trino.plugin.jdbc.JdbcExpression; import io.trino.plugin.jdbc.JdbcJoinCondition; -import io.trino.plugin.jdbc.JdbcMetadataSessionProperties; import io.trino.plugin.jdbc.JdbcOutputTableHandle; import io.trino.plugin.jdbc.JdbcSortItem; import io.trino.plugin.jdbc.JdbcSplit; -import io.trino.plugin.jdbc.JdbcStatisticsConfig; import io.trino.plugin.jdbc.JdbcTableHandle; import io.trino.plugin.jdbc.JdbcTypeHandle; -import io.trino.plugin.jdbc.LongReadFunction; -import io.trino.plugin.jdbc.LongWriteFunction; -import io.trino.plugin.jdbc.ObjectReadFunction; -import io.trino.plugin.jdbc.ObjectWriteFunction; -import io.trino.plugin.jdbc.PredicatePushdownController; import io.trino.plugin.jdbc.PreparedQuery; import io.trino.plugin.jdbc.QueryBuilder; -import io.trino.plugin.jdbc.ReadFunction; import io.trino.plugin.jdbc.RemoteTableName; -import io.trino.plugin.jdbc.SliceReadFunction; -import io.trino.plugin.jdbc.SliceWriteFunction; import io.trino.plugin.jdbc.UnsupportedTypeHandling; import io.trino.plugin.jdbc.WriteMapping; import io.trino.plugin.jdbc.aggregation.ImplementAvgDecimal; @@ -106,58 +73,30 @@ import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.spi.StandardErrorCode; import io.trino.spi.TrinoException; -import io.trino.spi.block.Block; -import io.trino.spi.block.BlockBuilder; import io.trino.spi.connector.AggregateFunction; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.FixedSplitSource; import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.JoinStatistics; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.expression.ConnectorExpression; -import io.trino.spi.predicate.Domain; -import io.trino.spi.statistics.ColumnStatistics; -import io.trino.spi.statistics.Estimate; import io.trino.spi.statistics.TableStatistics; -import io.trino.spi.type.ArrayType; import io.trino.spi.type.CharType; import io.trino.spi.type.DecimalType; -import io.trino.spi.type.Decimals; -import io.trino.spi.type.LongTimestamp; -import io.trino.spi.type.LongTimestampWithTimeZone; -import io.trino.spi.type.TimeType; -import io.trino.spi.type.TimestampType; -import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.Type; -import io.trino.spi.type.TypeManager; -import io.trino.spi.type.TypeSignature; -import io.trino.spi.type.UuidType; import io.trino.spi.type.VarcharType; -import org.jdbi.v3.core.Handle; -import org.jdbi.v3.core.Jdbi; -import org.postgresql.core.TypeInfo; -import org.postgresql.jdbc.PgConnection; - -import java.io.IOException; -import java.math.RoundingMode; -import java.sql.Array; + import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Timestamp; import java.sql.Types; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -167,133 +106,37 @@ import java.util.OptionalInt; import java.util.OptionalLong; import java.util.StringJoiner; -import java.util.UUID; import java.util.function.BiFunction; -import java.util.stream.IntStream; import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Throwables.throwIfInstanceOf; import static com.google.common.base.Verify.verify; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.AS_ARRAY; -import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.AS_JSON; -import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.DISABLED; -import static io.trino.plugin.adb.TypeUtil.TIME_TYPE_FORMATTER; -import static io.trino.plugin.adb.TypeUtil.arrayDepth; -import static io.trino.plugin.adb.TypeUtil.getArrayElementPgTypeName; -import static io.trino.plugin.adb.TypeUtil.getJdbcObjectArray; -import static io.trino.plugin.adb.TypeUtil.toPgTimestamp; +import static io.trino.plugin.adb.AdbPluginConfig.IDENTIFIER_QUOTE; import static io.trino.plugin.adb.connector.AdbSessionProperties.getArrayMapping; -import static io.trino.plugin.base.util.JsonTypeUtil.jsonParse; -import static io.trino.plugin.base.util.JsonTypeUtil.toJsonValue; -import static io.trino.plugin.jdbc.DecimalConfig.DecimalMapping.ALLOW_OVERFLOW; -import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalDefaultScale; -import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRounding; -import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRoundingMode; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; import static io.trino.plugin.jdbc.JdbcJoinPushdownUtil.implementJoinCostAware; -import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN; -import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.booleanColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.charReadFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.dateColumnMappingUsingLocalDate; -import static io.trino.plugin.jdbc.StandardColumnMappings.dateWriteFunctionUsingLocalDate; -import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.doubleColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.doubleWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.fromTrinoTimestamp; -import static io.trino.plugin.jdbc.StandardColumnMappings.integerColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.realColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.timestampReadFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping; -import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryWriteFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.varcharReadFunction; -import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling; -import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR; import static io.trino.plugin.jdbc.UnsupportedTypeHandling.IGNORE; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.spi.type.BooleanType.BOOLEAN; -import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; -import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc; -import static io.trino.spi.type.DateType.DATE; -import static io.trino.spi.type.DecimalType.createDecimalType; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static io.trino.spi.type.IntegerType.INTEGER; -import static io.trino.spi.type.RealType.REAL; -import static io.trino.spi.type.SmallintType.SMALLINT; -import static io.trino.spi.type.TimeType.createTimeType; -import static io.trino.spi.type.TimeZoneKey.UTC_KEY; -import static io.trino.spi.type.TimestampType.createTimestampType; -import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; -import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; -import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_DAY; -import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; -import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_DAY; -import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MICROSECOND; -import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; -import static io.trino.spi.type.Timestamps.round; -import static io.trino.spi.type.TinyintType.TINYINT; -import static io.trino.spi.type.UuidType.javaUuidToTrinoUuid; -import static io.trino.spi.type.UuidType.trinoUuidToJavaUuid; -import static io.trino.spi.type.VarbinaryType.VARBINARY; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static java.lang.Math.floorDiv; -import static java.lang.Math.floorMod; import static java.lang.Math.max; -import static java.lang.Math.min; import static java.lang.String.format; import static java.sql.DatabaseMetaData.columnNoNulls; -import static java.util.Objects.requireNonNull; -import static java.util.function.Function.identity; import static java.util.stream.Collectors.joining; public class AdbSqlClient extends BaseJdbcClient { private static final Logger log = Logger.get(AdbSqlClient.class); - private static final int POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION = 6; - private static final int PRECISION_OF_UNSPECIFIED_DECIMAL = 0; - private static final int ARRAY_RESULT_SET_VALUE_COLUMN = 2; - private static final String IDENTIFIER_QUOTE = "\""; - public static final String COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE = "Column type %s is not supported"; private final AdbMetadataDao metadata; private final List tableTypes; private final Optional fetchSize; - private final Type jsonType; - private final Type uuidType; - private final boolean statisticsEnabled; private final AdbPluginConfig pluginConfig; private final ConnectorExpressionRewriter connectorExpressionRewriter; private final AggregateFunctionRewriter aggregateFunctionRewriter; - private static final PredicatePushdownController ADB_STRING_COLLATION_AWARE_PUSHDOWN = (session, domain) -> { - if (domain.isOnlyNull()) { - return PredicatePushdownController.FULL_PUSHDOWN.apply(session, domain); - } - else if (AdbSessionProperties.isEnableStringPushdownWithCollate(session)) { - return PredicatePushdownController.FULL_PUSHDOWN.apply(session, domain); - } - else { - Domain simplifiedDomain = domain.simplify(JdbcMetadataSessionProperties.getDomainCompactionThreshold(session)); - return !simplifiedDomain.getValues().isDiscreteSet() - ? DISABLE_PUSHDOWN.apply(session, domain) - : PredicatePushdownController.FULL_PUSHDOWN.apply(session, simplifiedDomain); - } - }; + private final DataTypeMapper dataTypeMapper; + private final StatisticsManager statisticsManager; + private final SplitSourceManager splitSourceManager; @Inject public AdbSqlClient(ConnectionFactory connectionFactory, @@ -301,10 +144,11 @@ public AdbSqlClient(ConnectionFactory connectionFactory, BaseJdbcConfig jdbcConfig, IdentifierMapping identifierMapping, RemoteQueryModifier remoteQueryModifier, - TypeManager typeManager, AdbPluginConfig config, - JdbcStatisticsConfig statisticsConfig, - AdbMetadataDao metadata) + AdbMetadataDao metadata, + DataTypeMapper dataTypeMapper, + StatisticsManager statisticsManager, + SplitSourceManager splitSourceManager) { super(IDENTIFIER_QUOTE, connectionFactory, @@ -313,6 +157,9 @@ public AdbSqlClient(ConnectionFactory connectionFactory, identifierMapping, remoteQueryModifier, true); + this.dataTypeMapper = dataTypeMapper; + this.statisticsManager = statisticsManager; + this.splitSourceManager = splitSourceManager; ImmutableList.Builder tableTypes = ImmutableList.builder(); tableTypes.add("TABLE", "PARTITIONED TABLE", "VIEW", "MATERIALIZED VIEW", "FOREIGN TABLE"); if (config.isIncludeSystemTables()) { @@ -320,11 +167,7 @@ public AdbSqlClient(ConnectionFactory connectionFactory, } this.tableTypes = tableTypes.build(); this.metadata = metadata; - //todo could be taken from config, clarify this.fetchSize = Optional.empty(); - this.jsonType = typeManager.getType(new TypeSignature("json")); - this.uuidType = typeManager.getType(new TypeSignature("uuid")); - statisticsEnabled = statisticsConfig.isEnabled(); this.pluginConfig = config; connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .add(new RewriteVariable(this::quoted)) @@ -398,14 +241,26 @@ public void execute(ConnectorSession session, Connection connection, String quer super.execute(session, connection, query); } - public void executeAsPreparedStatement(ConnectorSession session, Connection connection, String sql) + @Override + public PreparedQuery prepareQuery( + ConnectorSession session, + Connection connection, + JdbcTableHandle table, + Optional>> groupingSets, + List columns, + Map columnExpressions, + Optional split) + { + return super.prepareQuery(session, connection, table, groupingSets, columns, columnExpressions, split); + } + + public void executeAsPreparedStatement(ConnectorSession session, Connection connection, PreparedQuery preparedQuery) throws SQLException { - PreparedQuery query = new PreparedQuery(sql, List.of()); try (PreparedStatement preparedStatement = queryBuilder.prepareStatement(this, session, connection, - query, + preparedQuery, Optional.empty())) { preparedStatement.executeUpdate(); } @@ -414,58 +269,7 @@ public void executeAsPreparedStatement(ConnectorSession session, Connection conn @Override public ConnectorSplitSource getSplits(ConnectorSession session, JdbcTableHandle tableHandle) { - int parallelism = getSplitParallelism(session, tableHandle); - List splits = segmentedSplits(parallelism); - return new FixedSplitSource(this.createAdbSplits(session, tableHandle, splits)); - } - - private int getSplitParallelism(ConnectorSession session, JdbcTableHandle tableHandle) - { - if (!tableHandle.isNamedRelation()) { - return 1; - } - else { - int maxParallelism = AdbSessionProperties.getMaxScanParallelism(session); - if (maxParallelism <= 1) { - return 1; - } - else { - boolean segmented = this.metadata.isSegmentedTable(session, - this.quoted(tableHandle.getRequiredNamedRelation().getRemoteTableName())); - if (!segmented) { - return 1; - } - else { - int segmentCount = this.metadata.getSegmentCount(session); - if (segmentCount <= 0) { - segmentCount = 1; - } - return min(segmentCount, maxParallelism); - } - } - } - } - - private static List segmentedSplits(int parallelism) - { - return parallelism == 1 - ? List.of(new JdbcSplit(Optional.empty())) - : IntStream.range(0, parallelism) - .boxed() - .map(id -> new JdbcSplit(Optional.of(format("gp_segment_id %% %s = %s", parallelism, id)))) - .collect(toImmutableList()); - } - - private List createAdbSplits(ConnectorSession session, - JdbcTableHandle tableHandle, - List splits) - { - String objectName = this.quoted(tableHandle.getRequiredNamedRelation().getRemoteTableName()); - Map tableProperties = this.metadata.getTableProperties(session, objectName, this.getIdentifierMapping()); - List distributionInfo = AdbTableProperties.getDistributedBy(tableProperties).orElse(List.of()); - return splits.stream() - .map(split -> new AdbJdbcSplit(distributionInfo, split.getAdditionalPredicate(), split.getDynamicFilter())) - .toList(); + return splitSourceManager.getSplits(session, tableHandle); } private static Optional toDecimalTypeToTypeHandle(DecimalType decimalType) @@ -806,255 +610,18 @@ private static Map getArrayColumnDimensions(Connection connecti @Override public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) { - return Optional.ofNullable(toColumnMappingInternal(session, Optional.of(connection), typeHandle)) - .map(AdbColumnMapping::columnMapping); + return dataTypeMapper.toColumnMapping(session, connection, typeHandle); } - private AdbColumnMapping toColumnMappingInternal(ConnectorSession session, Optional connection, JdbcTypeHandle typeHandle) + public ColumnDataType getColumnDataType(ConnectorSession session, JdbcTypeHandle typeHandle) { - String jdbcTypeName = typeHandle.jdbcTypeName() - .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); - Optional mapping = getForcedMappingToVarchar(typeHandle); - if (mapping.isPresent()) { - return new AdbColumnMapping(mapping.get(), new UnknownDataType()); - } - switch (jdbcTypeName) { - case "bool": - return new AdbColumnMapping(booleanColumnMapping(), new BooleanDataType()); - case "money": - return new AdbColumnMapping(moneyColumnMapping(), new MoneyDataType()); - case "uuid": - return new AdbColumnMapping(uuidColumnMapping(), new UuidDataType()); - case "jsonb": - case "json": - return new AdbColumnMapping(jsonColumnMapping(), new JsonbDataType()); - case "timestamptz": - int decimalDigits = typeHandle.requiredDecimalDigits(); - return timestampWithTimeZoneColumnMapping(decimalDigits); - } - switch (typeHandle.jdbcType()) { - case Types.BIT: - return new AdbColumnMapping(booleanColumnMapping(), new BitDataType(typeHandle.columnSize())); - case Types.SMALLINT: - return new AdbColumnMapping(smallintColumnMapping(), new SmallintDataType(SMALLINT)); - case Types.INTEGER: - return new AdbColumnMapping(integerColumnMapping(), new IntegerDataType()); - case Types.BIGINT: - return new AdbColumnMapping(bigintColumnMapping(), new BigintDataType()); - case Types.REAL: - return new AdbColumnMapping(realColumnMapping(), new RealDataType()); - case Types.DOUBLE: - return new AdbColumnMapping(doubleColumnMapping(), new DoubleDataType()); - case Types.NUMERIC: { - int columnSize = typeHandle.requiredColumnSize(); - int precision; - int decimalDigits = typeHandle.decimalDigits().orElse(0); - if (getDecimalRounding(session) == ALLOW_OVERFLOW) { - if (columnSize == PRECISION_OF_UNSPECIFIED_DECIMAL) { - // decimal type with unspecified scale - up to 131072 digits before the decimal point; up to 16383 digits after the decimal point) - DecimalType decimalType = createDecimalType(Decimals.MAX_PRECISION, getDecimalDefaultScale(session)); - return new AdbColumnMapping(decimalColumnMapping(decimalType, getDecimalRoundingMode(session)), new UnknownDataType()); - } - precision = columnSize; - if (precision > Decimals.MAX_PRECISION) { - int scale = min(decimalDigits, getDecimalDefaultScale(session)); - DecimalType decimalType = createDecimalType(Decimals.MAX_PRECISION, scale); - return new AdbColumnMapping(decimalColumnMapping(decimalType, getDecimalRoundingMode(session)), new UnknownDataType()); - } - } - precision = columnSize + max(-decimalDigits, 0); - if (columnSize != PRECISION_OF_UNSPECIFIED_DECIMAL && precision <= Decimals.MAX_PRECISION) { - DecimalType decimalType = createDecimalType(precision, max(decimalDigits, 0)); - ColumnDataType columnDataType = decimalType.isShort() - ? new DecimalShortDataType(decimalType) - : new DecimalLongDataType(decimalType); - return new AdbColumnMapping(decimalColumnMapping(decimalType, RoundingMode.UNNECESSARY), - columnDataType); - } - break; - } - case Types.CHAR: - ColumnMapping charColumnMapping = charColumnMapping(typeHandle.requiredColumnSize()); - return new AdbColumnMapping(charColumnMapping, new CharDataType((CharType) charColumnMapping.getType())); - case Types.VARCHAR: - if (!jdbcTypeName.equals("varchar")) { - return new AdbColumnMapping(enumColumnMapping(session, jdbcTypeName), new UnknownDataType()); - } - ColumnMapping varcharColumnMapping = varcharColumnMapping(typeHandle.requiredColumnSize()); - return new AdbColumnMapping(varcharColumnMapping, new VarcharDataType((VarcharType) varcharColumnMapping.getType())); - case Types.BINARY: - ColumnMapping columnMapping = varbinaryColumnMapping(); - if (jdbcTypeName.equals("bytea")) { - return new AdbColumnMapping(columnMapping, new ByteaDataType()); - } - return new AdbColumnMapping(varbinaryColumnMapping(), new UnknownDataType()); - case Types.DATE: - return new AdbColumnMapping(dateColumnMappingUsingLocalDate(), new DateDataType()); - case Types.TIME: - int requiredDecimalDigits = typeHandle.requiredDecimalDigits(); - return new AdbColumnMapping(timeColumnMapping(requiredDecimalDigits), new TimeDataType(requiredDecimalDigits)); - case Types.TIMESTAMP: - TimestampType timestampType = createTimestampType(typeHandle.requiredDecimalDigits()); - return new AdbColumnMapping( - ColumnMapping.longMapping( - timestampType, timestampReadFunction(timestampType), AdbSqlClient::shortTimestampWriteFunction), - new TimestampWithoutTimeZoneDataType(timestampType)); - case Types.ARRAY: - if (connection.isPresent()) { - Optional arrayColumnMapping = arrayToTrinoType(session, connection.get(), typeHandle); - if (arrayColumnMapping.isPresent()) { - return new AdbColumnMapping(arrayColumnMapping.get(), new UnknownDataType()); - } - } - break; - } - if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { - Optional columnMapping = mapToUnboundedVarchar(typeHandle); - if (columnMapping.isPresent()) { - return new AdbColumnMapping(columnMapping.get(), new UnknownDataType()); - } - } - return null; - } - - public Optional fromTrinoType(Type type) - { - if (type == BOOLEAN) { - return Optional.of(new BigintDataType()); - } - else if (type == TINYINT) { - return Optional.of(new SmallintDataType(TINYINT)); - } - else if (type == SMALLINT) { - return Optional.of(new SmallintDataType(SMALLINT)); - } - else if (type == INTEGER) { - return Optional.of(new IntegerDataType()); - } - else if (type == BIGINT) { - return Optional.of(new BigintDataType()); - } - else if (type == REAL) { - return Optional.of(new RealDataType()); - } - else if (type == DOUBLE) { - return Optional.of(new DoubleDataType()); - } - else if (type instanceof DecimalType decimalType) { - return decimalType.isShort() - ? Optional.of(new DecimalShortDataType(decimalType)) - : Optional.of(new DecimalLongDataType(decimalType)); - } - else if (type == DATE) { - return Optional.of(new DateDataType()); - } - else if (type instanceof TimeType timeType) { - return Optional.of(new TimeDataType(timeType.getPrecision())); - } - else if (type instanceof TimestampType timestampType) { - return Optional.of(new TimestampWithoutTimeZoneDataType(timestampType)); - } - else if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) { - return timestampWithTimeZoneType.isShort() - ? Optional.of(new TimestampShortWithTimeZoneDataType(timestampWithTimeZoneType.getPrecision())) - : Optional.of(new TimestampLongWithTimeZoneDataType(timestampWithTimeZoneType.getPrecision())); - } - else if (type instanceof CharType charType) { - return Optional.of(new CharDataType(charType)); - } - else if (type instanceof VarcharType varcharType) { - return Optional.of(new VarcharDataType(varcharType)); - } - else if (type == VARBINARY) { - return Optional.of(new ByteaDataType()); - } - else { - return type == UuidType.UUID ? Optional.of(new UuidDataType()) : Optional.empty(); - } + return dataTypeMapper.getColumnDataType(session, typeHandle); } @Override public WriteMapping toWriteMapping(ConnectorSession session, Type type) { - if (type == BOOLEAN) { - return WriteMapping.booleanMapping("boolean", booleanWriteFunction()); - } - if (type == TINYINT) { - return WriteMapping.longMapping("smallint", tinyintWriteFunction()); - } - if (type == SMALLINT) { - return WriteMapping.longMapping("smallint", smallintWriteFunction()); - } - if (type == INTEGER) { - return WriteMapping.longMapping("integer", integerWriteFunction()); - } - if (type == BIGINT) { - return WriteMapping.longMapping("bigint", bigintWriteFunction()); - } - if (type == REAL) { - return WriteMapping.longMapping("real", realWriteFunction()); - } - if (type == DOUBLE) { - return WriteMapping.doubleMapping("double precision", doubleWriteFunction()); - } - if (type instanceof DecimalType decimalType) { - String dataType = format("decimal(%s, %s)", decimalType.getPrecision(), decimalType.getScale()); - if (decimalType.isShort()) { - return WriteMapping.longMapping(dataType, shortDecimalWriteFunction(decimalType)); - } - return WriteMapping.objectMapping(dataType, longDecimalWriteFunction(decimalType)); - } - if (type instanceof CharType) { - return WriteMapping.sliceMapping("char(" + ((CharType) type).getLength() + ")", charWriteFunction()); - } - if (type instanceof VarcharType varcharType) { - String dataType; - if (varcharType.isUnbounded()) { - dataType = "varchar"; - } - else { - dataType = "varchar(" + varcharType.getBoundedLength() + ")"; - } - return WriteMapping.sliceMapping(dataType, varcharWriteFunction()); - } - if (VARBINARY.equals(type)) { - return WriteMapping.sliceMapping("bytea", varbinaryWriteFunction()); - } - if (type == DATE) { - return WriteMapping.longMapping("date", dateWriteFunctionUsingLocalDate()); - } - if (type instanceof TimeType timeType) { - verify(timeType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION); - return WriteMapping.longMapping(format("time(%s)", timeType.getPrecision()), timeWriteFunction(timeType.getPrecision())); - } - if (type instanceof TimestampType timestampType) { - if (timestampType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION) { - return WriteMapping.longMapping(format("timestamp(%s)", timestampType.getPrecision()), AdbSqlClient::shortTimestampWriteFunction); - } - else { - return WriteMapping.objectMapping(format("timestamp(%s)", POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION), longTimestampWriteFunction()); - } - } - if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) { - verify(timestampWithTimeZoneType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION); - String dataType = format("timestamptz(%d)", timestampWithTimeZoneType.getPrecision()); - if (timestampWithTimeZoneType.isShort()) { - return WriteMapping.longMapping(dataType, shortTimestampWithTimeZoneWriteFunction()); - } - return WriteMapping.objectMapping(dataType, longTimestampWithTimeZoneWriteFunction()); - } - if (type.equals(jsonType)) { - return WriteMapping.sliceMapping("jsonb", typedVarcharWriteFunction("json")); - } - if (type.equals(uuidType)) { - return WriteMapping.sliceMapping("uuid", uuidWriteFunction()); - } - if (type instanceof ArrayType arrayType && getArrayMapping(session) == AS_ARRAY) { - Type elementType = arrayType.getElementType(); - String elementDataType = toWriteMapping(session, elementType).getDataType(); - return WriteMapping.objectMapping(elementDataType + "[]", arrayWriteFunction(session, elementType, getArrayElementPgTypeName(session, this, elementType))); - } - throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName()); + return dataTypeMapper.toWriteMapping(session, type); } @Override @@ -1202,104 +769,7 @@ public OptionalInt getMaxColumnNameLength(ConnectorSession session) @Override public TableStatistics getTableStatistics(ConnectorSession session, JdbcTableHandle handle) { - if (!statisticsEnabled) { - return TableStatistics.empty(); - } - if (!handle.isNamedRelation()) { - return TableStatistics.empty(); - } - try { - return readTableStatistics(session, handle); - } - catch (SQLException | RuntimeException e) { - throwIfInstanceOf(e, TrinoException.class); - throw new TrinoException(JDBC_ERROR, "Failed fetching statistics for table: " + handle, e); - } - } - - private TableStatistics readTableStatistics(ConnectorSession session, JdbcTableHandle table) - throws SQLException - { - checkArgument(table.isNamedRelation(), "Relation is not a table: %s", table); - try (Connection connection = connectionFactory.openConnection(session); - Handle handle = Jdbi.open(connection)) { - StatisticsDao statisticsDao = new StatisticsDao(handle); - - Optional optionalRowCount = readRowCountTableStat(statisticsDao, table); - if (optionalRowCount.isEmpty()) { - // Table not found - return TableStatistics.empty(); - } - long rowCount = optionalRowCount.get(); - if (rowCount == -1) { - // Table has never yet been vacuumed or analyzed - return TableStatistics.empty(); - } - TableStatistics.Builder tableStatistics = TableStatistics.builder(); - tableStatistics.setRowCount(Estimate.of(rowCount)); - - if (rowCount == 0) { - return tableStatistics.build(); - } - - RemoteTableName remoteTableName = table.getRequiredNamedRelation().getRemoteTableName(); - Map columnStatistics = statisticsDao.getColumnStatistics(remoteTableName.getSchemaName().orElse(null), remoteTableName.getTableName()).stream() - .collect(toImmutableMap(ColumnStatisticsResult::getColumnName, identity())); - - for (JdbcColumnHandle column : this.getColumns(session, table)) { - ColumnStatisticsResult result = columnStatistics.get(column.getColumnName()); - if (result == null) { - continue; - } - - ColumnStatistics statistics = ColumnStatistics.builder() - .setNullsFraction(result.getNullsFraction() - .map(Estimate::of) - .orElseGet(Estimate::unknown)) - .setDistinctValuesCount(result.getDistinctValuesIndicator() - .map(distinctValuesIndicator -> { - if (distinctValuesIndicator >= 0.0) { - return distinctValuesIndicator; - } - return -distinctValuesIndicator * rowCount; - }) - .map(Estimate::of) - .orElseGet(Estimate::unknown)) - .setDataSize(result.getAverageColumnLength() - .flatMap(averageColumnLength -> - result.getNullsFraction().map(nullsFraction -> - Estimate.of(1.0 * averageColumnLength * rowCount * (1 - nullsFraction)))) - .orElseGet(Estimate::unknown)) - .build(); - - tableStatistics.setColumnStatistics(column, statistics); - } - - return tableStatistics.build(); - } - } - - private static Optional readRowCountTableStat(StatisticsDao statisticsDao, JdbcTableHandle table) - { - RemoteTableName remoteTableName = table.getRequiredNamedRelation().getRemoteTableName(); - String schemaName = remoteTableName.getSchemaName().orElse(null); - Optional rowCount = statisticsDao.getRowCountFromPgClass(schemaName, remoteTableName.getTableName()); - if (rowCount.isEmpty()) { - // Table not found - return Optional.empty(); - } - if (statisticsDao.isPartitionedTable(schemaName, remoteTableName.getTableName())) { - Optional partitionedTableRowCount = statisticsDao.getRowCountPartitionedTableFromPgClass(schemaName, remoteTableName.getTableName()); - if (partitionedTableRowCount.isPresent()) { - return partitionedTableRowCount; - } - - return statisticsDao.getRowCountPartitionedTableFromPgStats(schemaName, remoteTableName.getTableName()); - } - if (rowCount.get() == 0) { - rowCount = statisticsDao.getRowCountFromPgStat(schemaName, remoteTableName.getTableName()); - } - return rowCount; + return statisticsManager.getTableStatistics(session, handle, getColumns(session, handle)); } @Override @@ -1397,245 +867,6 @@ protected void verifyColumnName(DatabaseMetaData databaseMetadata, String column } } - private static LongWriteFunction timeWriteFunction(int precision) - { - checkArgument(precision <= 6, "Unsupported precision: %s", precision); // PostgreSQL limit but also assumption within this method - String bindExpression = format("CAST(? AS time(%s))", precision); - return new LongWriteFunction() - { - @Override - public String getBindExpression() - { - return bindExpression; - } - - @Override - public void set(PreparedStatement statement, int index, long picosOfDay) - throws SQLException - { - picosOfDay = round(picosOfDay, 12 - precision); - if (picosOfDay == PICOSECONDS_PER_DAY) { - picosOfDay = 0; - } - LocalTime localTime = LocalTime.ofNanoOfDay(picosOfDay / PICOSECONDS_PER_NANOSECOND); - // statement.setObject(.., localTime) would yield incorrect end result for 23:59:59.999000 - statement.setString(index, TIME_TYPE_FORMATTER.format(localTime)); - } - }; - } - - private static ColumnMapping moneyColumnMapping() - { - return ColumnMapping.sliceMapping( - VARCHAR, - new SliceReadFunction() - { - @Override - public boolean isNull(ResultSet resultSet, int columnIndex) - throws SQLException - { - resultSet.getString(columnIndex); - return resultSet.wasNull(); - } - - @Override - public Slice readSlice(ResultSet resultSet, int columnIndex) - throws SQLException - { - return utf8Slice(resultSet.getString(columnIndex)); - } - }, - (statement, index, value) -> { - throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); - }, - DISABLE_PUSHDOWN); - } - - private ColumnMapping uuidColumnMapping() - { - return ColumnMapping.sliceMapping( - uuidType, - (resultSet, columnIndex) -> javaUuidToTrinoUuid((UUID) resultSet.getObject(columnIndex)), - uuidWriteFunction()); - } - - private ColumnMapping jsonColumnMapping() - { - return ColumnMapping.sliceMapping( - jsonType, - (resultSet, columnIndex) -> jsonParse(utf8Slice(resultSet.getString(columnIndex))), - typedVarcharWriteFunction("json"), - DISABLE_PUSHDOWN); - } - - private static SliceWriteFunction typedVarcharWriteFunction(String jdbcTypeName) - { - requireNonNull(jdbcTypeName, "jdbcTypeName is null"); - String quotedJdbcTypeName = jdbcTypeName.startsWith("\"") && jdbcTypeName.endsWith("\"") ? jdbcTypeName : "\"%s\"".formatted(jdbcTypeName.replace("\"", "\"\"")); - String bindExpression = format("CAST(? AS %s)", quotedJdbcTypeName); - - return new SliceWriteFunction() - { - @Override - public String getBindExpression() - { - return bindExpression; - } - - @Override - public void set(PreparedStatement statement, int index, Slice value) - throws SQLException - { - statement.setString(index, value.toStringUtf8()); - } - }; - } - - private static AdbColumnMapping timestampWithTimeZoneColumnMapping(int precision) - { - // Adb supports timestamptz precision up to microseconds - checkArgument(precision <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION, "unsupported precision value %s", precision); - TimestampWithTimeZoneType trinoType = createTimestampWithTimeZoneType(precision); - if (precision <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) { - return new AdbColumnMapping(ColumnMapping.longMapping( - trinoType, - shortTimestampWithTimeZoneReadFunction(), - shortTimestampWithTimeZoneWriteFunction()), - new TimestampShortWithTimeZoneDataType(precision)); - } - return new AdbColumnMapping(ColumnMapping.objectMapping( - trinoType, - longTimestampWithTimeZoneReadFunction(), - longTimestampWithTimeZoneWriteFunction()), - new TimestampLongWithTimeZoneDataType(precision)); - } - - private static LongReadFunction shortTimestampWithTimeZoneReadFunction() - { - return (resultSet, columnIndex) -> { - // Adb does not store zone information in "timestamp with time zone" data type - long millisUtc = resultSet.getTimestamp(columnIndex).getTime(); - return packDateTimeWithZone(millisUtc, UTC_KEY); - }; - } - - private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction() - { - return (statement, index, value) -> { - // Adb does not store zone information in "timestamp with time zone" data type - long millisUtc = unpackMillisUtc(value); - statement.setTimestamp(index, new Timestamp(millisUtc)); - }; - } - - private static ObjectReadFunction longTimestampWithTimeZoneReadFunction() - { - return ObjectReadFunction.of( - LongTimestampWithTimeZone.class, - (resultSet, columnIndex) -> { - // Adb does not store zone information in "timestamp with time zone" data type - OffsetDateTime offsetDateTime = resultSet.getObject(columnIndex, OffsetDateTime.class); - return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( - offsetDateTime.toEpochSecond(), - (long) offsetDateTime.getNano() * PICOSECONDS_PER_NANOSECOND, - UTC_KEY); - }); - } - - private static ObjectWriteFunction longTimestampWithTimeZoneWriteFunction() - { - return ObjectWriteFunction.of( - LongTimestampWithTimeZone.class, - (statement, index, value) -> { - // Adb does not store zone information in "timestamp with time zone" data type - long epochSeconds = floorDiv(value.getEpochMillis(), MILLISECONDS_PER_SECOND); - long nanosOfSecond = (long) floorMod(value.getEpochMillis(), MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; - statement.setObject(index, OffsetDateTime.ofInstant(Instant.ofEpochSecond(epochSeconds, nanosOfSecond), UTC_KEY.getZoneId())); - }); - } - - private static SliceWriteFunction uuidWriteFunction() - { - return (statement, index, value) -> statement.setObject(index, trinoUuidToJavaUuid(value), Types.OTHER); - } - - private static ColumnMapping charColumnMapping(int charLength) - { - if (charLength > 65536) { - return varcharColumnMapping(charLength); - } - else { - CharType charType = CharType.createCharType(charLength); - return ColumnMapping.sliceMapping( - charType, charReadFunction(charType), charWriteFunction(), ADB_STRING_COLLATION_AWARE_PUSHDOWN); - } - } - - private static ColumnMapping varcharColumnMapping(int varcharLength) - { - VarcharType varcharType = varcharLength <= 2147483646 ? VarcharType.createVarcharType(varcharLength) : VarcharType.createUnboundedVarcharType(); - return ColumnMapping.sliceMapping( - varcharType, - varcharReadFunction(varcharType), - varcharWriteFunction(), - ADB_STRING_COLLATION_AWARE_PUSHDOWN); - } - - private static ColumnMapping enumColumnMapping(ConnectorSession session, String jdbcTypeName) - { - //todo implement AdbAdvancedPushdownSessionProperties.isPushdownEnums(session); - boolean pushdownEnums = false; - PredicatePushdownController pushdownController = pushdownEnums ? ADB_STRING_COLLATION_AWARE_PUSHDOWN : DISABLE_PUSHDOWN; - return ColumnMapping.sliceMapping( - VARCHAR, - (resultSet, columnIndex) -> utf8Slice(resultSet.getString(columnIndex)), - typedVarcharWriteFunction(jdbcTypeName), - pushdownController); - } - - private static ColumnMapping timeColumnMapping(int precision) - { - // adb limit but also assumption within this method - verify(precision <= 6, "Unsupported precision: %s", precision); - return ColumnMapping.longMapping( - createTimeType(precision), - (resultSet, columnIndex) -> { - LocalTime time = resultSet.getObject(columnIndex, LocalTime.class); - long nanosOfDay = time.toNanoOfDay(); - if (nanosOfDay == NANOSECONDS_PER_DAY - 1) { - // adb 24:00:00 is returned as 23:59:59.999999999, regardless of column precision - nanosOfDay = NANOSECONDS_PER_DAY - LongMath.pow(10, 9 - precision); - } - - long picosOfDay = nanosOfDay * PICOSECONDS_PER_NANOSECOND; - return round(picosOfDay, 12 - precision); - }, - timeWriteFunction(precision), - // Pushdown disabled because adb distinguishes TIME '24:00:00' and TIME '00:00:00' whereas Trino does not. - DISABLE_PUSHDOWN); - } - - private static void shortTimestampWriteFunction(PreparedStatement statement, int index, long epochMicros) - throws SQLException - { - LocalDateTime localDateTime = fromTrinoTimestamp(epochMicros); - statement.setObject(index, toPgTimestamp(localDateTime)); - } - - private static ObjectWriteFunction longTimestampWriteFunction() - { - return ObjectWriteFunction.of(LongTimestamp.class, (statement, index, timestamp) -> { - // adb supports up to 6 digits of precision - //noinspection ConstantConditions - verify(POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION == 6); - long epochMicros = timestamp.getEpochMicros(); - if (timestamp.getPicosOfMicro() >= PICOSECONDS_PER_MICROSECOND / 2) { - epochMicros++; - } - shortTimestampWriteFunction(statement, index, epochMicros); - }); - } - @Override public void setColumnComment(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Optional comment) { @@ -1648,179 +879,9 @@ public void setColumnComment(ConnectorSession session, JdbcTableHandle handle, J execute(session, sql); } - private Optional arrayToTrinoType(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + public List getColumnDataTypes(ConnectorSession session, JdbcOutputTableHandle outputTableHandle) { - checkArgument(typeHandle.jdbcType() == Types.ARRAY, "Not array type"); - AdbPluginConfig.ArrayMapping arrayMapping = getArrayMapping(session); - if (arrayMapping == DISABLED) { - return Optional.empty(); - } - JdbcTypeHandle baseElementTypeHandle = getArrayElementTypeHandle(connection, typeHandle); - String baseElementTypeName = baseElementTypeHandle.jdbcTypeName() - .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Element type name is missing: " + baseElementTypeHandle)); - if (baseElementTypeHandle.jdbcType() == Types.BINARY) { - // adb jdbc driver doesn't currently support array of varbinary (bytea[]) - return Optional.empty(); - } - Optional baseElementMapping = toColumnMapping(session, connection, baseElementTypeHandle); - - if (arrayMapping == AS_ARRAY) { - if (typeHandle.arrayDimensions().isEmpty()) { - return Optional.empty(); - } - return baseElementMapping - .map(elementMapping -> { - ArrayType trinoArrayType = new ArrayType(elementMapping.getType()); - ColumnMapping arrayColumnMapping = arrayColumnMapping(session, trinoArrayType, elementMapping, baseElementTypeName); - - int arrayDimensions = typeHandle.arrayDimensions().get(); - for (int i = 1; i < arrayDimensions; i++) { - trinoArrayType = new ArrayType(trinoArrayType); - arrayColumnMapping = arrayColumnMapping(session, trinoArrayType, arrayColumnMapping, baseElementTypeName); - } - return arrayColumnMapping; - }); - } - if (arrayMapping == AS_JSON) { - return baseElementMapping - .map(elementMapping -> arrayAsJsonColumnMapping(session, elementMapping)); - } - throw new IllegalStateException("Unsupported array mapping type: " + arrayMapping); - } - - private static ColumnMapping arrayColumnMapping(ConnectorSession session, ArrayType arrayType, ColumnMapping arrayElementMapping, String baseElementJdbcTypeName) - { - return ColumnMapping.objectMapping( - arrayType, - arrayReadFunction(arrayType.getElementType(), arrayElementMapping.getReadFunction()), - arrayWriteFunction(session, arrayType.getElementType(), baseElementJdbcTypeName)); - } - - private static ObjectWriteFunction arrayWriteFunction(ConnectorSession session, Type elementType, String baseElementJdbcTypeName) - { - return ObjectWriteFunction.of(Block.class, (statement, index, block) -> { - Array jdbcArray = statement.getConnection().createArrayOf(baseElementJdbcTypeName, getJdbcObjectArray(session, elementType, block)); - statement.setArray(index, jdbcArray); - }); - } - - private ColumnMapping arrayAsJsonColumnMapping(ConnectorSession session, ColumnMapping baseElementMapping) - { - return ColumnMapping.sliceMapping( - jsonType, - arrayAsJsonReadFunction(session, baseElementMapping), - (statement, index, block) -> { - throw new TrinoException(NOT_SUPPORTED, "Writing to array type is unsupported"); - }, - DISABLE_PUSHDOWN); - } - - private static JdbcTypeHandle getArrayElementTypeHandle(Connection connection, JdbcTypeHandle arrayTypeHandle) - { - String jdbcTypeName = arrayTypeHandle.jdbcTypeName() - .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + arrayTypeHandle)); - try { - TypeInfo typeInfo = connection.unwrap(PgConnection.class).getTypeInfo(); - int pgElementOid = typeInfo.getPGArrayElement(typeInfo.getPGType(jdbcTypeName)); - verify(arrayTypeHandle.caseSensitivity().isEmpty(), "Case sensitivity not supported"); - return new JdbcTypeHandle( - typeInfo.getSQLType(pgElementOid), - Optional.of(typeInfo.getPGType(pgElementOid)), - arrayTypeHandle.columnSize(), - arrayTypeHandle.decimalDigits(), - Optional.empty(), - Optional.empty()); - } - catch (SQLException e) { - throw new TrinoException(JDBC_ERROR, e); - } - } - - private static SliceReadFunction arrayAsJsonReadFunction(ConnectorSession session, ColumnMapping baseElementMapping) - { - return (resultSet, columnIndex) -> { - Object jdbcArray = resultSet.getArray(columnIndex).getArray(); - int arrayDimensions = arrayDepth(jdbcArray); - - ReadFunction readFunction = baseElementMapping.getReadFunction(); - Type type = baseElementMapping.getType(); - for (int i = 0; i < arrayDimensions; i++) { - readFunction = arrayReadFunction(type, readFunction); - type = new ArrayType(type); - } - Block block = (Block) ((ObjectReadFunction) readFunction).readObject(resultSet, columnIndex); - BlockBuilder builder = type.createBlockBuilder(null, 1); - type.writeObject(builder, block); - Object value = type.getObjectValue(session, builder.build(), 0); - - try { - return toJsonValue(value); - } - catch (IOException e) { - throw new TrinoException(JDBC_ERROR, "Conversion to JSON failed for " + type.getDisplayName(), e); - } - }; - } - - private static ObjectReadFunction arrayReadFunction(Type elementType, ReadFunction elementReadFunction) - { - return ObjectReadFunction.of(Block.class, (resultSet, columnIndex) -> { - Array array = resultSet.getArray(columnIndex); - BlockBuilder builder = elementType.createBlockBuilder(null, 10); - try (ResultSet arrayAsResultSet = array.getResultSet()) { - while (arrayAsResultSet.next()) { - if (elementReadFunction.isNull(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)) { - builder.appendNull(); - } - else if (elementType.getJavaType() == boolean.class) { - elementType.writeBoolean(builder, ((BooleanReadFunction) elementReadFunction).readBoolean(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); - } - else if (elementType.getJavaType() == long.class) { - elementType.writeLong(builder, ((LongReadFunction) elementReadFunction).readLong(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); - } - else if (elementType.getJavaType() == double.class) { - elementType.writeDouble(builder, ((DoubleReadFunction) elementReadFunction).readDouble(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); - } - else if (elementType.getJavaType() == Slice.class) { - elementType.writeSlice(builder, ((SliceReadFunction) elementReadFunction).readSlice(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); - } - else { - elementType.writeObject(builder, ((ObjectReadFunction) elementReadFunction).readObject(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); - } - } - } - return builder.build(); - }); - } - - public List getColumnDataTypes(ConnectorSession session, - JdbcOutputTableHandle outputTableHandle) - { - List columnDataTypes = new ArrayList<>(); - for (int i = 0; i < outputTableHandle.getColumnNames().size(); i++) { - ColumnDataType columnDataType; - if (outputTableHandle.getJdbcColumnTypes().isEmpty()) { - Type columnType = outputTableHandle.getColumnTypes().get(i); - columnDataType = fromTrinoType(columnType) - .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, - format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType))); - } - else { - JdbcTypeHandle columnType = (outputTableHandle.getJdbcColumnTypes().get()).get(i); - columnDataType = Optional.ofNullable(toColumnMappingInternal(session, - Optional.empty(), - columnType)) - .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, - format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType))) - .columnDataType(); - if (columnDataType.getType() == ConnectorDataType.UNKNOWN) { - throw new TrinoException(NOT_SUPPORTED, - format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType)); - } - } - columnDataTypes.add(columnDataType); - } - return columnDataTypes; + return dataTypeMapper.getColumnDataTypes(session, outputTableHandle); } public String getTargetTableName(JdbcOutputTableHandle tableHandle) @@ -1829,127 +890,4 @@ public String getTargetTableName(JdbcOutputTableHandle tableHandle) tableHandle.getSchemaName(), tableHandle.getTemporaryTableName().orElseGet(tableHandle::getTableName)); } - - private static class StatisticsDao - { - private final Handle handle; - - public StatisticsDao(Handle handle) - { - this.handle = requireNonNull(handle, "handle is null"); - } - - Optional getRowCountFromPgClass(String schema, String tableName) - { - return handle.createQuery("SELECT reltuples " + - "FROM pg_class " + - "WHERE relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = :schema) " + - "AND relname = :table_name") - .bind("schema", schema) - .bind("table_name", tableName) - .mapTo(Long.class) - .findOne(); - } - - Optional getRowCountFromPgStat(String schema, String tableName) - { - return handle.createQuery("SELECT n_live_tup FROM pg_stat_all_tables WHERE schemaname = :schema AND relname = :table_name") - .bind("schema", schema) - .bind("table_name", tableName) - .mapTo(Long.class) - .findOne(); - } - - Optional getRowCountPartitionedTableFromPgClass(String schema, String tableName) - { - return handle.createQuery("SELECT SUM(child.reltuples) " + - "FROM pg_inherits " + - "JOIN pg_class parent ON pg_inherits.inhparent = parent.oid " + - "JOIN pg_class child ON pg_inherits.inhrelid = child.oid " + - "JOIN pg_namespace parent_ns ON parent_ns.oid = parent.relnamespace " + - "JOIN pg_namespace child_ns ON child_ns.oid = child.relnamespace " + - "WHERE parent.oid = :schema_table_name::regclass") - .bind("schema_table_name", format("%s.%s", schema, tableName)) - .mapTo(Long.class) - .findOne(); - } - - Optional getRowCountPartitionedTableFromPgStats(String schema, String tableName) - { - return handle.createQuery("SELECT SUM(stat.n_live_tup) " + - "FROM pg_inherits " + - "JOIN pg_class parent ON pg_inherits.inhparent = parent.oid " + - "JOIN pg_class child ON pg_inherits.inhrelid = child.oid " + - "JOIN pg_namespace parent_ns ON parent_ns.oid = parent.relnamespace " + - "JOIN pg_namespace child_ns ON child_ns.oid = child.relnamespace " + - "JOIN pg_stat_all_tables stat ON stat.schemaname = child_ns.nspname AND stat.relname = child.relname " + - "WHERE parent.oid = :schema_table_name::regclass") - .bind("schema_table_name", format("%s.%s", schema, tableName)) - .mapTo(Long.class) - .findOne(); - } - - List getColumnStatistics(String schema, String tableName) - { - return handle.createQuery("SELECT attname, null_frac, n_distinct, avg_width FROM pg_stats WHERE schemaname = :schema AND tablename = :table_name") - .bind("schema", schema) - .bind("table_name", tableName) - .map((rs, ctx) -> new ColumnStatisticsResult( - requireNonNull(rs.getString("attname"), "attname is null"), - Optional.ofNullable(rs.getObject("null_frac", Float.class)), - Optional.ofNullable(rs.getObject("n_distinct", Float.class)), - Optional.ofNullable(rs.getObject("avg_width", Integer.class)))) - .list(); - } - - boolean isPartitionedTable(String schema, String tableName) - { - return handle.createQuery("SELECT true " + - "FROM pg_class " + - "WHERE relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = :schema) " + - "AND relname = :table_name " + - "AND relkind = 'p'") - .bind("schema", schema) - .bind("table_name", tableName) - .mapTo(Boolean.class) - .findOne() - .orElse(false); - } - } - - private static class ColumnStatisticsResult - { - private final String columnName; - private final Optional nullsFraction; - private final Optional distinctValuesIndicator; - private final Optional averageColumnLength; - - public ColumnStatisticsResult(String columnName, Optional nullsFraction, Optional distinctValuesIndicator, Optional averageColumnLength) - { - this.columnName = columnName; - this.nullsFraction = nullsFraction; - this.distinctValuesIndicator = distinctValuesIndicator; - this.averageColumnLength = averageColumnLength; - } - - public String getColumnName() - { - return columnName; - } - - public Optional getNullsFraction() - { - return nullsFraction; - } - - public Optional getDistinctValuesIndicator() - { - return distinctValuesIndicator; - } - - public Optional getAverageColumnLength() - { - return averageColumnLength; - } - } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapper.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapper.java new file mode 100644 index 000000000000..1e4d435a6151 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapper.java @@ -0,0 +1,39 @@ +/* + * 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.adb.connector.datatype.mapper; + +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.Type; + +import java.sql.Connection; +import java.util.List; +import java.util.Optional; + +public interface DataTypeMapper +{ + Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle); + + WriteMapping toWriteMapping(ConnectorSession session, Type type); + + ColumnDataType getColumnDataType(ConnectorSession session, JdbcTypeHandle typeHandle); + + List getColumnDataTypes(ConnectorSession session, JdbcOutputTableHandle outputTableHandle); + + Optional fromTrinoType(Type type); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapperImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapperImpl.java new file mode 100644 index 000000000000..f25c3a609732 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/datatype/mapper/DataTypeMapperImpl.java @@ -0,0 +1,920 @@ +/* + * 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.adb.connector.datatype.mapper; + +import com.google.common.collect.ImmutableSortedSet; +import com.google.common.math.LongMath; +import com.google.inject.Inject; +import io.airlift.slice.Slice; +import io.trino.plugin.adb.AdbPluginConfig; +import io.trino.plugin.adb.connector.AdbColumnMapping; +import io.trino.plugin.adb.connector.AdbSessionProperties; +import io.trino.plugin.adb.connector.datatype.BigintDataType; +import io.trino.plugin.adb.connector.datatype.BitDataType; +import io.trino.plugin.adb.connector.datatype.BooleanDataType; +import io.trino.plugin.adb.connector.datatype.ByteaDataType; +import io.trino.plugin.adb.connector.datatype.CharDataType; +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.adb.connector.datatype.ConnectorDataType; +import io.trino.plugin.adb.connector.datatype.DateDataType; +import io.trino.plugin.adb.connector.datatype.DecimalLongDataType; +import io.trino.plugin.adb.connector.datatype.DecimalShortDataType; +import io.trino.plugin.adb.connector.datatype.DoubleDataType; +import io.trino.plugin.adb.connector.datatype.IntegerDataType; +import io.trino.plugin.adb.connector.datatype.JsonbDataType; +import io.trino.plugin.adb.connector.datatype.MoneyDataType; +import io.trino.plugin.adb.connector.datatype.RealDataType; +import io.trino.plugin.adb.connector.datatype.SmallintDataType; +import io.trino.plugin.adb.connector.datatype.TimeDataType; +import io.trino.plugin.adb.connector.datatype.TimestampLongWithTimeZoneDataType; +import io.trino.plugin.adb.connector.datatype.TimestampShortWithTimeZoneDataType; +import io.trino.plugin.adb.connector.datatype.TimestampWithoutTimeZoneDataType; +import io.trino.plugin.adb.connector.datatype.UnknownDataType; +import io.trino.plugin.adb.connector.datatype.UuidDataType; +import io.trino.plugin.adb.connector.datatype.VarcharDataType; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.BooleanReadFunction; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.DoubleReadFunction; +import io.trino.plugin.jdbc.JdbcMetadataSessionProperties; +import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.LongReadFunction; +import io.trino.plugin.jdbc.LongWriteFunction; +import io.trino.plugin.jdbc.ObjectReadFunction; +import io.trino.plugin.jdbc.ObjectWriteFunction; +import io.trino.plugin.jdbc.PredicatePushdownController; +import io.trino.plugin.jdbc.ReadFunction; +import io.trino.plugin.jdbc.SliceReadFunction; +import io.trino.plugin.jdbc.SliceWriteFunction; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.spi.TrinoException; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.predicate.Domain; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.TimeType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; +import io.trino.spi.type.UuidType; +import io.trino.spi.type.VarcharType; +import org.postgresql.core.TypeInfo; +import org.postgresql.jdbc.PgConnection; + +import java.io.IOException; +import java.math.RoundingMode; +import java.sql.Array; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.AS_ARRAY; +import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.AS_JSON; +import static io.trino.plugin.adb.AdbPluginConfig.ArrayMapping.DISABLED; +import static io.trino.plugin.adb.TypeUtil.TIME_TYPE_FORMATTER; +import static io.trino.plugin.adb.TypeUtil.arrayDepth; +import static io.trino.plugin.adb.TypeUtil.getArrayElementPgTypeName; +import static io.trino.plugin.adb.TypeUtil.getJdbcObjectArray; +import static io.trino.plugin.adb.TypeUtil.toPgTimestamp; +import static io.trino.plugin.adb.connector.AdbSessionProperties.getArrayMapping; +import static io.trino.plugin.base.util.JsonTypeUtil.jsonParse; +import static io.trino.plugin.base.util.JsonTypeUtil.toJsonValue; +import static io.trino.plugin.jdbc.DecimalConfig.DecimalMapping.ALLOW_OVERFLOW; +import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalDefaultScale; +import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRounding; +import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRoundingMode; +import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.booleanColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateColumnMappingUsingLocalDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateWriteFunctionUsingLocalDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.fromTrinoTimestamp; +import static io.trino.plugin.jdbc.StandardColumnMappings.integerColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.realColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.timestampReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; +import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling; +import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; +import static io.trino.spi.type.DateTimeEncoding.unpackMillisUtc; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TimeType.createTimeType; +import static io.trino.spi.type.TimeZoneKey.UTC_KEY; +import static io.trino.spi.type.TimestampType.createTimestampType; +import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_DAY; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_DAY; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; +import static io.trino.spi.type.Timestamps.round; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.UuidType.javaUuidToTrinoUuid; +import static io.trino.spi.type.UuidType.trinoUuidToJavaUuid; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.Math.floorDiv; +import static java.lang.Math.floorMod; +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.lang.String.CASE_INSENSITIVE_ORDER; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class DataTypeMapperImpl + implements DataTypeMapper +{ + private static final int POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION = 6; + private static final int PRECISION_OF_UNSPECIFIED_DECIMAL = 0; + private static final int ARRAY_RESULT_SET_VALUE_COLUMN = 2; + private static final String COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE = "Column type %s is not supported"; + private final Type jsonType; + private final Type uuidType; + private final Set jdbcTypesMappedToVarchar; + private static final PredicatePushdownController ADB_STRING_COLLATION_AWARE_PUSHDOWN = (session, domain) -> { + if (domain.isOnlyNull()) { + return PredicatePushdownController.FULL_PUSHDOWN.apply(session, domain); + } + else if (AdbSessionProperties.isEnableStringPushdownWithCollate(session)) { + return PredicatePushdownController.FULL_PUSHDOWN.apply(session, domain); + } + else { + Domain simplifiedDomain = domain.simplify(JdbcMetadataSessionProperties.getDomainCompactionThreshold(session)); + return !simplifiedDomain.getValues().isDiscreteSet() + ? DISABLE_PUSHDOWN.apply(session, domain) + : PredicatePushdownController.FULL_PUSHDOWN.apply(session, simplifiedDomain); + } + }; + + @Inject + public DataTypeMapperImpl(TypeManager typeManager, BaseJdbcConfig jdbcConfig) + { + this.jsonType = typeManager.getType(new TypeSignature("json")); + this.uuidType = typeManager.getType(new TypeSignature("uuid")); + this.jdbcTypesMappedToVarchar = ImmutableSortedSet.orderedBy(CASE_INSENSITIVE_ORDER) + .addAll(requireNonNull(jdbcConfig.getJdbcTypesMappedToVarchar(), "jdbcTypesMappedToVarchar is null")) + .build(); + } + + @Override + public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + return Optional.ofNullable(toColumnMappingInternal(session, Optional.of(connection), typeHandle)) + .map(AdbColumnMapping::columnMapping); + } + + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + if (type == BOOLEAN) { + return WriteMapping.booleanMapping("boolean", booleanWriteFunction()); + } + if (type == TINYINT) { + return WriteMapping.longMapping("smallint", tinyintWriteFunction()); + } + if (type == SMALLINT) { + return WriteMapping.longMapping("smallint", smallintWriteFunction()); + } + if (type == INTEGER) { + return WriteMapping.longMapping("integer", integerWriteFunction()); + } + if (type == BIGINT) { + return WriteMapping.longMapping("bigint", bigintWriteFunction()); + } + if (type == REAL) { + return WriteMapping.longMapping("real", realWriteFunction()); + } + if (type == DOUBLE) { + return WriteMapping.doubleMapping("double precision", doubleWriteFunction()); + } + if (type instanceof DecimalType decimalType) { + String dataType = format("decimal(%s, %s)", decimalType.getPrecision(), decimalType.getScale()); + if (decimalType.isShort()) { + return WriteMapping.longMapping(dataType, shortDecimalWriteFunction(decimalType)); + } + return WriteMapping.objectMapping(dataType, longDecimalWriteFunction(decimalType)); + } + if (type instanceof CharType) { + return WriteMapping.sliceMapping("char(" + ((CharType) type).getLength() + ")", charWriteFunction()); + } + if (type instanceof VarcharType varcharType) { + String dataType; + if (varcharType.isUnbounded()) { + dataType = "varchar"; + } + else { + dataType = "varchar(" + varcharType.getBoundedLength() + ")"; + } + return WriteMapping.sliceMapping(dataType, varcharWriteFunction()); + } + if (VARBINARY.equals(type)) { + return WriteMapping.sliceMapping("bytea", varbinaryWriteFunction()); + } + if (type == DATE) { + return WriteMapping.longMapping("date", dateWriteFunctionUsingLocalDate()); + } + if (type instanceof TimeType timeType) { + verify(timeType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION); + return WriteMapping.longMapping(format("time(%s)", timeType.getPrecision()), timeWriteFunction(timeType.getPrecision())); + } + if (type instanceof TimestampType timestampType) { + if (timestampType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION) { + return WriteMapping.longMapping(format("timestamp(%s)", timestampType.getPrecision()), DataTypeMapperImpl::shortTimestampWriteFunction); + } + else { + return WriteMapping.objectMapping(format("timestamp(%s)", POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION), longTimestampWriteFunction()); + } + } + if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) { + verify(timestampWithTimeZoneType.getPrecision() <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION); + String dataType = format("timestamptz(%d)", timestampWithTimeZoneType.getPrecision()); + if (timestampWithTimeZoneType.isShort()) { + return WriteMapping.longMapping(dataType, shortTimestampWithTimeZoneWriteFunction()); + } + return WriteMapping.objectMapping(dataType, longTimestampWithTimeZoneWriteFunction()); + } + if (type.equals(jsonType)) { + return WriteMapping.sliceMapping("jsonb", typedVarcharWriteFunction("json")); + } + if (type.equals(uuidType)) { + return WriteMapping.sliceMapping("uuid", uuidWriteFunction()); + } + if (type instanceof ArrayType arrayType && getArrayMapping(session) == AS_ARRAY) { + Type elementType = arrayType.getElementType(); + String elementDataType = toWriteMapping(session, elementType).getDataType(); + return WriteMapping.objectMapping(elementDataType + "[]", arrayWriteFunction(session, elementType, getArrayElementPgTypeName(session, this, elementType))); + } + throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName()); + } + + @Override + public ColumnDataType getColumnDataType(ConnectorSession session, JdbcTypeHandle typeHandle) + { + return Optional.ofNullable(toColumnMappingInternal(session, Optional.empty(), typeHandle)) + .map(AdbColumnMapping::columnDataType) + .orElseThrow(() -> new IllegalArgumentException("Failed to get column data type for type: " + typeHandle)); + } + + private AdbColumnMapping toColumnMappingInternal(ConnectorSession session, Optional connection, JdbcTypeHandle typeHandle) + { + String jdbcTypeName = typeHandle.jdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); + Optional mapping = getForcedMappingToVarchar(typeHandle); + if (mapping.isPresent()) { + return new AdbColumnMapping(mapping.get(), new UnknownDataType()); + } + switch (jdbcTypeName) { + case "bool": + return new AdbColumnMapping(booleanColumnMapping(), new BooleanDataType()); + case "money": + return new AdbColumnMapping(moneyColumnMapping(), new MoneyDataType()); + case "uuid": + return new AdbColumnMapping(uuidColumnMapping(), new UuidDataType()); + case "jsonb": + case "json": + return new AdbColumnMapping(jsonColumnMapping(), new JsonbDataType()); + case "timestamptz": + int decimalDigits = typeHandle.requiredDecimalDigits(); + return timestampWithTimeZoneColumnMapping(decimalDigits); + } + switch (typeHandle.jdbcType()) { + case Types.BIT: + return new AdbColumnMapping(booleanColumnMapping(), new BitDataType(typeHandle.columnSize())); + case Types.SMALLINT: + return new AdbColumnMapping(smallintColumnMapping(), new SmallintDataType(SMALLINT)); + case Types.INTEGER: + return new AdbColumnMapping(integerColumnMapping(), new IntegerDataType()); + case Types.BIGINT: + return new AdbColumnMapping(bigintColumnMapping(), new BigintDataType()); + case Types.REAL: + return new AdbColumnMapping(realColumnMapping(), new RealDataType()); + case Types.DOUBLE: + return new AdbColumnMapping(doubleColumnMapping(), new DoubleDataType()); + case Types.NUMERIC: { + int columnSize = typeHandle.requiredColumnSize(); + int precision; + int decimalDigits = typeHandle.decimalDigits().orElse(0); + if (getDecimalRounding(session) == ALLOW_OVERFLOW) { + if (columnSize == PRECISION_OF_UNSPECIFIED_DECIMAL) { + // decimal type with unspecified scale - up to 131072 digits before the decimal point; up to 16383 digits after the decimal point) + DecimalType decimalType = createDecimalType(Decimals.MAX_PRECISION, getDecimalDefaultScale(session)); + return new AdbColumnMapping(decimalColumnMapping(decimalType, getDecimalRoundingMode(session)), new UnknownDataType()); + } + precision = columnSize; + if (precision > Decimals.MAX_PRECISION) { + int scale = min(decimalDigits, getDecimalDefaultScale(session)); + DecimalType decimalType = createDecimalType(Decimals.MAX_PRECISION, scale); + return new AdbColumnMapping(decimalColumnMapping(decimalType, getDecimalRoundingMode(session)), new UnknownDataType()); + } + } + precision = columnSize + max(-decimalDigits, 0); + if (columnSize != PRECISION_OF_UNSPECIFIED_DECIMAL && precision <= Decimals.MAX_PRECISION) { + DecimalType decimalType = createDecimalType(precision, max(decimalDigits, 0)); + ColumnDataType columnDataType = decimalType.isShort() + ? new DecimalShortDataType(decimalType) + : new DecimalLongDataType(decimalType); + return new AdbColumnMapping(decimalColumnMapping(decimalType, RoundingMode.UNNECESSARY), + columnDataType); + } + break; + } + case Types.CHAR: + ColumnMapping charColumnMapping = charColumnMapping(typeHandle.requiredColumnSize()); + return new AdbColumnMapping(charColumnMapping, new CharDataType((CharType) charColumnMapping.getType())); + case Types.VARCHAR: + if (!jdbcTypeName.equals("varchar")) { + return new AdbColumnMapping(enumColumnMapping(session, jdbcTypeName), new UnknownDataType()); + } + ColumnMapping varcharColumnMapping = varcharColumnMapping(typeHandle.requiredColumnSize()); + return new AdbColumnMapping(varcharColumnMapping, new VarcharDataType((VarcharType) varcharColumnMapping.getType())); + case Types.BINARY: + ColumnMapping columnMapping = varbinaryColumnMapping(); + if (jdbcTypeName.equals("bytea")) { + return new AdbColumnMapping(columnMapping, new ByteaDataType()); + } + return new AdbColumnMapping(varbinaryColumnMapping(), new UnknownDataType()); + case Types.DATE: + return new AdbColumnMapping(dateColumnMappingUsingLocalDate(), new DateDataType()); + case Types.TIME: + int requiredDecimalDigits = typeHandle.requiredDecimalDigits(); + return new AdbColumnMapping(timeColumnMapping(requiredDecimalDigits), new TimeDataType(requiredDecimalDigits)); + case Types.TIMESTAMP: + TimestampType timestampType = createTimestampType(typeHandle.requiredDecimalDigits()); + return new AdbColumnMapping( + ColumnMapping.longMapping( + timestampType, timestampReadFunction(timestampType), DataTypeMapperImpl::shortTimestampWriteFunction), + new TimestampWithoutTimeZoneDataType(timestampType)); + case Types.ARRAY: + if (connection.isPresent()) { + Optional arrayColumnMapping = arrayToTrinoType(session, connection.get(), typeHandle); + if (arrayColumnMapping.isPresent()) { + return new AdbColumnMapping(arrayColumnMapping.get(), new UnknownDataType()); + } + } + break; + } + if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { + Optional columnMapping = mapToUnboundedVarchar(typeHandle); + if (columnMapping.isPresent()) { + return new AdbColumnMapping(columnMapping.get(), new UnknownDataType()); + } + } + return null; + } + + @Override + public Optional fromTrinoType(Type type) + { + if (type == BOOLEAN) { + return Optional.of(new BigintDataType()); + } + else if (type == TINYINT) { + return Optional.of(new SmallintDataType(TINYINT)); + } + else if (type == SMALLINT) { + return Optional.of(new SmallintDataType(SMALLINT)); + } + else if (type == INTEGER) { + return Optional.of(new IntegerDataType()); + } + else if (type == BIGINT) { + return Optional.of(new BigintDataType()); + } + else if (type == REAL) { + return Optional.of(new RealDataType()); + } + else if (type == DOUBLE) { + return Optional.of(new DoubleDataType()); + } + else if (type instanceof DecimalType decimalType) { + return decimalType.isShort() + ? Optional.of(new DecimalShortDataType(decimalType)) + : Optional.of(new DecimalLongDataType(decimalType)); + } + else if (type == DATE) { + return Optional.of(new DateDataType()); + } + else if (type instanceof TimeType timeType) { + return Optional.of(new TimeDataType(timeType.getPrecision())); + } + else if (type instanceof TimestampType timestampType) { + return Optional.of(new TimestampWithoutTimeZoneDataType(timestampType)); + } + else if (type instanceof TimestampWithTimeZoneType timestampWithTimeZoneType) { + return timestampWithTimeZoneType.isShort() + ? Optional.of(new TimestampShortWithTimeZoneDataType(timestampWithTimeZoneType.getPrecision())) + : Optional.of(new TimestampLongWithTimeZoneDataType(timestampWithTimeZoneType.getPrecision())); + } + else if (type instanceof CharType charType) { + return Optional.of(new CharDataType(charType)); + } + else if (type instanceof VarcharType varcharType) { + return Optional.of(new VarcharDataType(varcharType)); + } + else if (type == VARBINARY) { + return Optional.of(new ByteaDataType()); + } + else { + return type == UuidType.UUID ? Optional.of(new UuidDataType()) : Optional.empty(); + } + } + + @Override + public List getColumnDataTypes(ConnectorSession session, JdbcOutputTableHandle outputTableHandle) + { + List columnDataTypes = new ArrayList<>(); + for (int i = 0; i < outputTableHandle.getColumnNames().size(); i++) { + ColumnDataType columnDataType; + if (outputTableHandle.getJdbcColumnTypes().isEmpty()) { + Type columnType = outputTableHandle.getColumnTypes().get(i); + columnDataType = fromTrinoType(columnType) + .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, + format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType))); + } + else { + JdbcTypeHandle columnType = (outputTableHandle.getJdbcColumnTypes().get()).get(i); + columnDataType = Optional.ofNullable(toColumnMappingInternal(session, + Optional.empty(), + columnType)) + .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, + format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType))) + .columnDataType(); + if (columnDataType.getType() == ConnectorDataType.UNKNOWN) { + throw new TrinoException(NOT_SUPPORTED, + format(COLUMN_TYPE_NOT_SUPPORTED_ERROR_MSG_TEMPLATE, columnType)); + } + } + columnDataTypes.add(columnDataType); + } + return columnDataTypes; + } + + private Optional getForcedMappingToVarchar(JdbcTypeHandle typeHandle) + { + if (typeHandle.jdbcTypeName().isPresent() && jdbcTypesMappedToVarchar.contains(typeHandle.jdbcTypeName().get())) { + return mapToUnboundedVarchar(typeHandle); + } + return Optional.empty(); + } + + protected static Optional mapToUnboundedVarchar(JdbcTypeHandle typeHandle) + { + VarcharType unboundedVarcharType = createUnboundedVarcharType(); + return Optional.of(ColumnMapping.sliceMapping( + unboundedVarcharType, + varcharReadFunction(unboundedVarcharType), + (statement, index, value) -> { + throw new TrinoException( + NOT_SUPPORTED, + "Underlying type that is mapped to VARCHAR is not supported for INSERT: " + typeHandle.jdbcTypeName().get()); + }, + DISABLE_PUSHDOWN)); + } + + private static ColumnMapping moneyColumnMapping() + { + return ColumnMapping.sliceMapping( + VARCHAR, + new SliceReadFunction() + { + @Override + public boolean isNull(ResultSet resultSet, int columnIndex) + throws SQLException + { + resultSet.getString(columnIndex); + return resultSet.wasNull(); + } + + @Override + public Slice readSlice(ResultSet resultSet, int columnIndex) + throws SQLException + { + return utf8Slice(resultSet.getString(columnIndex)); + } + }, + (statement, index, value) -> { + throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); + }, + DISABLE_PUSHDOWN); + } + + private static LongWriteFunction timeWriteFunction(int precision) + { + checkArgument(precision <= 6, "Unsupported precision: %s", precision); // PostgreSQL limit but also assumption within this method + String bindExpression = format("CAST(? AS time(%s))", precision); + return new LongWriteFunction() + { + @Override + public String getBindExpression() + { + return bindExpression; + } + + @Override + public void set(PreparedStatement statement, int index, long picosOfDay) + throws SQLException + { + picosOfDay = round(picosOfDay, 12 - precision); + if (picosOfDay == PICOSECONDS_PER_DAY) { + picosOfDay = 0; + } + LocalTime localTime = LocalTime.ofNanoOfDay(picosOfDay / PICOSECONDS_PER_NANOSECOND); + // statement.setObject(.., localTime) would yield incorrect end result for 23:59:59.999000 + statement.setString(index, TIME_TYPE_FORMATTER.format(localTime)); + } + }; + } + + private static SliceWriteFunction typedVarcharWriteFunction(String jdbcTypeName) + { + requireNonNull(jdbcTypeName, "jdbcTypeName is null"); + String quotedJdbcTypeName = jdbcTypeName.startsWith("\"") && jdbcTypeName.endsWith("\"") ? jdbcTypeName : "\"%s\"".formatted(jdbcTypeName.replace("\"", "\"\"")); + String bindExpression = format("CAST(? AS %s)", quotedJdbcTypeName); + + return new SliceWriteFunction() + { + @Override + public String getBindExpression() + { + return bindExpression; + } + + @Override + public void set(PreparedStatement statement, int index, Slice value) + throws SQLException + { + statement.setString(index, value.toStringUtf8()); + } + }; + } + + private ColumnMapping uuidColumnMapping() + { + return ColumnMapping.sliceMapping( + uuidType, + (resultSet, columnIndex) -> javaUuidToTrinoUuid((UUID) resultSet.getObject(columnIndex)), + uuidWriteFunction()); + } + + private ColumnMapping jsonColumnMapping() + { + return ColumnMapping.sliceMapping( + jsonType, + (resultSet, columnIndex) -> jsonParse(utf8Slice(resultSet.getString(columnIndex))), + typedVarcharWriteFunction("json"), + DISABLE_PUSHDOWN); + } + + private static AdbColumnMapping timestampWithTimeZoneColumnMapping(int precision) + { + // Adb supports timestamptz precision up to microseconds + checkArgument(precision <= POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION, "unsupported precision value %s", precision); + TimestampWithTimeZoneType trinoType = createTimestampWithTimeZoneType(precision); + if (precision <= TimestampWithTimeZoneType.MAX_SHORT_PRECISION) { + return new AdbColumnMapping(ColumnMapping.longMapping( + trinoType, + shortTimestampWithTimeZoneReadFunction(), + shortTimestampWithTimeZoneWriteFunction()), + new TimestampShortWithTimeZoneDataType(precision)); + } + return new AdbColumnMapping(ColumnMapping.objectMapping( + trinoType, + longTimestampWithTimeZoneReadFunction(), + longTimestampWithTimeZoneWriteFunction()), + new TimestampLongWithTimeZoneDataType(precision)); + } + + private static LongReadFunction shortTimestampWithTimeZoneReadFunction() + { + return (resultSet, columnIndex) -> { + // Adb does not store zone information in "timestamp with time zone" data type + long millisUtc = resultSet.getTimestamp(columnIndex).getTime(); + return packDateTimeWithZone(millisUtc, UTC_KEY); + }; + } + + private static LongWriteFunction shortTimestampWithTimeZoneWriteFunction() + { + return (statement, index, value) -> { + // Adb does not store zone information in "timestamp with time zone" data type + long millisUtc = unpackMillisUtc(value); + statement.setTimestamp(index, new Timestamp(millisUtc)); + }; + } + + private static ObjectReadFunction longTimestampWithTimeZoneReadFunction() + { + return ObjectReadFunction.of( + LongTimestampWithTimeZone.class, + (resultSet, columnIndex) -> { + // Adb does not store zone information in "timestamp with time zone" data type + OffsetDateTime offsetDateTime = resultSet.getObject(columnIndex, OffsetDateTime.class); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( + offsetDateTime.toEpochSecond(), + (long) offsetDateTime.getNano() * PICOSECONDS_PER_NANOSECOND, + UTC_KEY); + }); + } + + private static ObjectWriteFunction longTimestampWithTimeZoneWriteFunction() + { + return ObjectWriteFunction.of( + LongTimestampWithTimeZone.class, + (statement, index, value) -> { + // Adb does not store zone information in "timestamp with time zone" data type + long epochSeconds = floorDiv(value.getEpochMillis(), MILLISECONDS_PER_SECOND); + long nanosOfSecond = (long) floorMod(value.getEpochMillis(), MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; + statement.setObject(index, OffsetDateTime.ofInstant(Instant.ofEpochSecond(epochSeconds, nanosOfSecond), UTC_KEY.getZoneId())); + }); + } + + private static SliceWriteFunction uuidWriteFunction() + { + return (statement, index, value) -> statement.setObject(index, trinoUuidToJavaUuid(value), Types.OTHER); + } + + private static ColumnMapping charColumnMapping(int charLength) + { + if (charLength > 65536) { + return varcharColumnMapping(charLength); + } + else { + CharType charType = CharType.createCharType(charLength); + return ColumnMapping.sliceMapping( + charType, charReadFunction(charType), charWriteFunction(), ADB_STRING_COLLATION_AWARE_PUSHDOWN); + } + } + + private static ColumnMapping varcharColumnMapping(int varcharLength) + { + VarcharType varcharType = varcharLength <= 2147483646 ? VarcharType.createVarcharType(varcharLength) : VarcharType.createUnboundedVarcharType(); + return ColumnMapping.sliceMapping( + varcharType, + varcharReadFunction(varcharType), + varcharWriteFunction(), + ADB_STRING_COLLATION_AWARE_PUSHDOWN); + } + + private static ColumnMapping enumColumnMapping(ConnectorSession session, String jdbcTypeName) + { + //todo implement AdbAdvancedPushdownSessionProperties.isPushdownEnums(session); + boolean pushdownEnums = false; + PredicatePushdownController pushdownController = pushdownEnums ? ADB_STRING_COLLATION_AWARE_PUSHDOWN : DISABLE_PUSHDOWN; + return ColumnMapping.sliceMapping( + VARCHAR, + (resultSet, columnIndex) -> utf8Slice(resultSet.getString(columnIndex)), + typedVarcharWriteFunction(jdbcTypeName), + pushdownController); + } + + private static ColumnMapping timeColumnMapping(int precision) + { + // adb limit but also assumption within this method + verify(precision <= 6, "Unsupported precision: %s", precision); + return ColumnMapping.longMapping( + createTimeType(precision), + (resultSet, columnIndex) -> { + LocalTime time = resultSet.getObject(columnIndex, LocalTime.class); + long nanosOfDay = time.toNanoOfDay(); + if (nanosOfDay == NANOSECONDS_PER_DAY - 1) { + // adb 24:00:00 is returned as 23:59:59.999999999, regardless of column precision + nanosOfDay = NANOSECONDS_PER_DAY - LongMath.pow(10, 9 - precision); + } + + long picosOfDay = nanosOfDay * PICOSECONDS_PER_NANOSECOND; + return round(picosOfDay, 12 - precision); + }, + timeWriteFunction(precision), + // Pushdown disabled because adb distinguishes TIME '24:00:00' and TIME '00:00:00' whereas Trino does not. + DISABLE_PUSHDOWN); + } + + private static void shortTimestampWriteFunction(PreparedStatement statement, int index, long epochMicros) + throws SQLException + { + LocalDateTime localDateTime = fromTrinoTimestamp(epochMicros); + statement.setObject(index, toPgTimestamp(localDateTime)); + } + + private static ObjectWriteFunction longTimestampWriteFunction() + { + return ObjectWriteFunction.of(LongTimestamp.class, (statement, index, timestamp) -> { + // adb supports up to 6 digits of precision + //noinspection ConstantConditions + verify(POSTGRESQL_MAX_SUPPORTED_TIMESTAMP_PRECISION == 6); + long epochMicros = timestamp.getEpochMicros(); + if (timestamp.getPicosOfMicro() >= PICOSECONDS_PER_MICROSECOND / 2) { + epochMicros++; + } + shortTimestampWriteFunction(statement, index, epochMicros); + }); + } + + private Optional arrayToTrinoType(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + checkArgument(typeHandle.jdbcType() == Types.ARRAY, "Not array type"); + AdbPluginConfig.ArrayMapping arrayMapping = getArrayMapping(session); + if (arrayMapping == DISABLED) { + return Optional.empty(); + } + JdbcTypeHandle baseElementTypeHandle = getArrayElementTypeHandle(connection, typeHandle); + String baseElementTypeName = baseElementTypeHandle.jdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Element type name is missing: " + baseElementTypeHandle)); + if (baseElementTypeHandle.jdbcType() == Types.BINARY) { + // adb jdbc driver doesn't currently support array of varbinary (bytea[]) + return Optional.empty(); + } + Optional baseElementMapping = toColumnMapping(session, connection, baseElementTypeHandle); + + if (arrayMapping == AS_ARRAY) { + if (typeHandle.arrayDimensions().isEmpty()) { + return Optional.empty(); + } + return baseElementMapping + .map(elementMapping -> { + ArrayType trinoArrayType = new ArrayType(elementMapping.getType()); + ColumnMapping arrayColumnMapping = arrayColumnMapping(session, trinoArrayType, elementMapping, baseElementTypeName); + + int arrayDimensions = typeHandle.arrayDimensions().get(); + for (int i = 1; i < arrayDimensions; i++) { + trinoArrayType = new ArrayType(trinoArrayType); + arrayColumnMapping = arrayColumnMapping(session, trinoArrayType, arrayColumnMapping, baseElementTypeName); + } + return arrayColumnMapping; + }); + } + if (arrayMapping == AS_JSON) { + return baseElementMapping + .map(elementMapping -> arrayAsJsonColumnMapping(session, elementMapping)); + } + throw new IllegalStateException("Unsupported array mapping type: " + arrayMapping); + } + + private static ColumnMapping arrayColumnMapping(ConnectorSession session, ArrayType arrayType, ColumnMapping arrayElementMapping, String baseElementJdbcTypeName) + { + return ColumnMapping.objectMapping( + arrayType, + arrayReadFunction(arrayType.getElementType(), arrayElementMapping.getReadFunction()), + arrayWriteFunction(session, arrayType.getElementType(), baseElementJdbcTypeName)); + } + + private static ObjectWriteFunction arrayWriteFunction(ConnectorSession session, Type elementType, String baseElementJdbcTypeName) + { + return ObjectWriteFunction.of(Block.class, (statement, index, block) -> { + Array jdbcArray = statement.getConnection().createArrayOf(baseElementJdbcTypeName, getJdbcObjectArray(session, elementType, block)); + statement.setArray(index, jdbcArray); + }); + } + + private ColumnMapping arrayAsJsonColumnMapping(ConnectorSession session, ColumnMapping baseElementMapping) + { + return ColumnMapping.sliceMapping( + jsonType, + arrayAsJsonReadFunction(session, baseElementMapping), + (statement, index, block) -> { + throw new TrinoException(NOT_SUPPORTED, "Writing to array type is unsupported"); + }, + DISABLE_PUSHDOWN); + } + + private static JdbcTypeHandle getArrayElementTypeHandle(Connection connection, JdbcTypeHandle arrayTypeHandle) + { + String jdbcTypeName = arrayTypeHandle.jdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + arrayTypeHandle)); + try { + TypeInfo typeInfo = connection.unwrap(PgConnection.class).getTypeInfo(); + int pgElementOid = typeInfo.getPGArrayElement(typeInfo.getPGType(jdbcTypeName)); + verify(arrayTypeHandle.caseSensitivity().isEmpty(), "Case sensitivity not supported"); + return new JdbcTypeHandle( + typeInfo.getSQLType(pgElementOid), + Optional.of(typeInfo.getPGType(pgElementOid)), + arrayTypeHandle.columnSize(), + arrayTypeHandle.decimalDigits(), + Optional.empty(), + Optional.empty()); + } + catch (SQLException e) { + throw new TrinoException(JDBC_ERROR, e); + } + } + + private static SliceReadFunction arrayAsJsonReadFunction(ConnectorSession session, ColumnMapping baseElementMapping) + { + return (resultSet, columnIndex) -> { + Object jdbcArray = resultSet.getArray(columnIndex).getArray(); + int arrayDimensions = arrayDepth(jdbcArray); + + ReadFunction readFunction = baseElementMapping.getReadFunction(); + Type type = baseElementMapping.getType(); + for (int i = 0; i < arrayDimensions; i++) { + readFunction = arrayReadFunction(type, readFunction); + type = new ArrayType(type); + } + Block block = (Block) ((ObjectReadFunction) readFunction).readObject(resultSet, columnIndex); + BlockBuilder builder = type.createBlockBuilder(null, 1); + type.writeObject(builder, block); + Object value = type.getObjectValue(session, builder.build(), 0); + + try { + return toJsonValue(value); + } + catch (IOException e) { + throw new TrinoException(JDBC_ERROR, "Conversion to JSON failed for " + type.getDisplayName(), e); + } + }; + } + + private static ObjectReadFunction arrayReadFunction(Type elementType, ReadFunction elementReadFunction) + { + return ObjectReadFunction.of(Block.class, (resultSet, columnIndex) -> { + Array array = resultSet.getArray(columnIndex); + BlockBuilder builder = elementType.createBlockBuilder(null, 10); + try (ResultSet arrayAsResultSet = array.getResultSet()) { + while (arrayAsResultSet.next()) { + if (elementReadFunction.isNull(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)) { + builder.appendNull(); + } + else if (elementType.getJavaType() == boolean.class) { + elementType.writeBoolean(builder, ((BooleanReadFunction) elementReadFunction).readBoolean(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); + } + else if (elementType.getJavaType() == long.class) { + elementType.writeLong(builder, ((LongReadFunction) elementReadFunction).readLong(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); + } + else if (elementType.getJavaType() == double.class) { + elementType.writeDouble(builder, ((DoubleReadFunction) elementReadFunction).readDouble(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); + } + else if (elementType.getJavaType() == Slice.class) { + elementType.writeSlice(builder, ((SliceReadFunction) elementReadFunction).readSlice(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); + } + else { + elementType.writeObject(builder, ((ObjectReadFunction) elementReadFunction).readObject(arrayAsResultSet, ARRAY_RESULT_SET_VALUE_COLUMN)); + } + } + } + return builder.build(); + }); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/ColumnValue.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/ColumnValue.java new file mode 100644 index 000000000000..9fd5c439a85a --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/ColumnValue.java @@ -0,0 +1,18 @@ +/* + * 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.adb.connector.decode; + +public record ColumnValue(Object value, long estimatedSize) +{ +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoder.java new file mode 100644 index 000000000000..3cb1a52c87e0 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoder.java @@ -0,0 +1,24 @@ +/* + * 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.adb.connector.decode; + +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; + +public interface RowDecoder +{ + ConnectorRow decode(T[] data); + + DataFormat getFormat(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoderFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoderFactory.java new file mode 100644 index 000000000000..a83e9f8253a6 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/RowDecoderFactory.java @@ -0,0 +1,24 @@ +/* + * 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.adb.connector.decode; + +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.spi.connector.ConnectorSession; + +import java.util.List; + +public interface RowDecoderFactory +{ + RowDecoder create(ConnectorSession session, List columnDataTypes); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoder.java new file mode 100644 index 000000000000..cfdfaf6eafce --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoder.java @@ -0,0 +1,403 @@ +/* + * 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.adb.connector.decode.csv; + +import com.google.common.math.LongMath; +import io.airlift.slice.SizeOf; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.plugin.adb.connector.datatype.CharDataType; +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.adb.connector.datatype.ConnectorDataType; +import io.trino.plugin.adb.connector.datatype.DecimalLongDataType; +import io.trino.plugin.adb.connector.datatype.DecimalShortDataType; +import io.trino.plugin.adb.connector.datatype.TimeDataType; +import io.trino.plugin.adb.connector.datatype.TimestampWithoutTimeZoneDataType; +import io.trino.plugin.adb.connector.datatype.VarcharDataType; +import io.trino.plugin.adb.connector.decode.ColumnValue; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.GpfdistConnectorRow; +import io.trino.plugin.base.util.JsonTypeUtil; +import io.trino.plugin.jdbc.StandardColumnMappings; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DateTimeEncoding; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.Int128; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.TimeZoneKey; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Timestamps; +import io.trino.spi.type.Type; +import io.trino.spi.type.UuidType; +import io.trino.spi.type.VarcharType; +import org.postgresql.util.PGbytea; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.charset.StandardCharsets; +import java.sql.SQLException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.BiFunction; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.io.BaseEncoding.base16; +import static io.airlift.slice.SliceUtf8.countCodePoints; +import static io.trino.plugin.adb.TypeUtil.DATE_TYPE_FORMATTER; +import static io.trino.plugin.adb.TypeUtil.TIMESTAMP_TYPE_FORMATTER; +import static java.lang.String.format; + +public class CsvRowDecoder + implements RowDecoder +{ + private static final String DECODE_VALUE_ERROR_MSG_TEMPLATE = "Unexpected value %s for type %s"; + private static final Map> DECODE_FUNCTIONS_MAP = + Map.ofEntries( + Map.entry(ConnectorDataType.BOOLEAN, decodeBoolean()), + Map.entry(ConnectorDataType.MONEY, decodeMoney()), + Map.entry(ConnectorDataType.UUID, decodeUuid()), + Map.entry(ConnectorDataType.JSONB, decodeJsonb()), + Map.entry(ConnectorDataType.BIT, decodeBit()), + Map.entry(ConnectorDataType.BIGINT, decodeBigint()), + Map.entry(ConnectorDataType.BYTEA, decodeBytes()), + Map.entry(ConnectorDataType.CHAR, decodeChar()), + Map.entry(ConnectorDataType.VARCHAR, decodeVarchar()), + Map.entry(ConnectorDataType.DECIMAL_SHORT, decodeDecimalShort()), + Map.entry(ConnectorDataType.DECIMAL_LONG, decodeDecimalLong()), + Map.entry(ConnectorDataType.INTEGER, decodeInteger()), + Map.entry(ConnectorDataType.SMALLINT, decodeSmallint()), + Map.entry(ConnectorDataType.REAL, decodeReal()), + Map.entry(ConnectorDataType.DOUBLE_PRECISION, decodeDouble()), + Map.entry(ConnectorDataType.DATE, decodeDate()), + Map.entry(ConnectorDataType.TIME, decodeTime()), + Map.entry(ConnectorDataType.TIMESTAMP_SHORT_WITH_TIME_ZONE, decodeTimestampShortWithTimeZone()), + Map.entry(ConnectorDataType.TIMESTAMP_LONG_WITH_TIME_ZONE, decodeTimestampLongWithTimeZone()), + Map.entry(ConnectorDataType.TIMESTAMP_WITHOUT_TIME_ZONE, decodeTimestampWithoutTimeZone())); + + private final List columnDataTypes; + private final List> decodeFunctions; + + public CsvRowDecoder(List columnDataTypes) + { + this.columnDataTypes = columnDataTypes; + decodeFunctions = columnDataTypes.stream() + .map(columnDataType -> { + BiFunction decodeFunc = + DECODE_FUNCTIONS_MAP.get(columnDataType.getType()); + if (decodeFunc == null) { + throw new IllegalArgumentException(format("Unsupported column %s with type %s", + columnDataType.getName(), + columnDataType)); + } + else { + return decodeFunc; + } + }) + .toList(); + checkArgument(columnDataTypes.size() == decodeFunctions.size(), + "Column list size does not match decode function list size"); + } + + @Override + public ConnectorRow decode(String[] data) + { + try { + checkArgument(data.length == decodeFunctions.size(), + format("Parsed string values array length does not match decode function list size. Expected: %d, actual: %s", + decodeFunctions.size(), data.length)); + return decodeColumnsValues(data); + } + catch (Exception e) { + throw new IllegalArgumentException("Failed to decode data: " + e.getMessage(), e); + } + } + + private ConnectorRow decodeColumnsValues(String[] tokens) + { + ColumnValue[] values = new ColumnValue[tokens.length]; + long estimatedSize = 0L; + for (int i = 0; i < values.length; i++) { + ColumnValue columnValue = decodeValue(tokens[i], i); + values[i] = columnValue; + estimatedSize += columnValue.estimatedSize(); + } + return new GpfdistConnectorRow(estimatedSize, values); + } + + private ColumnValue decodeValue(String data, int position) + { + if (data != null) { + return decodeFunctions.get(position).apply(columnDataTypes.get(position), data); + } + else { + return new ColumnValue(null, 0); + } + } + + @Override + public DataFormat getFormat() + { + return DataFormat.CSV; + } + + private static BiFunction decodeBoolean() + { + return (_, data) -> new ColumnValue(getBooleanFromString(data), SizeOf.BOOLEAN_INSTANCE_SIZE); + } + + private static BiFunction decodeMoney() + { + return (_, data) -> { + Slice value = Slices.utf8Slice(data); + return new ColumnValue(value, value.getRetainedSize()); + }; + } + + private static BiFunction decodeBigint() + { + return (_, data) -> new ColumnValue(Long.parseLong(data), SizeOf.LONG_INSTANCE_SIZE); + } + + private static BiFunction decodeUuid() + { + return (_, data) -> { + Slice value = UuidType.javaUuidToTrinoUuid(UUID.fromString(data)); + return new ColumnValue(value, value.getRetainedSize()); + }; + } + + private static BiFunction decodeJsonb() + { + return (_, data) -> { + Slice value = JsonTypeUtil.jsonParse(Slices.utf8Slice(data)); + return new ColumnValue(value, value.getRetainedSize()); + }; + } + + private static BiFunction decodeBit() + { + return (_, data) -> new ColumnValue(getBooleanFromBitString(data), SizeOf.BOOLEAN_INSTANCE_SIZE); + } + + private static BiFunction decodeBytes() + { + return (_, data) -> { + try { + Slice value = Slices.wrappedBuffer(PGbytea.toBytes(data.getBytes(StandardCharsets.UTF_8))); + return new ColumnValue(value, value.getRetainedSize()); + } + catch (SQLException e) { + throw new IllegalArgumentException(format(DECODE_VALUE_ERROR_MSG_TEMPLATE, data, ConnectorDataType.BYTEA)); + } + }; + } + + private static BiFunction decodeChar() + { + return (dataType, data) -> { + CharType charType = ((CharDataType) dataType).getCharType(); + int trimCount = 0; + for (int i = data.length() - 1; i >= 0 && data.charAt(i) == ' '; i--) { + trimCount++; + } + if (trimCount > 0) { + data = data.substring(0, data.length() - trimCount); + } + Slice value = Slices.utf8Slice(data); + checkLengthInCodePoints(value, charType, charType.getLength()); + return new ColumnValue(value, value.getRetainedSize()); + }; + } + + private static BiFunction decodeVarchar() + { + return (dataType, data) -> { + VarcharType varcharType = ((VarcharDataType) dataType).getVarcharType(); + Slice value = Slices.utf8Slice(data); + if (!varcharType.isUnbounded()) { + checkLengthInCodePoints(value, varcharType, varcharType.getBoundedLength()); + } + return new ColumnValue(value, value.getRetainedSize()); + }; + } + + private static BiFunction decodeDecimalShort() + { + return (dataType, data) -> { + DecimalType decimalType = ((DecimalShortDataType) dataType).getDecimalType(); + return new ColumnValue(Decimals.encodeShortScaledValue(new BigDecimal(data), + decimalType.getScale(), + RoundingMode.UNNECESSARY), + SizeOf.LONG_INSTANCE_SIZE); + }; + } + + private static BiFunction decodeDecimalLong() + { + return (dataType, data) -> { + DecimalType decimalType = ((DecimalLongDataType) dataType).getDecimalType(); + return new ColumnValue(Decimals.valueOf(new BigDecimal(data).setScale(decimalType.getScale(), + RoundingMode.UNNECESSARY)), + Int128.INSTANCE_SIZE); + }; + } + + private static BiFunction decodeInteger() + { + return (_, data) -> { + long value = Integer.parseInt(data); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeSmallint() + { + return (_, data) -> { + long value = Short.parseShort(data); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeReal() + { + return (_, data) -> { + long value = Float.floatToRawIntBits(Float.parseFloat(data)); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeDouble() + { + return (_, data) -> { + double value = Double.parseDouble(data); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeDate() + { + return (_, data) -> new ColumnValue(LocalDate.parse(data, DATE_TYPE_FORMATTER).toEpochDay(), SizeOf.LONG_INSTANCE_SIZE); + } + + private static BiFunction decodeTime() + { + return (dataType, data) -> { + int precision = ((TimeDataType) dataType).getPrecision(); + LocalTime time; + if ("24:00:00".equals(data)) { + time = LocalTime.MAX; + } + else { + time = LocalTime.parse(data); + } + long nanosOfDay = time.toNanoOfDay(); + if (nanosOfDay == 86399999999999L) { + nanosOfDay = 86400000000000L - LongMath.pow(10L, 9 - precision); + } + long picosOfDay = nanosOfDay * 1000L; + long value = Timestamps.round(picosOfDay, 12 - precision); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeTimestampShortWithTimeZone() + { + return (_, data) -> { + LongTimestampWithTimeZone timestampValue = decodeTimestampWithTimeZoneLong(data); + long millisUtc = timestampValue.getEpochMillis(); + long value = DateTimeEncoding.packDateTimeWithZone(millisUtc, TimeZoneKey.getTimeZoneKey(timestampValue.getTimeZoneKey())); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static BiFunction decodeTimestampLongWithTimeZone() + { + return (_, data) -> new ColumnValue(decodeTimestampWithTimeZoneLong(data), LongTimestampWithTimeZone.INSTANCE_SIZE); + } + + private static BiFunction decodeTimestampWithoutTimeZone() + { + return (dataType, data) -> { + TimestampType timestampType = ((TimestampWithoutTimeZoneDataType) dataType).getTimestampType(); + LocalDateTime res = LocalDateTime.parse(data, TIMESTAMP_TYPE_FORMATTER); + long value = StandardColumnMappings.toTrinoTimestamp(timestampType, res); + return new ColumnValue(value, SizeOf.sizeOf(value)); + }; + } + + private static boolean getBooleanFromString(String data) + { + if ("t".equals(data)) { + return true; + } + else if ("f".equals(data)) { + return false; + } + else { + throw new IllegalArgumentException(format(DECODE_VALUE_ERROR_MSG_TEMPLATE, data, ConnectorDataType.BOOLEAN)); + } + } + + private static boolean getBooleanFromBitString(String data) + { + if ("1".equals(data) + || "true".equalsIgnoreCase(data) + || "t".equalsIgnoreCase(data) + || "yes".equalsIgnoreCase(data) + || "y".equalsIgnoreCase(data) + || "on".equalsIgnoreCase(data)) { + return true; + } + else if (!"0".equals(data) + && !"false".equalsIgnoreCase(data) + && !"f".equalsIgnoreCase(data) + && !"no".equalsIgnoreCase(data) + && !"n".equalsIgnoreCase(data) + && !"off".equalsIgnoreCase(data)) { + throw new IllegalArgumentException(format(DECODE_VALUE_ERROR_MSG_TEMPLATE, data, ConnectorDataType.BIT)); + } + else { + return false; + } + } + + private static void checkLengthInCodePoints(Slice value, Type characterDataType, int lengthLimit) + { + if (value.length() > lengthLimit) { + if (countCodePoints(value) > lengthLimit) { + throw new IllegalStateException( + String.format("Illegal value for trino type %s: '%s' [%s]", + characterDataType, + value.toStringUtf8(), + base16().encode(value.getBytes()))); + } + } + } + + private static LongTimestampWithTimeZone decodeTimestampWithTimeZoneLong(String data) + { + OffsetDateTime offsetDateTime = OffsetDateTime.parse(data, TIMESTAMP_TYPE_FORMATTER); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction(offsetDateTime.toEpochSecond(), + (long) offsetDateTime.getNano() * 1000L, + TimeZoneKey.UTC_KEY); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoderFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoderFactory.java new file mode 100644 index 000000000000..2c544da54268 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/decode/csv/CsvRowDecoderFactory.java @@ -0,0 +1,31 @@ +/* + * 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.adb.connector.decode.csv; + +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.decode.RowDecoderFactory; +import io.trino.spi.connector.ConnectorSession; + +import java.util.List; + +public class CsvRowDecoderFactory + implements RowDecoderFactory +{ + @Override + public RowDecoder create(ConnectorSession session, List columnDataTypes) + { + return new CsvRowDecoder(columnDataTypes); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/AbstractRowEncoder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/AbstractRowEncoder.java index 55336c9f0e6e..857f84d0b226 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/AbstractRowEncoder.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/AbstractRowEncoder.java @@ -116,10 +116,12 @@ public void appendColumnValue(Block block, int position) if (block.isNull(position)) { appendNullValue(); } - BiConsumer func = map.get(type); - checkArgument(func != null, - format("Encoder for column type '%s' is not supported", type)); - func.accept(this, new EncoderMetadata(columnDataType, block, position)); + else { + BiConsumer func = map.get(type); + checkArgument(func != null, + format("Encoder for column type '%s' is not supported", type)); + func.accept(this, new EncoderMetadata(columnDataType, block, position)); + } currentColumnIndex++; } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderFormat.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormat.java similarity index 95% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderFormat.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormat.java index 2a01fb753784..e1ca94d1c151 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderFormat.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormat.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.adb.connector.encode; -public enum EncoderFormat +public enum DataFormat { CSV } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderConfig.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatConfig.java similarity index 89% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderConfig.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatConfig.java index ee6203a0181e..52a29ab27fe6 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderConfig.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatConfig.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.adb.connector.encode; -public interface EncoderConfig +public interface DataFormatConfig { - EncoderFormat getEncoderFormat(); + DataFormat getDataFormat(); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderModule.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatModule.java similarity index 67% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderModule.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatModule.java index 003343e9be84..377652c65e84 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/EncoderModule.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/DataFormatModule.java @@ -15,17 +15,23 @@ import com.google.inject.Binder; import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.plugin.adb.connector.encode.csv.CsvEncoderConfig; +import io.trino.plugin.adb.connector.decode.RowDecoderFactory; +import io.trino.plugin.adb.connector.decode.csv.CsvRowDecoderFactory; +import io.trino.plugin.adb.connector.encode.csv.CsvFormatConfig; import io.trino.plugin.adb.connector.encode.csv.CsvRowEncoderFactory; -public class EncoderModule +public class DataFormatModule extends AbstractConfigurationAwareModule { @Override protected void setup(Binder binder) { - binder.bind(EncoderConfig.class).to(CsvEncoderConfig.class).in(Scopes.SINGLETON); + binder.bind(DataFormatConfig.class).to(CsvFormatConfig.class).in(Scopes.SINGLETON); binder.bind(RowEncoderFactory.class).to(CsvRowEncoderFactory.class).in(Scopes.SINGLETON); + binder.bind(new TypeLiteral>() + { + }).to(CsvRowDecoderFactory.class).in(Scopes.SINGLETON); } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/RowEncoder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/RowEncoder.java index 11424f17578c..c6208a748770 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/RowEncoder.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/RowEncoder.java @@ -24,5 +24,5 @@ public interface RowEncoder byte[] toByteArray(); - EncoderFormat getFormat(); + DataFormat getFormat(); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvEncoderConfig.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvFormatConfig.java similarity index 63% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvEncoderConfig.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvFormatConfig.java index 520c84ca97d6..59a72ee2e6a2 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvEncoderConfig.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvFormatConfig.java @@ -13,28 +13,34 @@ */ package io.trino.plugin.adb.connector.encode.csv; -import io.trino.plugin.adb.connector.encode.EncoderConfig; -import io.trino.plugin.adb.connector.encode.EncoderFormat; +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.encode.DataFormatConfig; -public class CsvEncoderConfig - implements EncoderConfig +public class CsvFormatConfig + implements DataFormatConfig { private char delimiter = '|'; - private String nullValue = "\\0"; + private String nullValue; private String encoding = "UTF-8"; - public static CsvEncoderConfig create() + public CsvFormatConfig() { - return new CsvEncoderConfig(); + //default value, otherwise checkstyle plugin will raise error + nullValue = null; } - public CsvEncoderConfig delimiter(char delimiter) + public static CsvFormatConfig create() + { + return new CsvFormatConfig(); + } + + public CsvFormatConfig delimiter(char delimiter) { this.delimiter = delimiter; return this; } - public CsvEncoderConfig nullValue(String nullValue) + public CsvFormatConfig nullValue(String nullValue) { this.nullValue = nullValue; return this; @@ -51,12 +57,12 @@ public String getNullValue() } @Override - public EncoderFormat getEncoderFormat() + public DataFormat getDataFormat() { - return EncoderFormat.CSV; + return DataFormat.CSV; } - public CsvEncoderConfig encoding(String encoding) + public CsvFormatConfig encoding(String encoding) { this.encoding = encoding; return this; diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoder.java index ff2555e842d6..046d4215b362 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoder.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.adb.connector.datatype.ColumnDataType; import io.trino.plugin.adb.connector.datatype.ConnectorDataType; import io.trino.plugin.adb.connector.encode.AbstractRowEncoder; -import io.trino.plugin.adb.connector.encode.EncoderFormat; +import io.trino.plugin.adb.connector.encode.DataFormat; import io.trino.spi.connector.ConnectorSession; import org.postgresql.util.PGbytea; @@ -43,10 +43,10 @@ public class CsvRowEncoder extends AbstractRowEncoder { - private final CsvEncoderConfig encoderConfig; + private final CsvFormatConfig encoderConfig; private final String[] row; - public CsvRowEncoder(ConnectorSession session, List columnDataTypes, CsvEncoderConfig encoderConfig) + public CsvRowEncoder(ConnectorSession session, List columnDataTypes, CsvFormatConfig encoderConfig) { super(session, columnDataTypes); for (ColumnDataType columnDataType : this.columnDataTypes) { @@ -165,8 +165,8 @@ public byte[] toByteArray() } @Override - public EncoderFormat getFormat() + public DataFormat getFormat() { - return EncoderFormat.CSV; + return DataFormat.CSV; } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoderFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoderFactory.java index dc81cd146da2..f469a2d57e6d 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoderFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/encode/csv/CsvRowEncoderFactory.java @@ -15,8 +15,8 @@ import com.google.inject.Inject; import io.trino.plugin.adb.connector.datatype.ColumnDataType; -import io.trino.plugin.adb.connector.encode.EncoderConfig; -import io.trino.plugin.adb.connector.encode.EncoderFormat; +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.encode.DataFormatConfig; import io.trino.plugin.adb.connector.encode.RowEncoder; import io.trino.plugin.adb.connector.encode.RowEncoderFactory; import io.trino.spi.connector.ConnectorSession; @@ -28,21 +28,21 @@ public class CsvRowEncoderFactory implements RowEncoderFactory { - private final EncoderConfig encoderConfig; + private final DataFormatConfig dataFormatConfig; @Inject - public CsvRowEncoderFactory(EncoderConfig encoderConfig) + public CsvRowEncoderFactory(DataFormatConfig dataFormatConfig) { - checkArgument(encoderConfig.getEncoderFormat() == EncoderFormat.CSV, + checkArgument(dataFormatConfig.getDataFormat() == DataFormat.CSV, "Unexpected encoder config format '%s' defined for current encoder factory '%s'", - encoderConfig.getEncoderFormat(), + dataFormatConfig.getDataFormat(), getClass().getSimpleName()); - this.encoderConfig = encoderConfig; + this.dataFormatConfig = dataFormatConfig; } @Override public RowEncoder create(ConnectorSession session, List columnDataTypes) { - return new CsvRowEncoder(session, columnDataTypes, (CsvEncoderConfig) encoderConfig); + return new CsvRowEncoder(session, columnDataTypes, (CsvFormatConfig) dataFormatConfig); } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractContextManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractContextManager.java new file mode 100644 index 000000000000..3ba9b97b53e5 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractContextManager.java @@ -0,0 +1,48 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +public abstract class AbstractContextManager + implements ContextManager +{ + private static final Logger log = Logger.get(AbstractContextManager.class); + private final Map contextMap = new ConcurrentHashMap<>(); + + @Override + public Optional get(ContextId contextId) + { + return Optional.ofNullable(contextMap.get(contextId)); + } + + @Override + public void add(T context) + { + contextMap.put(context.getId(), context); + log.debug("Added context %s", context.getId()); + } + + @Override + public void remove(ContextId contextId) + { + Optional.ofNullable(contextMap.remove(contextId)) + .ifPresent(ctx -> log.info("Removed context %s", ctx.getId())); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractDataTransferQueryExecutor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractDataTransferQueryExecutor.java new file mode 100644 index 000000000000..45726673ff72 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractDataTransferQueryExecutor.java @@ -0,0 +1,82 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; + +public abstract class AbstractDataTransferQueryExecutor + implements DataTransferQueryExecutor +{ + private static final Logger log = Logger.get(AbstractDataTransferQueryExecutor.class); + protected final AdbSqlClient client; + protected final ConnectorSession session; + protected final ExecutorService executor; + protected final CreateExternalTableQueryFactory externalTableQueryFactory; + protected final InsertDataQueryFactory insertDataQueryFactory; + protected Connection connection; + + public AbstractDataTransferQueryExecutor(AdbSqlClient client, + ConnectorSession session, + ExecutorService executor, + CreateExternalTableQueryFactory externalTableQueryFactory, + InsertDataQueryFactory insertDataQueryFactory) + { + this.client = client; + this.session = session; + this.executor = executor; + this.externalTableQueryFactory = externalTableQueryFactory; + this.insertDataQueryFactory = insertDataQueryFactory; + } + + @Override + public CompletableFuture execute() + { + return CompletableFuture.supplyAsync(() -> { + try { + connection = client.getConnection(session); + connection.setAutoCommit(false); + connection.setReadOnly(false); + executeQueries(); + connection.commit(); + connection.close(); + return null; + } + catch (Exception e) { + String errMsg = "Failed to execute data transfer query: " + e.getMessage(); + log.error(errMsg, e); + throw new RuntimeException(errMsg, e); + } + finally { + if (connection != null) { + try { + connection.close(); + } + catch (SQLException e) { + log.warn("Failed to close connection", e); + } + } + } + }, executor); + } + + protected abstract void executeQueries() + throws SQLException; +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractExternalTableQueryFactory.java new file mode 100644 index 000000000000..8ac40af9df07 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/AbstractExternalTableQueryFactory.java @@ -0,0 +1,47 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; + +import java.util.stream.IntStream; + +import static java.lang.String.format; +import static java.util.stream.Collectors.joining; + +public abstract class AbstractExternalTableQueryFactory + implements CreateExternalTableQueryFactory +{ + protected String createCommonQuery(GpfdistMetadata metadata) + { + String columnsDefinition = IntStream.range(0, metadata.getColumnNames().size()) + .boxed() + .map(i -> { + String columnName = metadata.getColumnNames().get(i); + String typeName = metadata.getDataTypes().get(i).getName(); + return columnName + " " + typeName; + }) + .collect(joining(",")); + return format( + "CREATE %s EXTERNAL TEMPORARY TABLE %s (%s) LOCATION ('%s') FORMAT '%s' (DELIMITER '%s' NULL AS '%s') ENCODING '%s'", + getExternalTableType().name(), + metadata.getSourceTable(), + columnsDefinition, + metadata.getGpfdistLocation(), + metadata.getExternalTableFormatConfig().dataFormat().name(), + metadata.getExternalTableFormatConfig().delimiter(), + metadata.getExternalTableFormatConfig().nullValue() == null ? "" : metadata.getExternalTableFormatConfig().nullValue(), + metadata.getExternalTableFormatConfig().encoding()); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ConnectorRow.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ConnectorRow.java new file mode 100644 index 000000000000..5915b921ae6a --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ConnectorRow.java @@ -0,0 +1,25 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.trino.plugin.adb.connector.decode.ColumnValue; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.WithEstimatedSize; + +public interface ConnectorRow + extends WithEstimatedSize +{ + ColumnValue[] getColumnValues(); + + long getEstimatedSize(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/Context.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/Context.java new file mode 100644 index 000000000000..5a39a57d04cd --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/Context.java @@ -0,0 +1,23 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; + +public interface Context +{ + ContextId getId(); + + void close(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ContextManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ContextManager.java new file mode 100644 index 000000000000..b3dc0ff3f6cb --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ContextManager.java @@ -0,0 +1,27 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; + +import java.util.Optional; + +public interface ContextManager +{ + Optional get(ContextId contextId); + + void add(T context); + + void remove(ContextId contextId); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/CreateExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/CreateExternalTableQueryFactory.java index 6fde33bf793a..027128d748f8 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/CreateExternalTableQueryFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/CreateExternalTableQueryFactory.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; public interface CreateExternalTableQueryFactory diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistModule.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistModule.java index 65643eee636b..171e8e03f2a7 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistModule.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistModule.java @@ -17,6 +17,7 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.Singleton; +import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.multibindings.Multibinder; import com.google.inject.multibindings.OptionalBinder; @@ -28,19 +29,29 @@ import io.airlift.node.NodeConfig; import io.airlift.node.NodeInfo; import io.opentelemetry.api.OpenTelemetry; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.CreateReadableExternalTableQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.GpfdistPageSinkProvider; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.InsertDataFromExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContextManager; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContextManagerImpl; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.process.GpfdistPageSinkProvider; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.query.CreateReadableExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.query.InsertDataFromExternalTableQueryFactory; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableFormatConfigFactory; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableFormatConfigFactoryImpl; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadataFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadataFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadataFactoryImpl; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLocationFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLocationFactoryImpl; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadataFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadataFactoryImpl; import io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistServerConfig; import io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistServerModule; -import io.trino.plugin.adb.connector.protocol.gpfdist.unload.CreateWritableExternalTableQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InsertDataToExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessorFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContext; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContextManager; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.process.GpfdistInputDataProcessorFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.process.GpfdistRecordSetProvider; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.query.CreateWritableExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.query.InsertDataToExternalTableQueryFactory; import io.trino.plugin.jdbc.BaseJdbcConfig; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.DriverConnectionFactory; @@ -48,6 +59,7 @@ import io.trino.plugin.jdbc.credential.CredentialProvider; import io.trino.spi.NodeManager; import io.trino.spi.connector.ConnectorPageSinkProvider; +import io.trino.spi.connector.ConnectorRecordSetProvider; import org.postgresql.Driver; import org.weakref.jmx.guice.ExportBinder; @@ -69,10 +81,21 @@ public void setup(Binder binder) Multibinder insertDataQueryFactories = Multibinder.newSetBinder(binder, InsertDataQueryFactory.class); insertDataQueryFactories.addBinding().to(InsertDataFromExternalTableQueryFactory.class).in(Scopes.SINGLETON); insertDataQueryFactories.addBinding().to(InsertDataToExternalTableQueryFactory.class).in(Scopes.SINGLETON); - OptionalBinder.newOptionalBinder(binder, ConnectorPageSinkProvider.class).setBinding().to(GpfdistPageSinkProvider.class).in(Scopes.SINGLETON); - binder.bind(GpfdistMetadataFactory.class).to(GpfdistLoadMetadataFactory.class).in(Scopes.SINGLETON); - binder.bind(WriteContextManager.class).to(WriteContextManagerImpl.class).in(Scopes.SINGLETON); + OptionalBinder.newOptionalBinder(binder, ConnectorRecordSetProvider.class).setBinding().to(GpfdistRecordSetProvider.class).in(Scopes.SINGLETON); + binder.bind(GpfdistLoadMetadataFactory.class).to(GpfdistLoadMetadataFactoryImpl.class).in(Scopes.SINGLETON); + binder.bind(GpfdistUnloadMetadataFactory.class).to(GpfdistUnloadMetadataFactoryImpl.class).in(Scopes.SINGLETON); + binder.bind(GpfdistLocationFactory.class).to(GpfdistLocationFactoryImpl.class).in(Scopes.SINGLETON); + binder.bind(GpfdistUnloadMetadataFactory.class).to(GpfdistUnloadMetadataFactoryImpl.class).in(Scopes.SINGLETON); + binder.bind(new TypeLiteral>() + { + }).to(ReadContextManager.class).in(Scopes.SINGLETON); + binder.bind(new TypeLiteral>() + { + }).to(WriteContextManager.class).in(Scopes.SINGLETON); + binder.bind(new TypeLiteral>() + { + }).to(GpfdistInputDataProcessorFactory.class).in(Scopes.SINGLETON); binder.bind(NodeInfo.class).in(Scopes.SINGLETON); binder.bind(HttpServerInfo.class).in(Scopes.SINGLETON); binder.bind(RequestStats.class).in(Scopes.SINGLETON); @@ -112,7 +135,6 @@ public static NodeConfig getNodeConfig(GpfdistServerConfig config, NodeManager n return new NodeConfig() .setEnvironment("adb_gpfdist") .setNodeId(nodeManager.getCurrentNode().getNodeIdentifier()) - //.setNodeBindIp(internalHost) .setNodeInternalAddress(internalHost) .setNodeExternalAddress(externalHost); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistUtil.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistUtil.java new file mode 100644 index 000000000000..6f5644df0b43 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/GpfdistUtil.java @@ -0,0 +1,26 @@ +/* + * 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.adb.connector.protocol.gpfdist; + +public final class GpfdistUtil +{ + private GpfdistUtil() + { + } + + public static String createGpfdistFileName(String externalTableName) + { + return "/adb/" + externalTableName; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/InsertDataQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/InsertDataQueryFactory.java index dd9ff4ead485..0ed5d45bb8ca 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/InsertDataQueryFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/InsertDataQueryFactory.java @@ -13,11 +13,16 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; +import io.trino.plugin.jdbc.PreparedQuery; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.Connection; public interface InsertDataQueryFactory { - String create(GpfdistMetadata gpfdistMetadata); + PreparedQuery create(ConnectorSession session, Connection connection, GpfdistMetadata gpfdistMetadata); ExternalTableType getExternalTableType(); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/CreateReadableExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/CreateReadableExternalTableQueryFactory.java deleted file mode 100644 index 6a9c87e999b9..000000000000 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/CreateReadableExternalTableQueryFactory.java +++ /dev/null @@ -1,56 +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.adb.connector.protocol.gpfdist.load; - -import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableType; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; - -import java.util.stream.IntStream; - -import static java.lang.String.format; -import static java.util.stream.Collectors.joining; - -public class CreateReadableExternalTableQueryFactory - implements CreateExternalTableQueryFactory -{ - @Override - public String createQuery(GpfdistMetadata metadata) - { - GpfdistLoadMetadata loadMetadata = (GpfdistLoadMetadata) metadata; - String columnsDefinition = IntStream.range(0, loadMetadata.getColumnNames().size()) - .boxed() - .map(i -> { - String columnName = loadMetadata.getColumnNames().get(i); - String typeName = loadMetadata.getDataTypes().get(i).getName(); - return columnName + " " + typeName; - }) - .collect(joining(",")); - return format( - "CREATE READABLE EXTERNAL TEMPORARY TABLE %s (%s) LOCATION ('%s') FORMAT '%s' (DELIMITER '%s' NULL AS '%s') ENCODING 'UTF8'", - loadMetadata.getSourceTableName(), - columnsDefinition, - loadMetadata.getGpfdistLocation(), - loadMetadata.getExternalTableFormatConfig().encoderFormat().name(), - loadMetadata.getExternalTableFormatConfig().delimiter(), - loadMetadata.getExternalTableFormatConfig().nullValue()); - } - - @Override - public ExternalTableType getExternalTableType() - { - return ExternalTableType.READABLE; - } -} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPacketBuilder.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPacketBuilder.java index efbf38278852..c00a253d2104 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPacketBuilder.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPacketBuilder.java @@ -74,6 +74,11 @@ public byte[] createDataPacket(PageSerializationResult serializationResult) return Bytes.concat(headerBuffer.array(), serializationResult.data()); } + public byte[] createSingleEmptyDataPacket() + { + return createDataPacket(new PageSerializationResult(new byte[] {}, 0)); + } + public byte[] createEndPacket() { endPacketBuffer.clear(); diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContext.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContext.java index 2838c3ebfaa8..6d41c662350e 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContext.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContext.java @@ -13,8 +13,10 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.load.context; +import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.trino.plugin.adb.connector.encode.RowEncoder; +import io.trino.plugin.adb.connector.protocol.gpfdist.Context; import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageProcessor; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; @@ -24,8 +26,12 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static java.lang.String.format; + public class WriteContext + implements Context { + private static final Logger log = Logger.get(WriteContext.class); private final ContextId id; private final GpfdistLoadMetadata metadata; private final ConcurrentLinkedQueue pageProcessors = new ConcurrentLinkedQueue<>(); @@ -37,6 +43,11 @@ public class WriteContext private final DataSize writeBufferSize; private final AtomicReference error = new AtomicReference<>(); + public WriteContext(GpfdistLoadMetadata metadata, RowEncoder rowEncoder, DataSize writeBufferSize) + { + this(new ContextId(metadata.getSourceTable()), metadata, rowEncoder, writeBufferSize); + } + public WriteContext(ContextId id, GpfdistLoadMetadata metadata, RowEncoder rowEncoder, @@ -48,6 +59,7 @@ public WriteContext(ContextId id, this.writeBufferSize = writeBufferSize; } + @Override public ContextId getId() { return id; @@ -98,9 +110,22 @@ public AtomicReference getError() return error; } + @Override public void close() { - pageProcessors.forEach(PageProcessor::stop); + StringBuilder sb = new StringBuilder(); + pageProcessors.forEach(processor -> { + try { + processor.stop(); + } + catch (Exception e) { + sb.append(format("Failed to stop page processor %s. Error: %s;", processor, e.getMessage())); + } + }); pageProcessors.clear(); + if (!sb.isEmpty()) { + throw new RuntimeException(sb.toString()); + } + log.debug("Closed write context %s", id); } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManager.java index 25de1a92bc06..3156fa2f6924 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManager.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManager.java @@ -13,15 +13,9 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.load.context; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractContextManager; -import java.util.Optional; - -public interface WriteContextManager +public class WriteContextManager + extends AbstractContextManager { - Optional get(ContextId contextId); - - void add(WriteContext context); - - void clear(ContextId contextId); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManagerImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManagerImpl.java deleted file mode 100644 index 0ae3c6be2a56..000000000000 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/context/WriteContextManagerImpl.java +++ /dev/null @@ -1,56 +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.adb.connector.protocol.gpfdist.load.context; - -import io.airlift.log.Logger; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; - -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; - -public class WriteContextManagerImpl - implements WriteContextManager -{ - private static final Logger log = Logger.get(WriteContextManagerImpl.class); - private final Map writeContextMap = new ConcurrentHashMap<>(); - - @Override - public Optional get(ContextId contextId) - { - return Optional.ofNullable(writeContextMap.get(contextId)); - } - - @Override - public void add(WriteContext context) - { - writeContextMap.put(context.getId(), context); - } - - @Override - public void clear(ContextId contextId) - { - if (writeContextMap.get(contextId) != null) { - try { - WriteContext writeContext = writeContextMap.get(contextId); - writeContext.close(); - writeContextMap.remove(contextId); - } - catch (Exception e) { - log.error("Failed to clear write context by contextId: %s", contextId, e); - throw e; - } - } - } -} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageProcessor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageProcessor.java similarity index 84% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageProcessor.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageProcessor.java index 22039b158e62..4525c316a306 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageProcessor.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageProcessor.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.load.processing; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.process; import io.airlift.log.Logger; import io.trino.plugin.adb.connector.encode.RowEncoder; @@ -20,7 +20,7 @@ import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageSerializationResult; import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageSerializer; import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; -import io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistReadableRequest; +import io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistReadableRequest; import io.trino.spi.Page; import io.trino.spi.StandardErrorCode; import io.trino.spi.TrinoException; @@ -29,6 +29,8 @@ import java.io.OutputStream; import java.nio.ByteBuffer; +import static io.trino.plugin.adb.connector.protocol.gpfdist.GpfdistUtil.createGpfdistFileName; + public class GpfdistPageProcessor implements PageProcessor { @@ -53,7 +55,7 @@ public GpfdistPageProcessor(OutputStream outputStream, this.request = request; this.pageSerializer = pageSerializer; pageLengthBuffer = ByteBuffer.allocate(4); - packetBuilder = new GpfdistPacketBuilder(createGpfdistFileName(writeContext.getMetadata().getSourceTableName())); + packetBuilder = new GpfdistPacketBuilder(createGpfdistFileName(writeContext.getMetadata().getSourceTable())); } @Override @@ -67,10 +69,10 @@ public void process(Page page) pageLengthBuffer.clear(); if (bytesWritten >= writeContext.getWriteBufferSize().toBytes()) { outputStream.flush(); + log.debug("Sent data packet with size %d for request %s", bytesWritten, request); writeContext.getCompletedBytes().addAndGet(bytesWritten); writeContext.getMemoryUsage().addAndGet(-bytesWritten); bytesWritten = 0; - log.debug("Sent data packet with size %d for request %s", bytesWritten, request); } } catch (Exception e) { @@ -85,8 +87,8 @@ public void stop() { Throwable error = writeContext.getError().get(); if (error != null) { - log.error("Stop writing process with error: %s", error); writePacket(packetBuilder.createErrorPacket(error)); + log.warn("Stopped writing process with error: %s. Sent error packet data", error); } else { writePacket(packetBuilder.createEndPacket()); @@ -102,7 +104,7 @@ private void writePacket(byte[] dataPacket) outputStream.flush(); } catch (IOException e) { - throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, "Failed to write end packet data", e); + throw new RuntimeException("Failed to write end packet data: " + e.getMessage(), e); } } @@ -113,12 +115,16 @@ private void close() rowEncoder.close(); } catch (IOException e) { - throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, "Failed to close processing data stream", e); + throw new RuntimeException("Failed to close processing data stream" + e.getMessage(), e); } } - private String createGpfdistFileName(String externalTableName) + @Override + public String toString() { - return "/adb/" + externalTableName; + return "GpfdistPageProcessor{" + + "request=" + request + + ", writeContext.id=" + writeContext.getId() + + '}'; } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSerializer.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSerializer.java similarity index 89% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSerializer.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSerializer.java index c7fb45fe2afa..6f46094da2ae 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSerializer.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSerializer.java @@ -11,15 +11,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.load.processing; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.process; import io.trino.plugin.adb.connector.datatype.ColumnDataType; import io.trino.plugin.adb.connector.encode.RowEncoder; import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageSerializationResult; import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageSerializer; import io.trino.spi.Page; -import io.trino.spi.StandardErrorCode; -import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import java.io.ByteArrayOutputStream; @@ -55,9 +53,7 @@ public PageSerializationResult serialize(Page page) return new PageSerializationResult(out.toByteArray(), rowCount); } catch (IOException e) { - throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, - "Failed to serialize data page: " + e.getMessage(), - e); + throw new RuntimeException("Failed to serialize data page: " + e.getMessage(), e); } } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSink.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSink.java similarity index 88% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSink.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSink.java index e93ee29681ee..3bf47aa7530d 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistPageSink.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSink.java @@ -11,16 +11,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.load.processing; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.process; import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; import io.trino.plugin.adb.connector.protocol.gpfdist.ExecutorServiceProvider; import io.trino.plugin.adb.connector.protocol.gpfdist.load.PageProcessor; import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContextManager; import io.trino.spi.Page; import io.trino.spi.StandardErrorCode; import io.trino.spi.TrinoException; @@ -36,13 +36,13 @@ public class GpfdistPageSink { private static final Logger log = Logger.get(GpfdistPageSink.class); private static final long ADB_SEGMENT_WAIT_TIMEOUT = 60000L; - private final WriteContextManager writeContextManager; + private final ContextManager writeContextManager; private final WriteContext writeContext; private final CompletableFuture queryLoadFuture; private final ExecutorService executorService; private CompletableFuture pageProcessingFuture; - public GpfdistPageSink(WriteContextManager writeContextManager, + public GpfdistPageSink(ContextManager writeContextManager, WriteContext writeContext, CompletableFuture queryLoadFuture) { @@ -50,6 +50,7 @@ public GpfdistPageSink(WriteContextManager writeContextManager, this.queryLoadFuture = queryLoadFuture; this.writeContextManager = writeContextManager; executorService = ExecutorServiceProvider.WRITE_DATA_EXECUTOR_SERVICE; + pageProcessingFuture = CompletableFuture.completedFuture(null); } @Override @@ -94,32 +95,27 @@ private void waitForProcessors() public CompletableFuture> finish() { return pageProcessingFuture - .thenCompose(_ -> clearContext()) + .thenCompose(_ -> closeCtx()) .thenCompose(_ -> queryLoadFuture) .thenApply(_ -> createFinishResult()) .whenComplete((_, e) -> { if (e != null) { log.error("Failed to load data into adb: %s", e); failContext(e); - writeContextManager.clear(writeContext.getId()); throw new CompletionException(e); } + writeContextManager.remove(writeContext.getId()); }); } - private CompletableFuture clearContext() + private CompletableFuture closeCtx() { return CompletableFuture.supplyAsync(() -> { - writeContextManager.clear(writeContext.getId()); + writeContext.close(); return null; }); } - private void failContext(Throwable e) - { - writeContext.getError().set(e); - } - private Collection createFinishResult() { Slice value = Slices.allocate(8); @@ -134,7 +130,17 @@ public void abort() queryLoadFuture.completeExceptionally(error); pageProcessingFuture.completeExceptionally(error); failContext(error); - writeContextManager.clear(writeContext.getId()); + } + + private void failContext(Throwable e) + { + try { + writeContext.getError().set(e); + writeContext.close(); + } + finally { + writeContextManager.remove(writeContext.getId()); + } } @Override diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPageSinkProvider.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSinkProvider.java similarity index 70% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPageSinkProvider.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSinkProvider.java index d8173e99e5ac..793bf35ef927 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/GpfdistPageSinkProvider.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/process/GpfdistPageSinkProvider.java @@ -11,25 +11,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.load; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.process; import com.google.inject.Inject; import io.trino.plugin.adb.AdbPluginConfig; import io.trino.plugin.adb.connector.AdbSqlClient; import io.trino.plugin.adb.connector.encode.RowEncoderFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; import io.trino.plugin.adb.connector.protocol.gpfdist.DataTransferQueryExecutor; import io.trino.plugin.adb.connector.protocol.gpfdist.ExecutorServiceProvider; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContextManager; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.processing.GpfdistDataTransferQueryExecutor; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.processing.GpfdistPageSink; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.query.GpfdistLoadDataTransferQueryExecutor; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableFormatConfigFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadataFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadataFactory; import io.trino.plugin.jdbc.ForBaseJdbc; import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcOutputTableHandle; @@ -48,24 +46,27 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkArgument; + public class GpfdistPageSinkProvider implements ConnectorPageSinkProvider { + private static final ExternalTableType EXTERNAL_TABLE_TYPE = ExternalTableType.READABLE; private final AdbSqlClient client; private final AdbPluginConfig pluginConfig; - private final GpfdistMetadataFactory loadMetadataFactory; + private final GpfdistLoadMetadataFactory loadMetadataFactory; private final ExecutorService loadQueryThreadExecutor; - private final WriteContextManager writeContextManager; + private final ContextManager contextManager; private final RowEncoderFactory rowEncoderFactory; private final ExternalTableFormatConfigFactory externalTableFormatConfigFactory; - private final Map externalTableCreateQueryFactoriesMap; - private final Map insertDataQueryFactoryMap; + private final CreateExternalTableQueryFactory externalTableCreateQueryFactory; + private final InsertDataQueryFactory insertDataQueryFactory; @Inject public GpfdistPageSinkProvider(@ForBaseJdbc JdbcClient client, AdbPluginConfig pluginConfig, - GpfdistMetadataFactory loadMetadataFactory, - WriteContextManager writeContextManager, + GpfdistLoadMetadataFactory loadMetadataFactory, + ContextManager contextManager, RowEncoderFactory rowEncoderFactory, ExternalTableFormatConfigFactory externalTableFormatConfigFactory, Set createExternalTableQueryFactories, Set insertDataQueryFactories) @@ -73,14 +74,22 @@ public GpfdistPageSinkProvider(@ForBaseJdbc JdbcClient client, this.client = (AdbSqlClient) client; this.pluginConfig = pluginConfig; this.loadMetadataFactory = loadMetadataFactory; - this.writeContextManager = writeContextManager; + this.contextManager = contextManager; this.rowEncoderFactory = rowEncoderFactory; this.externalTableFormatConfigFactory = externalTableFormatConfigFactory; this.loadQueryThreadExecutor = ExecutorServiceProvider.LOAD_DATA_QUERY_EXECUTOR_SERVICE; - this.externalTableCreateQueryFactoriesMap = createExternalTableQueryFactories.stream() + Map externalTableQueryFactoryMap = createExternalTableQueryFactories.stream() .collect(Collectors.toMap(CreateExternalTableQueryFactory::getExternalTableType, Function.identity())); - this.insertDataQueryFactoryMap = insertDataQueryFactories.stream() + externalTableCreateQueryFactory = externalTableQueryFactoryMap.get(EXTERNAL_TABLE_TYPE); + checkArgument(externalTableCreateQueryFactory != null, + "failed to get writable table query factory by externalTableType %s", + EXTERNAL_TABLE_TYPE); + Map insertDataFactoryMap = insertDataQueryFactories.stream() .collect(Collectors.toMap(InsertDataQueryFactory::getExternalTableType, Function.identity())); + insertDataQueryFactory = insertDataFactoryMap.get(EXTERNAL_TABLE_TYPE); + checkArgument(insertDataQueryFactory != null, + "failed to get insert data query factory by externalTableType %s", + EXTERNAL_TABLE_TYPE); } @Override @@ -106,25 +115,22 @@ private ConnectorPageSink createPageSinkInternal(ConnectorTransactionHandle tran ConnectorOutputTableHandle outputTableHandle, ConnectorPageSinkId pageSinkId) { - ExternalTableType externalTableType = ExternalTableType.READABLE; - GpfdistLoadMetadata loadMetadata = (GpfdistLoadMetadata) loadMetadataFactory.create(session, + GpfdistLoadMetadata loadMetadata = loadMetadataFactory.create(session, (JdbcOutputTableHandle) outputTableHandle, pageSinkId, externalTableFormatConfigFactory.create()); - ContextId contextId = new ContextId(loadMetadata.getSourceTableName()); - WriteContext writeContext = new WriteContext(contextId, + WriteContext writeContext = new WriteContext( loadMetadata, rowEncoderFactory.create(session, loadMetadata.getDataTypes()), pluginConfig.getWriteBufferSize()); - DataTransferQueryExecutor loadDataExecutor = new GpfdistDataTransferQueryExecutor(client, + DataTransferQueryExecutor loadDataExecutor = new GpfdistLoadDataTransferQueryExecutor(client, session, - loadMetadata, - writeContext, loadQueryThreadExecutor, - externalTableCreateQueryFactoriesMap.get(externalTableType), - insertDataQueryFactoryMap.get(externalTableType)); - writeContextManager.add(writeContext); + loadMetadata, + externalTableCreateQueryFactory, + insertDataQueryFactory); + contextManager.add(writeContext); CompletableFuture queryLoadFuture = loadDataExecutor.execute(); - return new GpfdistPageSink(writeContextManager, writeContext, queryLoadFuture); + return new GpfdistPageSink(contextManager, writeContext, queryLoadFuture); } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistDataTransferQueryExecutor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistDataTransferQueryExecutor.java deleted file mode 100644 index 94218e51dfae..000000000000 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/processing/GpfdistDataTransferQueryExecutor.java +++ /dev/null @@ -1,113 +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.adb.connector.protocol.gpfdist.load.processing; - -import io.airlift.log.Logger; -import io.trino.plugin.adb.connector.AdbSqlClient; -import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.DataTransferQueryExecutor; -import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; -import io.trino.spi.StandardErrorCode; -import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorSession; - -import java.sql.Connection; -import java.sql.SQLException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; - -public class GpfdistDataTransferQueryExecutor - implements DataTransferQueryExecutor -{ - private static final Logger log = Logger.get(GpfdistDataTransferQueryExecutor.class); - private final AdbSqlClient client; - private final ConnectorSession session; - private final ExecutorService executor; - private final GpfdistLoadMetadata loadMetadata; - private final WriteContext writeContext; - private final CreateExternalTableQueryFactory externalTableQueryFactory; - private final InsertDataQueryFactory insertDataQueryFactory; - private Connection connection; - - public GpfdistDataTransferQueryExecutor(AdbSqlClient client, - ConnectorSession session, - GpfdistLoadMetadata loadMetadata, - WriteContext writeContext, - ExecutorService executor, - CreateExternalTableQueryFactory externalTableQueryFactory, - InsertDataQueryFactory insertDataQueryFactory) - { - this.client = client; - this.session = session; - this.executor = executor; - this.loadMetadata = loadMetadata; - this.writeContext = writeContext; - this.externalTableQueryFactory = externalTableQueryFactory; - this.insertDataQueryFactory = insertDataQueryFactory; - } - - @Override - public CompletableFuture execute() - { - return CompletableFuture.supplyAsync(() -> { - try { - connection = client.getConnection(session); - connection.setAutoCommit(false); - connection.setReadOnly(false); - createReadableExternalTable(connection, loadMetadata); - insertIntoExternalTable(session, connection, loadMetadata); - connection.commit(); - connection.close(); - return null; - } - catch (Exception e) { - log.error("Failed to execute trino to adb load query", e); - TrinoException exception = new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, e.getMessage(), e); - writeContext.getAdbQueryException().set(exception); - throw exception; - } - finally { - if (connection != null) { - try { - connection.close(); - } - catch (SQLException e) { - log.warn("Failed to close connection", e); - } - } - } - }, executor); - } - - private void createReadableExternalTable(Connection connection, - GpfdistLoadMetadata loadMetadata) - throws SQLException - { - String sql = externalTableQueryFactory.createQuery(loadMetadata); - log.debug("Executing create readable external table query: " + sql); - client.execute(session, connection, sql); - } - - private void insertIntoExternalTable(ConnectorSession session, - Connection connection, - GpfdistLoadMetadata loadMetadata) - throws SQLException - { - String sql = insertDataQueryFactory.create(loadMetadata); - log.debug("Executing insert into target table from external table query: " + sql); - client.executeAsPreparedStatement(session, connection, sql); - } -} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/CreateWritableExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/CreateReadableExternalTableQueryFactory.java similarity index 64% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/CreateWritableExternalTableQueryFactory.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/CreateReadableExternalTableQueryFactory.java index e81805a9f6bb..0e636574ad43 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/CreateWritableExternalTableQueryFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/CreateReadableExternalTableQueryFactory.java @@ -11,24 +11,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.unload; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.query; -import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableType; +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; -public class CreateWritableExternalTableQueryFactory - implements CreateExternalTableQueryFactory +public class CreateReadableExternalTableQueryFactory + extends AbstractExternalTableQueryFactory { @Override public String createQuery(GpfdistMetadata metadata) { - throw new UnsupportedOperationException("Feature not yet implemented"); + return createCommonQuery(metadata); } @Override public ExternalTableType getExternalTableType() { - return ExternalTableType.WRITABLE; + return ExternalTableType.READABLE; } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/GpfdistLoadDataTransferQueryExecutor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/GpfdistLoadDataTransferQueryExecutor.java new file mode 100644 index 000000000000..ff01b1432980 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/GpfdistLoadDataTransferQueryExecutor.java @@ -0,0 +1,67 @@ +/* + * 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.adb.connector.protocol.gpfdist.load.query; + +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractDataTransferQueryExecutor; +import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; +import io.trino.plugin.jdbc.PreparedQuery; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.SQLException; +import java.util.concurrent.ExecutorService; + +public class GpfdistLoadDataTransferQueryExecutor + extends AbstractDataTransferQueryExecutor +{ + private static final Logger log = Logger.get(GpfdistLoadDataTransferQueryExecutor.class); + private final GpfdistLoadMetadata loadMetadata; + + public GpfdistLoadDataTransferQueryExecutor(AdbSqlClient client, + ConnectorSession session, + ExecutorService executor, GpfdistLoadMetadata loadMetadata, + CreateExternalTableQueryFactory externalTableQueryFactory, + InsertDataQueryFactory insertDataQueryFactory) + { + super(client, session, executor, externalTableQueryFactory, insertDataQueryFactory); + this.loadMetadata = loadMetadata; + } + + @Override + protected void executeQueries() + throws SQLException + { + createReadableExternalTable(); + insertIntoExternalTable(); + } + + private void createReadableExternalTable() + throws SQLException + { + String sql = externalTableQueryFactory.createQuery(loadMetadata); + client.execute(session, connection, sql); + log.info("Executed create readable external table query: %s", sql); + } + + private void insertIntoExternalTable() + throws SQLException + { + PreparedQuery query = insertDataQueryFactory.create(session, connection, loadMetadata); + client.executeAsPreparedStatement(session, connection, query); + log.info("Executed insert into target table from external table query: %s", query.query()); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/InsertDataFromExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/InsertDataFromExternalTableQueryFactory.java similarity index 75% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/InsertDataFromExternalTableQueryFactory.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/InsertDataFromExternalTableQueryFactory.java index bac0f2f74085..a25fbe9ddd0f 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/InsertDataFromExternalTableQueryFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/load/query/InsertDataFromExternalTableQueryFactory.java @@ -11,22 +11,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.load; +package io.trino.plugin.adb.connector.protocol.gpfdist.load.query; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistLoadMetadata; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; +import io.trino.plugin.jdbc.PreparedQuery; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.Connection; +import java.util.List; import static java.lang.String.format; public class InsertDataFromExternalTableQueryFactory implements InsertDataQueryFactory { - public static final String COLUMN_DELIMITER = ", "; + private static final String COLUMN_DELIMITER = ", "; @Override - public String create(GpfdistMetadata metadata) + public PreparedQuery create(ConnectorSession session, Connection connection, GpfdistMetadata metadata) { GpfdistLoadMetadata loadMetadata = (GpfdistLoadMetadata) metadata; String delimiter = COLUMN_DELIMITER; @@ -39,11 +44,11 @@ public String create(GpfdistMetadata metadata) else { targetTableColumnNames = externalTableColumnNames; } - return format("INSERT INTO %s (%s) SELECT %s FROM %s", + return new PreparedQuery(format("INSERT INTO %s (%s) SELECT %s FROM %s", loadMetadata.getTargetTable(), targetTableColumnNames, externalTableColumnNames, - loadMetadata.getSourceTableName()); + loadMetadata.getSourceTable()), List.of()); } @Override diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableFormatConfig.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfig.java similarity index 78% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableFormatConfig.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfig.java index 6d0eaa4f3b54..9caa4c80078c 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableFormatConfig.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfig.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist; +package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; -import io.trino.plugin.adb.connector.encode.EncoderFormat; +import io.trino.plugin.adb.connector.encode.DataFormat; -public record ExternalTableFormatConfig(char delimiter, String encoding, String nullValue, EncoderFormat encoderFormat) +public record ExternalTableFormatConfig(char delimiter, String encoding, String nullValue, DataFormat dataFormat) { } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactory.java index 8ab4cb47c785..c1297bec906f 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactory.java @@ -13,8 +13,6 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableFormatConfig; - public interface ExternalTableFormatConfigFactory { ExternalTableFormatConfig create(); diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactoryImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactoryImpl.java index 06420d3fd038..8ddbe7a8a2e2 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactoryImpl.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableFormatConfigFactoryImpl.java @@ -14,34 +14,33 @@ package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; import com.google.inject.Inject; -import io.trino.plugin.adb.connector.encode.EncoderConfig; -import io.trino.plugin.adb.connector.encode.EncoderFormat; -import io.trino.plugin.adb.connector.encode.csv.CsvEncoderConfig; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableFormatConfig; +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.encode.DataFormatConfig; +import io.trino.plugin.adb.connector.encode.csv.CsvFormatConfig; public class ExternalTableFormatConfigFactoryImpl implements ExternalTableFormatConfigFactory { - private final EncoderConfig encoderConfig; + private final DataFormatConfig dataFormatConfig; @Inject - public ExternalTableFormatConfigFactoryImpl(EncoderConfig encoderConfig) + public ExternalTableFormatConfigFactoryImpl(DataFormatConfig dataFormatConfig) { - this.encoderConfig = encoderConfig; + this.dataFormatConfig = dataFormatConfig; } @Override public ExternalTableFormatConfig create() { - if (encoderConfig.getEncoderFormat() == EncoderFormat.CSV) { - CsvEncoderConfig csvEncoderConfig = (CsvEncoderConfig) encoderConfig; - return new ExternalTableFormatConfig(csvEncoderConfig.getDelimiter(), - csvEncoderConfig.getEncoding(), - csvEncoderConfig.getNullValue(), - encoderConfig.getEncoderFormat()); + if (dataFormatConfig.getDataFormat() == DataFormat.CSV) { + CsvFormatConfig csvFormatConfig = (CsvFormatConfig) dataFormatConfig; + return new ExternalTableFormatConfig(csvFormatConfig.getDelimiter(), + csvFormatConfig.getEncoding(), + csvFormatConfig.getNullValue(), + dataFormatConfig.getDataFormat()); } else { - throw new IllegalArgumentException("Unsupported encoder format: " + encoderConfig.getEncoderFormat()); + throw new IllegalArgumentException("Unsupported encoder format: " + dataFormatConfig.getDataFormat()); } } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableType.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableType.java similarity index 90% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableType.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableType.java index cb91eccad493..fd9b575ee19b 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/ExternalTableType.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableType.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist; +package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; public enum ExternalTableType { diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableUtil.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableUtil.java new file mode 100644 index 000000000000..a7b8ba63055b --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/ExternalTableUtil.java @@ -0,0 +1,29 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +import java.util.UUID; + +public final class ExternalTableUtil +{ + private ExternalTableUtil() + { + } + + public static String createExternalTableName(ExternalTableType tableType) + { + return String.format("trino_external_%s_%s", tableType.name().toLowerCase(), + UUID.randomUUID().toString().replace("-", "")); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadata.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadata.java index 0b69e4f42798..e27d77e4ef0a 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadata.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadata.java @@ -14,7 +14,6 @@ package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; import io.trino.plugin.adb.connector.datatype.ColumnDataType; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableFormatConfig; import io.trino.spi.connector.ConnectorPageSinkId; import java.util.List; @@ -23,13 +22,12 @@ public class GpfdistLoadMetadata extends GpfdistMetadata { + private final String targetTable; private final ConnectorPageSinkId pageSinkId; private final Optional pageSinkIdColumn; - private final ExternalTableFormatConfig externalTableFormatConfig; - private final String gpfdistLocation; public GpfdistLoadMetadata(String targetTable, - String sourceTableName, + String sourceTable, List columnNames, List dataTypes, ConnectorPageSinkId pageSinkId, @@ -37,11 +35,10 @@ public GpfdistLoadMetadata(String targetTable, ExternalTableFormatConfig externalTableFormatConfig, String gpfdistLocation) { - super(targetTable, sourceTableName, columnNames, dataTypes); + super(sourceTable, columnNames, dataTypes, externalTableFormatConfig, gpfdistLocation); this.pageSinkId = pageSinkId; this.pageSinkIdColumn = pageSinkIdColumn; - this.externalTableFormatConfig = externalTableFormatConfig; - this.gpfdistLocation = gpfdistLocation; + this.targetTable = targetTable; } public ConnectorPageSinkId getPageSinkId() @@ -54,13 +51,8 @@ public Optional getPageSinkIdColumn() return pageSinkIdColumn; } - public ExternalTableFormatConfig getExternalTableFormatConfig() + public String getTargetTable() { - return externalTableFormatConfig; - } - - public String getGpfdistLocation() - { - return gpfdistLocation; + return targetTable; } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactory.java index 3ebfcee23a33..fdbb979626fa 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactory.java @@ -13,87 +13,14 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; -import com.google.inject.Inject; -import io.airlift.http.server.HttpServerInfo; -import io.airlift.log.Logger; -import io.trino.plugin.adb.connector.AdbSqlClient; -import io.trino.plugin.adb.connector.datatype.ColumnDataType; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableFormatConfig; -import io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistServerConfig; -import io.trino.plugin.jdbc.ForBaseJdbc; -import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcOutputTableHandle; -import io.trino.spi.StandardErrorCode; -import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorPageSinkId; import io.trino.spi.connector.ConnectorSession; -import java.net.URI; -import java.util.List; -import java.util.Optional; -import java.util.UUID; - -public class GpfdistLoadMetadataFactory - implements GpfdistMetadataFactory +public interface GpfdistLoadMetadataFactory { - private static final Logger log = Logger.get(GpfdistLoadMetadataFactory.class); - private final GpfdistServerConfig config; - private final AdbSqlClient sqlClient; - private final HttpServerInfo httpServerInfo; - - @Inject - public GpfdistLoadMetadataFactory(GpfdistServerConfig config, - @ForBaseJdbc JdbcClient sqlClient, - HttpServerInfo httpServerInfo) - { - this.config = config; - this.sqlClient = (AdbSqlClient) sqlClient; - this.httpServerInfo = httpServerInfo; - } - - @Override - public GpfdistMetadata create(ConnectorSession session, + GpfdistLoadMetadata create(ConnectorSession session, JdbcOutputTableHandle tableHandle, ConnectorPageSinkId pageSinkId, - ExternalTableFormatConfig externalTableFormatConfig) - { - try { - String targetTableName = sqlClient.getTargetTableName(tableHandle); - String externalTableName = createExternalTableName(); - List columnNames = tableHandle.getColumnNames().stream() - .map(sqlClient::quoted) - .toList(); - List dataTypes = sqlClient.getColumnDataTypes(session, tableHandle); - Optional pageSinkIdColumn = tableHandle.getPageSinkIdColumnName() - .map(sqlClient::quoted); - String gpfdistLocation = createGpfdistLocation(externalTableName); - return new GpfdistLoadMetadata( - targetTableName, - externalTableName, - columnNames, - dataTypes, - pageSinkId, - pageSinkIdColumn, - externalTableFormatConfig, - gpfdistLocation); - } - catch (Exception e) { - log.error("Failed to create gpfdistLoadMetadata", e); - throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, e); - } - } - - private String createGpfdistLocation(String externalTableName) - { - String protocol = config.isServerSslEnabled() ? "gpfdists" : "gpfdist"; - URI uri = config.isServerSslEnabled() ? httpServerInfo.getHttpsExternalUri() : httpServerInfo.getHttpExternalUri(); - String host = uri.getHost(); - int port = config.getServerExternalPort() > 0 ? config.getServerExternalPort() : uri.getPort(); - return String.format("%s://%s:%d/gpfdist/read/%s", protocol, host, port, externalTableName); - } - - private String createExternalTableName() - { - return String.format("trino_external_readable_%s", UUID.randomUUID().toString().replace("-", "")); - } + ExternalTableFormatConfig externalTableFormatConfig); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactoryImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactoryImpl.java new file mode 100644 index 000000000000..1a41e6d7dc29 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLoadMetadataFactoryImpl.java @@ -0,0 +1,80 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.spi.connector.ConnectorPageSinkId; +import io.trino.spi.connector.ConnectorSession; + +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableUtil.createExternalTableName; + +public class GpfdistLoadMetadataFactoryImpl + implements GpfdistLoadMetadataFactory +{ + private static final Logger log = Logger.get(GpfdistLoadMetadataFactoryImpl.class); + private final GpfdistLocationFactory gpfdistLocationFactory; + private final AdbSqlClient sqlClient; + + @Inject + public GpfdistLoadMetadataFactoryImpl( + @ForBaseJdbc JdbcClient sqlClient, + GpfdistLocationFactory gpfdistLocationFactory) + { + this.gpfdistLocationFactory = gpfdistLocationFactory; + this.sqlClient = (AdbSqlClient) sqlClient; + } + + @Override + public GpfdistLoadMetadata create(ConnectorSession session, + JdbcOutputTableHandle tableHandle, + ConnectorPageSinkId pageSinkId, + ExternalTableFormatConfig externalTableFormatConfig) + { + try { + ExternalTableType externalTableType = ExternalTableType.READABLE; + String targetTableName = sqlClient.getTargetTableName(tableHandle); + String externalTableName = createExternalTableName(externalTableType); + List columnNames = tableHandle.getColumnNames().stream() + .map(sqlClient::quoted) + .toList(); + List dataTypes = sqlClient.getColumnDataTypes(session, tableHandle); + Optional pageSinkIdColumn = tableHandle.getPageSinkIdColumnName() + .map(sqlClient::quoted); + String gpfdistLocation = gpfdistLocationFactory.create(externalTableName, externalTableType); + return new GpfdistLoadMetadata( + targetTableName, + externalTableName, + columnNames, + dataTypes, + pageSinkId, + pageSinkIdColumn, + externalTableFormatConfig, + gpfdistLocation); + } + catch (Exception e) { + String errMsg = "Failed to create gpfdistLoadMetadata: %s" + e.getMessage(); + log.error(errMsg, e); + throw new RuntimeException(errMsg, e); + } + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactory.java new file mode 100644 index 000000000000..35970631b9af --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactory.java @@ -0,0 +1,19 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +public interface GpfdistLocationFactory +{ + String create(String externalTableName, ExternalTableType externalTableType); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactoryImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactoryImpl.java new file mode 100644 index 000000000000..6330da691c57 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistLocationFactoryImpl.java @@ -0,0 +1,62 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +import com.google.inject.Inject; +import io.airlift.http.server.HttpServerInfo; +import io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistServerConfig; + +import java.net.URI; + +public class GpfdistLocationFactoryImpl + implements GpfdistLocationFactory +{ + private final GpfdistServerConfig config; + private final HttpServerInfo httpServerInfo; + + @Inject + public GpfdistLocationFactoryImpl(GpfdistServerConfig config, HttpServerInfo httpServerInfo) + { + this.config = config; + this.httpServerInfo = httpServerInfo; + } + + @Override + public String create(String externalTableName, ExternalTableType externalTableType) + { + String protocol = config.isServerSslEnabled() ? "gpfdists" : "gpfdist"; + URI uri = config.isServerSslEnabled() ? httpServerInfo.getHttpsExternalUri() : httpServerInfo.getHttpExternalUri(); + String host = uri.getHost(); + int port = config.getServerExternalPort() > 0 ? config.getServerExternalPort() : uri.getPort(); + return String.format("%s://%s:%d/gpfdist/%s/%s", + protocol, + host, + port, + getOperationPath(externalTableType), + externalTableName); + } + + private String getOperationPath(ExternalTableType externalTableType) + { + if (externalTableType == ExternalTableType.READABLE) { + return "read"; + } + else if (externalTableType == ExternalTableType.WRITABLE) { + return "write"; + } + else { + throw new IllegalArgumentException("Unsupported external table type: " + externalTableType); + } + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadata.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadata.java index afe0c6dacf1c..fd6463e3d77e 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadata.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadata.java @@ -20,30 +20,29 @@ public class GpfdistMetadata { - private final String targetTable; - private final String sourceTableName; + private final String sourceTable; private final List columnNames; private final List dataTypes; + private final ExternalTableFormatConfig externalTableFormatConfig; + private final String gpfdistLocation; - public GpfdistMetadata(String targetTable, - String sourceTableName, + public GpfdistMetadata( + String sourceTable, List columnNames, - List dataTypes) + List dataTypes, + ExternalTableFormatConfig externalTableFormatConfig, + String gpfdistLocation) { - this.targetTable = targetTable; - this.sourceTableName = sourceTableName; + this.sourceTable = sourceTable; this.columnNames = columnNames; this.dataTypes = dataTypes; + this.externalTableFormatConfig = externalTableFormatConfig; + this.gpfdistLocation = gpfdistLocation; } - public String getTargetTable() + public String getSourceTable() { - return targetTable; - } - - public String getSourceTableName() - { - return sourceTableName; + return sourceTable; } public List getColumnNames() @@ -56,6 +55,16 @@ public List getDataTypes() return dataTypes; } + public ExternalTableFormatConfig getExternalTableFormatConfig() + { + return externalTableFormatConfig; + } + + public String getGpfdistLocation() + { + return gpfdistLocation; + } + @Override public boolean equals(Object o) { @@ -66,12 +75,12 @@ public boolean equals(Object o) return false; } GpfdistMetadata that = (GpfdistMetadata) o; - return Objects.equals(targetTable, that.targetTable) && Objects.equals(sourceTableName, that.sourceTableName) && Objects.equals(columnNames, that.columnNames) && Objects.equals(dataTypes, that.dataTypes); + return Objects.equals(sourceTable, that.sourceTable) && Objects.equals(columnNames, that.columnNames) && Objects.equals(dataTypes, that.dataTypes) && Objects.equals(externalTableFormatConfig, that.externalTableFormatConfig) && Objects.equals(gpfdistLocation, that.gpfdistLocation); } @Override public int hashCode() { - return Objects.hash(targetTable, sourceTableName, columnNames, dataTypes); + return Objects.hash(sourceTable, columnNames, dataTypes, externalTableFormatConfig, gpfdistLocation); } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadata.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadata.java new file mode 100644 index 000000000000..02f628dda698 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadata.java @@ -0,0 +1,59 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.jdbc.JdbcSplit; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ColumnHandle; + +import java.util.List; + +public class GpfdistUnloadMetadata + extends GpfdistMetadata +{ + private final JdbcTableHandle targetTableHandle; + private final List columnHandles; + private final JdbcSplit split; + + public GpfdistUnloadMetadata(JdbcTableHandle targetTableHandle, + String sourceTable, + List columnNames, + List dataTypes, + List columnHandles, + JdbcSplit split, + ExternalTableFormatConfig externalTableFormatConfig, + String gpfdistLocation) + { + super(sourceTable, columnNames, dataTypes, externalTableFormatConfig, gpfdistLocation); + this.targetTableHandle = targetTableHandle; + this.columnHandles = columnHandles; + this.split = split; + } + + public JdbcTableHandle getTargetTableHandle() + { + return targetTableHandle; + } + + public JdbcSplit getSplit() + { + return split; + } + + public List getColumnHandles() + { + return columnHandles; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadataFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactory.java similarity index 60% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadataFactory.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactory.java index de1b5eb85f16..9f3ab15d5ce0 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistMetadataFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactory.java @@ -13,14 +13,18 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.metadata; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableFormatConfig; -import io.trino.plugin.jdbc.JdbcOutputTableHandle; -import io.trino.spi.connector.ConnectorPageSinkId; +import io.trino.plugin.jdbc.JdbcSplit; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; -public interface GpfdistMetadataFactory +import java.util.List; + +public interface GpfdistUnloadMetadataFactory { - GpfdistMetadata create(ConnectorSession session, - JdbcOutputTableHandle tableHandle, - ConnectorPageSinkId pageSinkId, ExternalTableFormatConfig externalTableFormatConfig); + GpfdistUnloadMetadata create(ConnectorSession session, + JdbcTableHandle tableHandle, + JdbcSplit split, + List columnHandles, + ExternalTableFormatConfig externalTableFormatConfig); } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactoryImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactoryImpl.java new file mode 100644 index 000000000000..386ea9460d8d --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/metadata/GpfdistUnloadMetadataFactoryImpl.java @@ -0,0 +1,94 @@ +/* + * 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.adb.connector.protocol.gpfdist.metadata; + +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.datatype.ColumnDataType; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcSplit; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorSession; + +import java.util.ArrayList; +import java.util.List; + +import static io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableUtil.createExternalTableName; + +public class GpfdistUnloadMetadataFactoryImpl + implements GpfdistUnloadMetadataFactory +{ + private static final Logger log = Logger.get(GpfdistUnloadMetadataFactoryImpl.class); + private static final ExternalTableType EXTERNAL_TABLE_TYPE = ExternalTableType.WRITABLE; + private final GpfdistLocationFactory gpfdistLocationFactory; + private final AdbSqlClient sqlClient; + + @Inject + public GpfdistUnloadMetadataFactoryImpl(@ForBaseJdbc JdbcClient sqlClient, + GpfdistLocationFactory gpfdistLocationFactory) + { + this.gpfdistLocationFactory = gpfdistLocationFactory; + this.sqlClient = (AdbSqlClient) sqlClient; + } + + @Override + public GpfdistUnloadMetadata create(ConnectorSession session, + JdbcTableHandle tableHandle, + JdbcSplit split, + List columnHandles, + ExternalTableFormatConfig externalTableFormatConfig) + { + try { + List columnNames = new ArrayList<>(); + List dataTypes = new ArrayList<>(); + String externalTableName = createExternalTableName(EXTERNAL_TABLE_TYPE); + JdbcTableHandle jdbcTableHandle = tableHandle.intersectedWithConstraint(split.getDynamicFilter() + .transformKeys(ColumnHandle.class::cast)); + initColumnsMetadata(session, columnHandles, columnNames, dataTypes); + String gpfdistLocation = gpfdistLocationFactory.create(externalTableName, EXTERNAL_TABLE_TYPE); + return new GpfdistUnloadMetadata( + jdbcTableHandle, + externalTableName, + columnNames, + dataTypes, + columnHandles, + split, + externalTableFormatConfig, + gpfdistLocation); + } + catch (Exception e) { + String errMsg = "Failed to create gpfdistUnloadMetadata: " + e.getMessage(); + log.error(errMsg, e); + throw new RuntimeException(errMsg, e); + } + } + + private void initColumnsMetadata(ConnectorSession session, + List columnHandles, + List columnNames, + List dataTypes) + { + List jdbcColumnHandles = columnHandles.stream() + .map(columnHandle -> ((JdbcColumnHandle) columnHandle)) + .toList(); + jdbcColumnHandles.forEach(columnHandle -> { + columnNames.add(columnHandle.getColumnName()); + dataTypes.add(sqlClient.getColumnDataType(session, columnHandle.getJdbcTypeHandle())); + }); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistResource.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistResource.java index ae42209c0355..846837971096 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistResource.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistResource.java @@ -16,13 +16,22 @@ import com.google.inject.Inject; import io.airlift.log.Logger; import io.trino.plugin.adb.AdbPluginConfig; +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; import io.trino.plugin.adb.connector.protocol.gpfdist.ExecutorServiceProvider; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.GpfdistPacketBuilder; import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContext; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.context.WriteContextManager; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.processing.GpfdistPageProcessor; -import io.trino.plugin.adb.connector.protocol.gpfdist.load.processing.GpfdistPageSerializer; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.process.GpfdistPageProcessor; +import io.trino.plugin.adb.connector.protocol.gpfdist.load.process.GpfdistPageSerializer; import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; +import io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistReadableRequest; +import io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistWritableRequest; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessorFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContext; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.process.GpfdistSegmentRequestProcessor; +import jakarta.ws.rs.Consumes; import jakarta.ws.rs.GET; +import jakarta.ws.rs.POST; import jakarta.ws.rs.Path; import jakarta.ws.rs.PathParam; import jakarta.ws.rs.Produces; @@ -33,29 +42,39 @@ import jakarta.ws.rs.core.MediaType; import jakarta.ws.rs.core.Response; +import java.io.InputStream; import java.io.PipedInputStream; import java.io.PipedOutputStream; import java.util.Optional; import java.util.concurrent.ExecutorService; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_PROTO; +import static io.trino.plugin.adb.connector.protocol.gpfdist.GpfdistUtil.createGpfdistFileName; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_PROTO; import static java.lang.String.format; @Path("/gpfdist") public class GpfdistResource { private static final Logger log = Logger.get(GpfdistResource.class); - private static final int GPFDIST_PROTOCOL_VERSION = 1; - private final WriteContextManager writeContextManager; + private static final int GPFDIST_FOR_READ_PROTOCOL_VERSION = 1; + private static final int GPFDIST_FOR_WRITE_PROTOCOL_VERSION = 0; + private final ContextManager writeContextManager; + private final ContextManager readContextManager; private final ExecutorService executorService; private final AdbPluginConfig pluginConfig; + private final InputDataProcessorFactory inputDataProcessorFactory; @Inject - public GpfdistResource(WriteContextManager writeContextManager, AdbPluginConfig pluginConfig) + public GpfdistResource(ContextManager writeContextManager, + ContextManager readContextManager, + AdbPluginConfig pluginConfig, + InputDataProcessorFactory inputDataProcessorFactory) { this.writeContextManager = writeContextManager; + this.readContextManager = readContextManager; this.pluginConfig = pluginConfig; + this.inputDataProcessorFactory = inputDataProcessorFactory; this.executorService = ExecutorServiceProvider.GPFDIST_HTTP_REQUEST_EXECUTOR_SERVICE; } @@ -65,14 +84,21 @@ public GpfdistResource(WriteContextManager writeContextManager, AdbPluginConfig public void get(@PathParam("tableName") String tableName, @Context HttpHeaders headers, @Suspended AsyncResponse asyncResponse) { GpfdistReadableRequest request = GpfdistReadableRequest.create(tableName, headers.getRequestHeaders()); - checkArgument(request.getGpProtocol() == GPFDIST_PROTOCOL_VERSION, - format("Gpfdist protocol of version %d is only supported", GPFDIST_PROTOCOL_VERSION)); + checkArgument(request.getGpProtocol() == GPFDIST_FOR_READ_PROTOCOL_VERSION, + format("Gpfdist protocol of version %d is only supported", GPFDIST_FOR_READ_PROTOCOL_VERSION)); log.info("Input GET gpfdist request: %s", request); Optional writeContextOptional = writeContextManager.get(new ContextId(tableName)); if (writeContextOptional.isPresent()) { WriteContext writeContext = writeContextOptional.get(); executorService.submit(() -> processGetRequest(asyncResponse, request, writeContext)); } + else { + log.info("There is no data for loading responded by request: %s", request); + asyncResponse.resume(createOkGetResponseBuilder(request) + .entity(new GpfdistPacketBuilder(createGpfdistFileName(tableName)) + .createSingleEmptyDataPacket()) + .build()); + } } private void processGetRequest(AsyncResponse asyncResponse, GpfdistReadableRequest request, WriteContext writeContext) @@ -85,10 +111,7 @@ private void processGetRequest(AsyncResponse asyncResponse, GpfdistReadableReque writeContext, new GpfdistPageSerializer(writeContext.getMetadata().getDataTypes(), writeContext.getRowEncoder())); writeContext.getPageProcessors().add(gpfdistPageProcessor); - asyncResponse.resume(Response.ok() - .header(HttpHeaders.CONTENT_TYPE, MediaType.TEXT_PLAIN) - .header(HttpHeaders.CACHE_CONTROL, "no-cache") - .header(X_GP_PROTO, request.getGpProtocol()) + asyncResponse.resume(createOkGetResponseBuilder(request) .entity(inputStream) .build()); log.info("Request %s completed successfully", request); @@ -104,4 +127,118 @@ private void processGetRequest(AsyncResponse asyncResponse, GpfdistReadableReque e.getMessage()); } } + + private Response.ResponseBuilder createOkGetResponseBuilder(GpfdistReadableRequest request) + { + return Response.ok() + .header(HttpHeaders.CONTENT_TYPE, MediaType.TEXT_PLAIN) + .header(HttpHeaders.CACHE_CONTROL, "no-cache") + .header(X_GP_PROTO, request.getGpProtocol()); + } + + @POST + @Consumes("*/*") + @Path("/write/{tableName}") + public void post(@PathParam("tableName") String tableName, InputStream data, @Context HttpHeaders headers, @Suspended AsyncResponse asyncResponse) + { + try { + GpfdistWritableRequest request = GpfdistWritableRequest.create(tableName, headers.getRequestHeaders()); + log.debug("Received POST request: %s", request); + checkArgument(request.getGpProtocol() == GPFDIST_FOR_WRITE_PROTOCOL_VERSION, + format("Gpfdist protocol version %s for write operation is supported", GPFDIST_FOR_WRITE_PROTOCOL_VERSION)); + Optional readContextOptional = readContextManager.get(new ContextId(tableName)); + if (readContextOptional.isEmpty()) { + processNotFoundQueryRequest(tableName, asyncResponse, request); + } + else { + ReadContext readContext = readContextOptional.get(); + if (initialRequest(request)) { + processInitialRequest(asyncResponse, readContext, request); + } + else { + if (!isLast(request)) { + processDataRequest(data, asyncResponse, readContext, request); + } + else { + processTearDownRequest(asyncResponse, readContext, request); + } + } + } + } + catch (Exception e) { + failWriteResponse(asyncResponse, e); + } + } + + private static void processNotFoundQueryRequest(String tableName, AsyncResponse asyncResponse, GpfdistWritableRequest request) + { + String errorMessage = "No active query for writeable table: " + tableName; + asyncResponse.resume(Response.status(Response.Status.BAD_REQUEST.getStatusCode(), errorMessage) + .header(X_GP_PROTO, request.getGpProtocol()) + .build()); + log.error("Failed to processed request: %s. " + errorMessage, request); + } + + private void processInitialRequest(AsyncResponse asyncResponse, ReadContext readContext, GpfdistWritableRequest request) + { + InputDataProcessor dataProcessor = inputDataProcessorFactory.create(readContext.getRowDecoder(), + readContext.getRowProcessor()); + readContext.getSegmentDataProcessors().putIfAbsent(request.getSegmentId(), + new GpfdistSegmentRequestProcessor(request.getSegmentId(), dataProcessor)); + asyncResponse.resume(Response.ok() + .header(X_GP_PROTO, request.getGpProtocol()) + .build()); + log.debug("Request for initial data transferring completed successfully: %s", request); + } + + private void processDataRequest(InputStream data, AsyncResponse asyncResponse, ReadContext readContext, GpfdistWritableRequest request) + { + executorService.submit(() -> { + try { + GpfdistSegmentRequestProcessor processor = getSegmentProcessor(readContext, request.getSegmentId()); + processor.process(data); + asyncResponse.resume(Response.ok() + .header(X_GP_PROTO, request.getGpProtocol()) + .build()); + log.debug("Processing request for transferring data completed successfully: %s", request); + } + catch (Exception e) { + failWriteResponse(asyncResponse, e); + } + }); + } + + private void processTearDownRequest(AsyncResponse asyncResponse, ReadContext readContext, GpfdistWritableRequest request) + { + GpfdistSegmentRequestProcessor processor = getSegmentProcessor(readContext, request.getSegmentId()); + processor.stop(); + asyncResponse.resume(Response.ok() + .header(X_GP_PROTO, request.getGpProtocol()) + .build()); + log.debug("Processing request for finishing data transferring completed successfully: %s", request); + } + + private GpfdistSegmentRequestProcessor getSegmentProcessor(ReadContext readContext, Integer segmentId) + { + return Optional.ofNullable(readContext.getSegmentDataProcessors().get(segmentId)) + .orElseThrow(() -> new IllegalStateException("Failed to get segment request processor by segmentId: " + segmentId)); + } + + private void failWriteResponse(AsyncResponse asyncResponse, Exception e) + { + asyncResponse.resume(Response.serverError() + .header(X_GP_PROTO, GPFDIST_FOR_WRITE_PROTOCOL_VERSION) + .build()); + log.error("Failed to process request: %s", e); + } + + private boolean initialRequest(GpfdistWritableRequest request) + { + return request.getGpSequence() == 1; + } + + private boolean isLast(GpfdistWritableRequest request) + { + return request.isLastChunk().isPresent() && request.isLastChunk().get(); + } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistReadableRequest.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistReadableRequest.java similarity index 90% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistReadableRequest.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistReadableRequest.java index ff085361330d..bbc94dd0ad42 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistReadableRequest.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistReadableRequest.java @@ -11,29 +11,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.server; +package io.trino.plugin.adb.connector.protocol.gpfdist.server.request; import jakarta.ws.rs.core.MultivaluedMap; import java.util.Optional; import java.util.UUID; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_CID; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_CSV_OPT; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_DATABASE; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_LINE_DELIM_LENGTH; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_MASTER_HOST; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_MASTER_PORT; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_PROTO; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SEGMENT_COUNT; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SEGMENT_ID; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SEG_DATADIR; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SEG_PG_CONF; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SESSION_ID; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_SN; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_XID; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_X_GP_USER; -import static io.trino.plugin.adb.connector.protocol.gpfdist.server.GpfdistRequestHeader.X_GP_X_SEG_PORT; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_CID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_CSV_OPT; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_DATABASE; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_LINE_DELIM_LENGTH; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_MASTER_HOST; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_MASTER_PORT; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_PROTO; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEGMENT_COUNT; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEGMENT_ID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEG_DATADIR; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEG_PG_CONF; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SESSION_ID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SN; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_XID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_X_GP_USER; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_X_SEG_PORT; public class GpfdistReadableRequest { diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistRequestHeader.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistRequestHeader.java similarity index 86% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistRequestHeader.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistRequestHeader.java index 41e5e46778f8..b7530cbf93e5 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/GpfdistRequestHeader.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistRequestHeader.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.adb.connector.protocol.gpfdist.server; +package io.trino.plugin.adb.connector.protocol.gpfdist.server.request; public final class GpfdistRequestHeader { @@ -22,6 +22,9 @@ public final class GpfdistRequestHeader public static final String X_GP_SEGMENT_COUNT = "X-GP-SEGMENT-COUNT"; public static final String X_GP_LINE_DELIM_LENGTH = "X-GP-LINE-DELIM-LENGTH"; public static final String X_GP_PROTO = "X-GP-PROTO"; + public static final String X_GP_PROTOCOL_VERSION = "X-GPFDIST-VERSION"; + public static final String X_GP_SEQUENCE = "X-GP-SEQ"; + public static final String X_GP_DONE = "X-GP-DONE"; public static final String X_GP_MASTER_HOST = "X-GP-MASTER_HOST"; public static final String X_GP_MASTER_PORT = "X-GP-MASTER_PORT"; public static final String X_GP_CSV_OPT = "X-GP-CSVOPT"; diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistWritableRequest.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistWritableRequest.java new file mode 100644 index 000000000000..72bef9a2f0fa --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/server/request/GpfdistWritableRequest.java @@ -0,0 +1,181 @@ +/* + * 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.adb.connector.protocol.gpfdist.server.request; + +import jakarta.ws.rs.core.MultivaluedMap; + +import java.util.Optional; +import java.util.UUID; + +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_CID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_DATABASE; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_DONE; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_LINE_DELIM_LENGTH; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_PROTO; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_PROTOCOL_VERSION; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEGMENT_COUNT; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEGMENT_ID; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SEQUENCE; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_SN; +import static io.trino.plugin.adb.connector.protocol.gpfdist.server.request.GpfdistRequestHeader.X_GP_XID; + +public class GpfdistWritableRequest +{ + private String requestId; + private String transactionId; + private String commandId; + private String scanId; + private Integer segmentId; + private Optional segmentsCount; + private Optional lineDelimiterLength; + private short gpProtocol; + private Optional gpProtocolVersion; + private int gpSequence; + private Optional isLastChunk; + private Optional gpDatabase; + + public GpfdistWritableRequest( + String requestId, + String transactionId, + String commandId, + String scanId, + Integer segmentId, + Optional segmentsCount, + Optional lineDelimiterLength, + short gpProtocol, + Optional gpProtocolVersion, + int gpSequence, + Optional isLastChunk, + Optional gpDatabase) + { + this.requestId = requestId; + this.transactionId = transactionId; + this.commandId = commandId; + this.scanId = scanId; + this.segmentId = segmentId; + this.segmentsCount = segmentsCount; + this.lineDelimiterLength = lineDelimiterLength; + this.gpProtocol = gpProtocol; + this.gpProtocolVersion = gpProtocolVersion; + this.gpSequence = gpSequence; + this.isLastChunk = isLastChunk; + this.gpDatabase = gpDatabase; + } + + public static GpfdistWritableRequest create(String tableName, MultivaluedMap values) + { + return new GpfdistWritableRequest( + createRequestId(tableName), + values.getFirst(X_GP_XID), + values.getFirst(X_GP_CID), + values.getFirst(X_GP_SN), + Optional.ofNullable(values.get(X_GP_SEGMENT_ID)) + .map(v -> Integer.parseInt(v.getFirst())) + .orElseThrow(() -> new IllegalArgumentException("Request header not found: " + X_GP_SEGMENT_ID)), + Optional.ofNullable(values.get(X_GP_SEGMENT_COUNT)) + .map(v -> Integer.parseInt(v.getFirst())), + Optional.ofNullable(values.get(X_GP_LINE_DELIM_LENGTH)) + .map(v -> Integer.parseInt(v.getFirst())), + Short.parseShort(values.get(X_GP_PROTO).getFirst()), + Optional.ofNullable(values.getFirst(X_GP_PROTOCOL_VERSION)), + Integer.parseInt(values.get(X_GP_SEQUENCE).getFirst()), + Optional.ofNullable(values.get(X_GP_DONE)) + .map(v -> v.getFirst().equals("1")), + Optional.ofNullable(values.getFirst(X_GP_DATABASE))); + } + + private static String createRequestId(String tableName) + { + return tableName + "_" + UUID.randomUUID(); + } + + public String getRequestId() + { + return requestId; + } + + public String getTransactionId() + { + return transactionId; + } + + public String getCommandId() + { + return commandId; + } + + public String getScanId() + { + return scanId; + } + + public Integer getSegmentId() + { + return segmentId; + } + + public Optional getSegmentsCount() + { + return segmentsCount; + } + + public Optional getLineDelimiterLength() + { + return lineDelimiterLength; + } + + public short getGpProtocol() + { + return gpProtocol; + } + + public Optional getGpProtocolVersion() + { + return gpProtocolVersion; + } + + public int getGpSequence() + { + return gpSequence; + } + + public Optional isLastChunk() + { + return isLastChunk; + } + + public Optional getGpDatabase() + { + return gpDatabase; + } + + @Override + public String toString() + { + return "GpfdistWritableRequest{" + + "requestId='" + requestId + '\'' + + ", transactionId='" + transactionId + '\'' + + ", commandId='" + commandId + '\'' + + ", scanId='" + scanId + '\'' + + ", segmentId=" + segmentId + + ", segmentsCount=" + segmentsCount + + ", lineDelimiterLength=" + lineDelimiterLength + + ", gpProtocol=" + gpProtocol + + ", gpProtocolVersion=" + gpProtocolVersion + + ", gpSequence=" + gpSequence + + ", isLastChunk=" + isLastChunk + + ", gpDatabase=" + gpDatabase + + '}'; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/AdbRecordSetProvider.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/AdbRecordSetProvider.java deleted file mode 100644 index 25a3e16e999a..000000000000 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/AdbRecordSetProvider.java +++ /dev/null @@ -1,48 +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.adb.connector.protocol.gpfdist.unload; - -import com.google.inject.Inject; -import io.trino.plugin.jdbc.JdbcRecordSetProvider; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorRecordSetProvider; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorSplit; -import io.trino.spi.connector.ConnectorTableHandle; -import io.trino.spi.connector.ConnectorTransactionHandle; -import io.trino.spi.connector.RecordSet; - -import java.util.List; - -public class AdbRecordSetProvider - implements ConnectorRecordSetProvider -{ - private final JdbcRecordSetProvider regularProvider; - - @Inject - public AdbRecordSetProvider(JdbcRecordSetProvider regularProvider) - { - this.regularProvider = regularProvider; - } - - @Override public RecordSet getRecordSet(ConnectorTransactionHandle transaction, - ConnectorSession session, - ConnectorSplit split, - ConnectorTableHandle table, - List columns) - { - //todo implement getting records through writable external table - return regularProvider.getRecordSet(transaction, session, split, table, columns); - } -} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InsertDataToExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/GpfdistConnectorRow.java similarity index 55% rename from plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InsertDataToExternalTableQueryFactory.java rename to plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/GpfdistConnectorRow.java index 92ccfa9e60f8..6430ce72cd46 100644 --- a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InsertDataToExternalTableQueryFactory.java +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/GpfdistConnectorRow.java @@ -13,22 +13,21 @@ */ package io.trino.plugin.adb.connector.protocol.gpfdist.unload; -import io.trino.plugin.adb.connector.protocol.gpfdist.ExternalTableType; -import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; -import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; +import io.trino.plugin.adb.connector.decode.ColumnValue; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; -public class InsertDataToExternalTableQueryFactory - implements InsertDataQueryFactory +public record GpfdistConnectorRow(long estimatedSize, ColumnValue[] columnValues) + implements ConnectorRow { @Override - public String create(GpfdistMetadata metadata) + public ColumnValue[] getColumnValues() { - throw new UnsupportedOperationException("Feature is not implemented yet"); + return columnValues; } @Override - public ExternalTableType getExternalTableType() + public long getEstimatedSize() { - return ExternalTableType.WRITABLE; + return estimatedSize; } } diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessor.java new file mode 100644 index 000000000000..02ddfd8a8cb6 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessor.java @@ -0,0 +1,23 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +import java.io.InputStream; + +public interface InputDataProcessor +{ + void process(InputStream dataStream); + + ProcessingDataResult getProcessedDataResult(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessorFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessorFactory.java new file mode 100644 index 000000000000..b075ad036c99 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/InputDataProcessorFactory.java @@ -0,0 +1,21 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +import io.trino.plugin.adb.connector.decode.RowDecoder; + +public interface InputDataProcessorFactory +{ + InputDataProcessor create(RowDecoder rowDecoder, RowProcessingService rowProcessingService); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/ProcessingDataResult.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/ProcessingDataResult.java new file mode 100644 index 000000000000..f183d0ad4667 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/ProcessingDataResult.java @@ -0,0 +1,20 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +import java.util.concurrent.atomic.AtomicLong; + +public record ProcessingDataResult(AtomicLong rowCount, AtomicLong estimatedProcessedBytes) +{ +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/RowProcessingService.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/RowProcessingService.java new file mode 100644 index 000000000000..1c645b39a293 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/RowProcessingService.java @@ -0,0 +1,27 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; + +public interface RowProcessingService +{ + void put(ConnectorRow row); + + ConnectorRow take(); + + boolean isEmpty(); + + void clear(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/SegmentRequestStatus.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/SegmentRequestStatus.java new file mode 100644 index 000000000000..7c2f97252a5a --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/SegmentRequestStatus.java @@ -0,0 +1,22 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +public enum SegmentRequestStatus +{ + INITIALIZED, + PROCESSING, + ERROR, + FINISHED; +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/WithEstimatedSize.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/WithEstimatedSize.java new file mode 100644 index 000000000000..b16445265e4a --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/WithEstimatedSize.java @@ -0,0 +1,19 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload; + +public interface WithEstimatedSize +{ + long getEstimatedSize(); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContext.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContext.java new file mode 100644 index 000000000000..1b65c1c3eef5 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContext.java @@ -0,0 +1,104 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.context; + +import io.airlift.log.Logger; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.protocol.gpfdist.Context; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ContextId; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadata; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.RowProcessingService; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.process.GpfdistSegmentRequestProcessor; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +public class ReadContext + implements Context +{ + private static final Logger log = Logger.get(ReadContext.class); + private final ContextId id; + private final AtomicLong memoryUsage; + private final GpfdistUnloadMetadata metadata; + private final RowDecoder rowDecoder; + private final RowProcessingService rowProcessingService; + private final Map segmentDataProcessors = new ConcurrentHashMap<>(); + private final AtomicLong completedBytes = new AtomicLong(); + + public ReadContext(GpfdistUnloadMetadata metadata, + AtomicLong memoryUsage, + RowDecoder rowDecoder, + RowProcessingService rowProcessingService) + { + this(new ContextId(metadata.getSourceTable()), metadata, memoryUsage, rowDecoder, rowProcessingService); + } + + public ReadContext(ContextId id, + GpfdistUnloadMetadata metadata, + AtomicLong memoryUsage, + RowDecoder rowDecoder, + RowProcessingService rowProcessingService) + { + this.id = id; + this.memoryUsage = memoryUsage; + this.metadata = metadata; + this.rowDecoder = rowDecoder; + this.rowProcessingService = rowProcessingService; + } + + @Override + public ContextId getId() + { + return id; + } + + public GpfdistUnloadMetadata getMetadata() + { + return metadata; + } + + public Map getSegmentDataProcessors() + { + return segmentDataProcessors; + } + + public RowDecoder getRowDecoder() + { + return rowDecoder; + } + + public RowProcessingService getRowProcessor() + { + return rowProcessingService; + } + + public AtomicLong getCompletedBytes() + { + return completedBytes; + } + + public AtomicLong getMemoryUsage() + { + return memoryUsage; + } + + @Override + public void close() + { + segmentDataProcessors.clear(); + rowProcessingService.clear(); + log.debug("Closed read context %s", id); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContextManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContextManager.java new file mode 100644 index 000000000000..b483ed0cc504 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/context/ReadContextManager.java @@ -0,0 +1,21 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.context; + +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractContextManager; + +public class ReadContextManager + extends AbstractContextManager +{ +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistBufferedRowProcessingService.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistBufferedRowProcessingService.java new file mode 100644 index 000000000000..625633fbd6a8 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistBufferedRowProcessingService.java @@ -0,0 +1,104 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import io.trino.plugin.adb.AdbPluginConfig; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.RowProcessingService; + +import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class GpfdistBufferedRowProcessingService + implements RowProcessingService +{ + private final Queue rowsQueue = new LinkedList<>(); + private final AtomicLong usedBufferedRowsMemory; + private final Condition isFullCondition; + private final Lock lock; + private final long maxByteBufferSize; + private long currentBufferSize; + + public GpfdistBufferedRowProcessingService(AdbPluginConfig pluginConfig, AtomicLong usedBufferedRowsMemory) + { + this.usedBufferedRowsMemory = usedBufferedRowsMemory; + this.maxByteBufferSize = pluginConfig.getReadBufferSize().toBytes(); + lock = new ReentrantLock(); + isFullCondition = lock.newCondition(); + } + + @Override + public void put(ConnectorRow row) + { + lock.lock(); + try { + while (isFull()) { + try { + isFullCondition.await(); + } + catch (InterruptedException ex) { + throw new RuntimeException(ex); + } + } + rowsQueue.add(row); + currentBufferSize += row.getEstimatedSize(); + usedBufferedRowsMemory.addAndGet(row.getEstimatedSize()); + } + finally { + lock.unlock(); + } + } + + @Override + public ConnectorRow take() + { + ConnectorRow element; + lock.lock(); + try { + element = rowsQueue.poll(); + currentBufferSize -= element.getEstimatedSize(); + isFullCondition.signalAll(); + } + finally { + lock.unlock(); + } + return element; + } + + @Override + public boolean isEmpty() + { + return rowsQueue.isEmpty(); + } + + @Override + public void clear() + { + rowsQueue.clear(); + } + + private boolean isFull() + { + return currentBufferSize >= maxByteBufferSize; + } + + public AtomicLong getUsedBufferedRowsMemory() + { + return usedBufferedRowsMemory; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistCsvDataProcessor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistCsvDataProcessor.java new file mode 100644 index 000000000000..90bd163048c1 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistCsvDataProcessor.java @@ -0,0 +1,97 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import com.opencsv.CSVReader; +import com.opencsv.CSVReaderBuilder; +import com.opencsv.RFC4180ParserBuilder; +import com.opencsv.enums.CSVReaderNullFieldIndicator; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.encode.DataFormatConfig; +import io.trino.plugin.adb.connector.encode.csv.CsvFormatConfig; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.ProcessingDataResult; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.RowProcessingService; +import io.trino.spi.StandardErrorCode; +import io.trino.spi.TrinoException; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.Charset; +import java.util.concurrent.atomic.AtomicLong; + +public class GpfdistCsvDataProcessor + implements InputDataProcessor +{ + private final CsvFormatConfig dataFormatConfig; + private final RowDecoder rowDecoder; + private final RowProcessingService rowProcessingService; + private final ProcessingDataResult processingDataResult; + + public GpfdistCsvDataProcessor(DataFormatConfig dataFormatConfig, + RowDecoder rowDecoder, + RowProcessingService rowProcessingService) + { + this.dataFormatConfig = (CsvFormatConfig) dataFormatConfig; + this.rowDecoder = rowDecoder; + this.rowProcessingService = rowProcessingService; + processingDataResult = new ProcessingDataResult(new AtomicLong(0), new AtomicLong(0)); + } + + @Override + public void process(InputStream dataStream) + { + try { + try (InputStreamReader streamReader = new InputStreamReader(dataStream, Charset.forName(dataFormatConfig.getEncoding())); + BufferedReader bufferedStreamReader = new BufferedReader(streamReader)) { + CSVReader csvReader = new CSVReaderBuilder(bufferedStreamReader) + .withCSVParser(new RFC4180ParserBuilder() + .withSeparator(dataFormatConfig.getDelimiter()) + .withFieldAsNull(CSVReaderNullFieldIndicator.EMPTY_SEPARATORS) + .build()) + .build(); + String[] rawRow; + try { + while ((rawRow = csvReader.readNext()) != null) { + ConnectorRow row = rowDecoder.decode(rawRow); + rowProcessingService.put(row); + processingDataResult.rowCount().incrementAndGet(); + processingDataResult.estimatedProcessedBytes().addAndGet(row.getEstimatedSize()); + } + } + catch (Exception e) { + try { + csvReader.close(); + } + catch (IOException ex) { + e.addSuppressed(ex); + } + throw new RuntimeException(e); + } + } + } + catch (Throwable e) { + throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, "Failed to process input data: " + e.getMessage(), e); + } + } + + @Override + public ProcessingDataResult getProcessedDataResult() + { + return processingDataResult; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistInputDataProcessorFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistInputDataProcessorFactory.java new file mode 100644 index 000000000000..81ef36e59c0d --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistInputDataProcessorFactory.java @@ -0,0 +1,45 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import com.google.inject.Inject; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.encode.DataFormat; +import io.trino.plugin.adb.connector.encode.DataFormatConfig; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessorFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.RowProcessingService; + +public class GpfdistInputDataProcessorFactory + implements InputDataProcessorFactory +{ + private final DataFormatConfig dataFormatConfig; + + @Inject + public GpfdistInputDataProcessorFactory(DataFormatConfig dataFormatConfig) + { + this.dataFormatConfig = dataFormatConfig; + } + + @Override + public InputDataProcessor create(RowDecoder rowDecoder, RowProcessingService rowProcessingService) + { + if (dataFormatConfig.getDataFormat() == DataFormat.CSV) { + return new GpfdistCsvDataProcessor(dataFormatConfig, rowDecoder, rowProcessingService); + } + else { + throw new UnsupportedOperationException("Unsupported data format: " + dataFormatConfig.getDataFormat()); + } + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordCursor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordCursor.java new file mode 100644 index 000000000000..82913816372c --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordCursor.java @@ -0,0 +1,232 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.trino.plugin.adb.connector.protocol.gpfdist.ConnectorRow; +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; +import io.trino.plugin.adb.connector.protocol.gpfdist.DataTransferQueryExecutor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.RowProcessingService; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.SegmentRequestStatus; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContext; +import io.trino.spi.StandardErrorCode; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.RecordCursor; +import io.trino.spi.type.Type; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.atomic.AtomicLong; + +import static com.google.common.base.Preconditions.checkArgument; + +public class GpfdistRecordCursor + implements RecordCursor +{ + private static final Logger log = Logger.get(GpfdistRecordCursor.class); + private final AtomicLong readTimeNanos = new AtomicLong(); + private final ReadContext readContext; + private final ContextManager contextManager; + private final DataTransferQueryExecutor dataTransferQueryExecutor; + private final List columnTypes; + private final RowProcessingService rowProcessingService; + private boolean isClosed; + private boolean isQueryFinished; + private boolean isQueryStarted; + private ConnectorRow currentRow; + private long processedRows; + private CompletableFuture dataTransferQueryFuture; + private Throwable queryExecutionException; + + public GpfdistRecordCursor(ContextManager contextManager, ReadContext readContext, + DataTransferQueryExecutor dataTransferQueryExecutor, + List columnTypes) + { + this.readContext = readContext; + this.contextManager = contextManager; + this.dataTransferQueryExecutor = dataTransferQueryExecutor; + this.columnTypes = columnTypes; + this.rowProcessingService = readContext.getRowProcessor(); + } + + @Override + public boolean advanceNextPosition() + { + if (isClosed) { + return false; + } + else { + long startNanoSec = System.nanoTime(); + try { + if (!isQueryStarted) { + dataTransferQueryFuture = executeTransferDataQuery(); + isQueryStarted = true; + } + while (queryExecutionException == null && (isDataTransferNotInitialized() || isDataNotProcessed())) { + try { + Thread.sleep(100); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + if (queryExecutionException != null) { + throw queryExecutionException; + } + else { + // if all request is completed and rows queue is empty + if (rowProcessingService.isEmpty()) { + log.info("Data processing is finished. Processing result: %s", + readContext.getSegmentDataProcessors().values()); + closeCtx(); + return false; + } + else { + currentRow = rowProcessingService.take(); + processedRows++; + readContext.getCompletedBytes().addAndGet(currentRow.getEstimatedSize()); + readContext.getMemoryUsage().addAndGet(-currentRow.getEstimatedSize()); + return true; + } + } + } + catch (Throwable e) { + if (!isQueryFinished) { + dataTransferQueryFuture.completeExceptionally(e); + } + log.warn("Processed rows to target: %d. Unloading from adb result: %s", + processedRows, + readContext.getSegmentDataProcessors().values()); + closeCtx(); + throw new TrinoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, + "Failed to unload data from adb: " + e.getMessage(), e); + } + finally { + readTimeNanos.addAndGet(System.nanoTime() - startNanoSec); + } + } + } + + private CompletableFuture executeTransferDataQuery() + { + return dataTransferQueryExecutor.execute().whenComplete((_, error) -> { + if (error != null) { + queryExecutionException = error; + throw new CompletionException(error); + } + isQueryFinished = true; + }); + } + + private boolean isDataNotProcessed() + { + //check that there are no rows in queue and segment processors are not finished yet + return rowProcessingService.isEmpty() + && readContext.getSegmentDataProcessors().values().stream().anyMatch(req -> req.getStatus() != SegmentRequestStatus.FINISHED); + } + + private boolean isDataTransferNotInitialized() + { + //checked that insert adb query is not finished and there are no segment processors yet + return !isQueryFinished && readContext.getSegmentDataProcessors().isEmpty(); + } + + @Override + public long getCompletedBytes() + { + return readContext.getCompletedBytes().get(); + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos.get(); + } + + @Override + public long getMemoryUsage() + { + return readContext.getMemoryUsage().get(); + } + + @Override + public Type getType(int field) + { + checkArgument(field < columnTypes.size()); + return columnTypes.get(field); + } + + @Override + public boolean getBoolean(int field) + { + return get(field); + } + + @Override + public long getLong(int field) + { + return get(field); + } + + @Override + public double getDouble(int field) + { + return get(field); + } + + @Override + public Slice getSlice(int field) + { + return get(field); + } + + @Override + public Object getObject(int field) + { + return get(field); + } + + @Override + public boolean isNull(int field) + { + return get(field) == null; + } + + @SuppressWarnings("unchecked") + private T get(int field) + { + return (T) currentRow.getColumnValues()[field].value(); + } + + @Override + public void close() + { + if (!this.isClosed) { + this.isClosed = true; + closeCtx(); + } + } + + private void closeCtx() + { + try { + readContext.close(); + } + finally { + contextManager.remove(readContext.getId()); + } + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSet.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSet.java new file mode 100644 index 000000000000..0246638be158 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSet.java @@ -0,0 +1,57 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; +import io.trino.plugin.adb.connector.protocol.gpfdist.DataTransferQueryExecutor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContext; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.spi.connector.RecordCursor; +import io.trino.spi.connector.RecordSet; +import io.trino.spi.type.Type; + +import java.util.List; + +public class GpfdistRecordSet + implements RecordSet +{ + private final ContextManager contextManager; + private final ReadContext readContext; + private final DataTransferQueryExecutor dataTransferQueryExecutor; + private final List columnTypes; + + public GpfdistRecordSet(ContextManager contextManager, + ReadContext readContext, + DataTransferQueryExecutor dataTransferQueryExecutor) + { + this.contextManager = contextManager; + this.readContext = readContext; + this.dataTransferQueryExecutor = dataTransferQueryExecutor; + columnTypes = readContext.getMetadata().getColumnHandles().stream() + .map(columnHandle -> ((JdbcColumnHandle) columnHandle).getColumnType()) + .toList(); + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + return new GpfdistRecordCursor(contextManager, readContext, dataTransferQueryExecutor, columnTypes); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSetProvider.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSetProvider.java new file mode 100644 index 000000000000..20b6d34ab7d1 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistRecordSetProvider.java @@ -0,0 +1,141 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import com.google.inject.Inject; +import io.trino.plugin.adb.AdbPluginConfig; +import io.trino.plugin.adb.connector.AdbJdbcSplit; +import io.trino.plugin.adb.connector.AdbSessionProperties; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.decode.RowDecoder; +import io.trino.plugin.adb.connector.decode.RowDecoderFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.ContextManager; +import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.DataTransferQueryExecutor; +import io.trino.plugin.adb.connector.protocol.gpfdist.ExecutorServiceProvider; +import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableFormatConfig; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableFormatConfigFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadata; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadataFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.context.ReadContext; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.query.GpfdistUnloadDataTransferQueryExecutor; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.ForRecordCursor; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcRecordSetProvider; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorRecordSetProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.RecordSet; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; + +public class GpfdistRecordSetProvider + implements ConnectorRecordSetProvider +{ + private static final ExternalTableType EXTERNAL_TABLE_TYPE = ExternalTableType.WRITABLE; + private final JdbcRecordSetProvider regularProvider; + private final AdbSqlClient client; + private final GpfdistUnloadMetadataFactory unloadMetadataFactory; + private final ExternalTableFormatConfigFactory externalTableFormatConfigFactory; + private final RowDecoderFactory rowDecoderFactory; + private final ExecutorService unloadQueryThreadExecutor; + private final CreateExternalTableQueryFactory externalTableQueryFactory; + private final ContextManager contextManager; + private final AdbPluginConfig pluginConfig; + private final InsertDataQueryFactory insertDataQueryFactory; + + @Inject + public GpfdistRecordSetProvider(@ForBaseJdbc JdbcClient client, + GpfdistUnloadMetadataFactory unloadMetadataFactory, + @ForRecordCursor ExecutorService recordSetExecutor, + ExternalTableFormatConfigFactory externalTableFormatConfigFactory, + RowDecoderFactory rowDecoderFactory, + Set createExternalTableQueryFactories, + ContextManager contextManager, + AdbPluginConfig pluginConfig, + Set insertDataQueryFactories) + { + this.regularProvider = new JdbcRecordSetProvider(client, recordSetExecutor); + this.client = (AdbSqlClient) client; + this.unloadMetadataFactory = unloadMetadataFactory; + this.externalTableFormatConfigFactory = externalTableFormatConfigFactory; + this.rowDecoderFactory = rowDecoderFactory; + this.contextManager = contextManager; + this.unloadQueryThreadExecutor = ExecutorServiceProvider.LOAD_DATA_QUERY_EXECUTOR_SERVICE; + Map externalTableQueryFactoryMap = createExternalTableQueryFactories.stream() + .collect(Collectors.toMap(CreateExternalTableQueryFactory::getExternalTableType, Function.identity())); + externalTableQueryFactory = externalTableQueryFactoryMap.get(EXTERNAL_TABLE_TYPE); + checkArgument(externalTableQueryFactory != null, + "failed to get writable table query factory by externalTableType %s", + EXTERNAL_TABLE_TYPE); + Map insertDataFactoryMap = insertDataQueryFactories.stream() + .collect(Collectors.toMap(InsertDataQueryFactory::getExternalTableType, Function.identity())); + insertDataQueryFactory = insertDataFactoryMap.get(EXTERNAL_TABLE_TYPE); + checkArgument(insertDataQueryFactory != null, + "failed to get insert data query factory by externalTableType %s", + EXTERNAL_TABLE_TYPE); + this.pluginConfig = pluginConfig; + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableHandle table, + List columns) + { + if (split instanceof AdbJdbcSplit) { + ExternalTableFormatConfig externalTableFormatConfig = externalTableFormatConfigFactory.create(); + GpfdistUnloadMetadata gpfdistUnloadMetadata = unloadMetadataFactory.create(session, + (JdbcTableHandle) table, + (AdbJdbcSplit) split, + columns, + externalTableFormatConfig); + RowDecoder rowDecoder = rowDecoderFactory.create(session, gpfdistUnloadMetadata.getDataTypes()); + GpfdistBufferedRowProcessingService rowProcessingService = + new GpfdistBufferedRowProcessingService(pluginConfig, new AtomicLong(0)); + ReadContext readContext = new ReadContext( + gpfdistUnloadMetadata, + rowProcessingService.getUsedBufferedRowsMemory(), + rowDecoder, + rowProcessingService); + DataTransferQueryExecutor dataTransferQueryExecutor = new GpfdistUnloadDataTransferQueryExecutor( + client, + session, + unloadQueryThreadExecutor, + gpfdistUnloadMetadata, + AdbSessionProperties.getGpfdistRetryTimeout(session), + externalTableQueryFactory, + insertDataQueryFactory); + contextManager.add(readContext); + return new GpfdistRecordSet(contextManager, readContext, dataTransferQueryExecutor); + } + return regularProvider.getRecordSet(transaction, session, split, table, columns); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistSegmentRequestProcessor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistSegmentRequestProcessor.java new file mode 100644 index 000000000000..09d94319408e --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/process/GpfdistSegmentRequestProcessor.java @@ -0,0 +1,79 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.process; + +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.InputDataProcessor; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.ProcessingDataResult; +import io.trino.plugin.adb.connector.protocol.gpfdist.unload.SegmentRequestStatus; + +import java.io.InputStream; + +import static java.lang.String.format; + +public class GpfdistSegmentRequestProcessor +{ + private final InputDataProcessor delegate; + private final int segmentId; + private final ProcessingDataResult processingDataResult; + private SegmentRequestStatus status; + + public GpfdistSegmentRequestProcessor(int segmentId, InputDataProcessor delegate) + { + this.delegate = delegate; + this.segmentId = segmentId; + this.processingDataResult = delegate.getProcessedDataResult(); + status = SegmentRequestStatus.INITIALIZED; + } + + public void process(InputStream dataStream) + { + try { + if (status != SegmentRequestStatus.PROCESSING) { + status = SegmentRequestStatus.PROCESSING; + } + delegate.process(dataStream); + } + catch (Exception e) { + status = SegmentRequestStatus.ERROR; + throw new RuntimeException(format("Failed to process data: %s. error: %s", this, e.getMessage()), e); + } + } + + public void stop() + { + status = SegmentRequestStatus.FINISHED; + } + + public SegmentRequestStatus getStatus() + { + return status; + } + + public int getSegmentId() + { + return segmentId; + } + + public ProcessingDataResult getProcessedDataResult() + { + return processingDataResult; + } + + @Override + public String toString() + { + return "Data processing for segmentId: " + segmentId + "; status: " + + status + "; result: " + processingDataResult; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/CreateWritableExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/CreateWritableExternalTableQueryFactory.java new file mode 100644 index 000000000000..adfb201d7b5e --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/CreateWritableExternalTableQueryFactory.java @@ -0,0 +1,56 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.query; + +import io.trino.plugin.adb.connector.AdbJdbcSplit; +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadata; + +import java.util.HashSet; +import java.util.List; +import java.util.stream.Collectors; + +public class CreateWritableExternalTableQueryFactory + extends AbstractExternalTableQueryFactory +{ + @Override + public String createQuery(GpfdistMetadata metadata) + { + GpfdistUnloadMetadata unloadMetadata = (GpfdistUnloadMetadata) metadata; + String externalTableDistribution = createExternalTableDistribution((AdbJdbcSplit) unloadMetadata.getSplit(), + unloadMetadata.getColumnNames()); + return createCommonQuery(metadata) + " " + externalTableDistribution; + } + + private String createExternalTableDistribution(AdbJdbcSplit split, List columns) + { + List distribution = split.getDistribution(); + if (!distribution.isEmpty() && new HashSet<>(columns).containsAll(distribution)) { + return distribution.stream() + .map(columnName -> "\"" + columnName + "\"") + .collect(Collectors.joining(", ", "DISTRIBUTED BY (", ")")); + } + else { + return "DISTRIBUTED RANDOMLY"; + } + } + + @Override + public ExternalTableType getExternalTableType() + { + return ExternalTableType.WRITABLE; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/GpfdistUnloadDataTransferQueryExecutor.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/GpfdistUnloadDataTransferQueryExecutor.java new file mode 100644 index 000000000000..0265f3c9d7cb --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/GpfdistUnloadDataTransferQueryExecutor.java @@ -0,0 +1,85 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.query; + +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.protocol.gpfdist.AbstractDataTransferQueryExecutor; +import io.trino.plugin.adb.connector.protocol.gpfdist.CreateExternalTableQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadata; +import io.trino.plugin.jdbc.PreparedQuery; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.SQLException; +import java.util.Optional; +import java.util.concurrent.ExecutorService; + +public class GpfdistUnloadDataTransferQueryExecutor + extends AbstractDataTransferQueryExecutor +{ + private static final Logger log = Logger.get(GpfdistUnloadDataTransferQueryExecutor.class); + private final GpfdistUnloadMetadata unloadMetadata; + private final Optional gpfdistRetryTimeout; + + public GpfdistUnloadDataTransferQueryExecutor(AdbSqlClient client, + ConnectorSession session, + ExecutorService executor, + GpfdistUnloadMetadata unloadMetadata, + Optional gpfdistRetryTimeout, + CreateExternalTableQueryFactory externalTableQueryFactory, + InsertDataQueryFactory insertDataQueryFactory) + { + super(client, session, executor, externalTableQueryFactory, insertDataQueryFactory); + this.unloadMetadata = unloadMetadata; + this.gpfdistRetryTimeout = gpfdistRetryTimeout; + } + + @Override + protected void executeQueries() + throws SQLException + { + createReadableExternalTable(); + setGpfdistRetryTimeoutIfNeeded(); + insertIntoExternalTable(); + } + + private void createReadableExternalTable() + throws SQLException + { + String sql = externalTableQueryFactory.createQuery(unloadMetadata); + client.execute(session, connection, sql); + log.info("Executed create writable external table query: %s", sql); + } + + private void setGpfdistRetryTimeoutIfNeeded() + throws SQLException + { + if (gpfdistRetryTimeout.isPresent()) { + long retryInSeconds = gpfdistRetryTimeout.get().toJavaTime().toSeconds(); + String query = String.format("SET gpfdist_retry_timeout TO %d", retryInSeconds); + client.execute(session, connection, query); + log.info("Executed set gpfdist retry timeout query: %s", query); + } + } + + private void insertIntoExternalTable() + throws SQLException + { + PreparedQuery query = insertDataQueryFactory.create(session, connection, unloadMetadata); + client.executeAsPreparedStatement(session, connection, query); + log.info("Executed insert into writable external table from source table query: %s", query.query()); + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/InsertDataToExternalTableQueryFactory.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/InsertDataToExternalTableQueryFactory.java new file mode 100644 index 000000000000..7b5b7c283724 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/protocol/gpfdist/unload/query/InsertDataToExternalTableQueryFactory.java @@ -0,0 +1,74 @@ +/* + * 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.adb.connector.protocol.gpfdist.unload.query; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.trino.plugin.adb.connector.AdbSqlClient; +import io.trino.plugin.adb.connector.protocol.gpfdist.InsertDataQueryFactory; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.ExternalTableType; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistMetadata; +import io.trino.plugin.adb.connector.protocol.gpfdist.metadata.GpfdistUnloadMetadata; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.PreparedQuery; +import io.trino.spi.connector.ConnectorSession; + +import java.sql.Connection; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class InsertDataToExternalTableQueryFactory + implements InsertDataQueryFactory +{ + private final AdbSqlClient client; + + @Inject + public InsertDataToExternalTableQueryFactory(@ForBaseJdbc JdbcClient client) + { + this.client = (AdbSqlClient) client; + } + + @Override + public PreparedQuery create(ConnectorSession session, Connection connection, GpfdistMetadata metadata) + { + GpfdistUnloadMetadata unloadMetadata = (GpfdistUnloadMetadata) metadata; + PreparedQuery preparedSelectQuery = client.prepareQuery(session, + connection, + unloadMetadata.getTargetTableHandle(), + Optional.empty(), + (List) unloadMetadata.getColumnHandles(), + ImmutableMap.of(), + Optional.of(unloadMetadata.getSplit())); + Function insertTransformer = query -> unloadMetadata.getColumnHandles().isEmpty() + ? String.format("INSERT INTO %s %s", unloadMetadata.getSourceTable(), query) + : String.format( + "INSERT INTO %s (%s) %s", + unloadMetadata.getSourceTable(), + unloadMetadata.getColumnHandles().stream() + .map(column -> client.quoted(((JdbcColumnHandle) column).getColumnName())) + .collect(Collectors.joining(", ")), + query); + return preparedSelectQuery.transformQuery(insertTransformer); + } + + @Override + public ExternalTableType getExternalTableType() + { + return ExternalTableType.WRITABLE; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManager.java new file mode 100644 index 000000000000..d176b1500d31 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManager.java @@ -0,0 +1,23 @@ +/* + * 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.adb.connector.table; + +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitSource; + +public interface SplitSourceManager +{ + ConnectorSplitSource getSplits(ConnectorSession session, JdbcTableHandle tableHandle); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManagerImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManagerImpl.java new file mode 100644 index 000000000000..9e771614a04c --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/SplitSourceManagerImpl.java @@ -0,0 +1,135 @@ +/* + * 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.adb.connector.table; + +import com.google.inject.Inject; +import io.trino.plugin.adb.connector.AdbJdbcSplit; +import io.trino.plugin.adb.connector.AdbSessionProperties; +import io.trino.plugin.adb.connector.metadata.AdbMetadataDao; +import io.trino.plugin.base.mapping.IdentifierMapping; +import io.trino.plugin.jdbc.JdbcSplit; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.RemoteTableName; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.FixedSplitSource; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.IntStream; + +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.adb.AdbPluginConfig.IDENTIFIER_QUOTE; +import static java.lang.Math.min; +import static java.lang.String.format; + +public class SplitSourceManagerImpl + implements SplitSourceManager +{ + private static final int MINIMUM_PARALLELISM_VALUE = 1; + private final AdbMetadataDao metadata; + private final IdentifierMapping identifierMapping; + + @Inject + public SplitSourceManagerImpl(AdbMetadataDao metadata, IdentifierMapping identifierMapping) + { + this.metadata = metadata; + this.identifierMapping = identifierMapping; + } + + @Override + public ConnectorSplitSource getSplits(ConnectorSession session, JdbcTableHandle tableHandle) + { + int parallelism = getSplitParallelism(session, tableHandle); + return new FixedSplitSource(createAdbSplits(session, tableHandle, segmentedSplits(parallelism))); + } + + private int getSplitParallelism(ConnectorSession session, JdbcTableHandle tableHandle) + { + if (!tableHandle.isNamedRelation()) { + return MINIMUM_PARALLELISM_VALUE; + } + else { + int maxParallelism = AdbSessionProperties.getMaxScanParallelism(session); + if (maxParallelism <= MINIMUM_PARALLELISM_VALUE) { + return MINIMUM_PARALLELISM_VALUE; + } + else { + boolean segmented = this.metadata.isSegmentedTable(session, + quoted(tableHandle.getRequiredNamedRelation().getRemoteTableName())); + if (!segmented) { + return MINIMUM_PARALLELISM_VALUE; + } + else { + int segmentCount = this.metadata.getSegmentCount(session); + if (segmentCount <= 0) { + segmentCount = MINIMUM_PARALLELISM_VALUE; + } + return min(segmentCount, maxParallelism); + } + } + } + } + + private List segmentedSplits(int parallelism) + { + return parallelism == MINIMUM_PARALLELISM_VALUE + ? List.of(new JdbcSplit(Optional.empty())) + : IntStream.range(0, parallelism) + .boxed() + .map(id -> new JdbcSplit(Optional.of(format("gp_segment_id %% %s = %s", parallelism, id)))) + .collect(toImmutableList()); + } + + private List createAdbSplits(ConnectorSession session, + JdbcTableHandle tableHandle, + List splits) + { + String objectName = quoted(tableHandle.getRequiredNamedRelation().getRemoteTableName()); + Map tableProperties = this.metadata.getTableProperties(session, objectName, identifierMapping); + List distributionInfo = AdbTableProperties.getDistributedBy(tableProperties).orElse(List.of()); + return splits.stream() + .map(split -> new AdbJdbcSplit(distributionInfo, split.getAdditionalPredicate(), split.getDynamicFilter())) + .toList(); + } + + public String quoted(RemoteTableName remoteTableName) + { + return quoted( + remoteTableName.getCatalogName().orElse(null), + remoteTableName.getSchemaName().orElse(null), + remoteTableName.getTableName()); + } + + private String quoted(String catalog, String schema, String table) + { + StringBuilder sb = new StringBuilder(); + if (!isNullOrEmpty(catalog)) { + sb.append(quoted(catalog)).append("."); + } + if (!isNullOrEmpty(schema)) { + sb.append(quoted(schema)).append("."); + } + sb.append(quoted(table)); + return sb.toString(); + } + + public String quoted(String name) + { + name = name.replace(IDENTIFIER_QUOTE, IDENTIFIER_QUOTE + IDENTIFIER_QUOTE); + return IDENTIFIER_QUOTE + name + IDENTIFIER_QUOTE; + } +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManager.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManager.java new file mode 100644 index 000000000000..034ee4d8ff68 --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManager.java @@ -0,0 +1,26 @@ +/* + * 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.adb.connector.table; + +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.statistics.TableStatistics; + +import java.util.List; + +public interface StatisticsManager +{ + TableStatistics getTableStatistics(ConnectorSession session, JdbcTableHandle handle, List columns); +} diff --git a/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManagerImpl.java b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManagerImpl.java new file mode 100644 index 000000000000..e505c4628a2a --- /dev/null +++ b/plugin/trino-adb/src/main/java/io/trino/plugin/adb/connector/table/StatisticsManagerImpl.java @@ -0,0 +1,255 @@ +/* + * 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.adb.connector.table; + +import com.google.inject.Inject; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcStatisticsConfig; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.RemoteTableName; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.statistics.ColumnStatistics; +import io.trino.spi.statistics.Estimate; +import io.trino.spi.statistics.TableStatistics; +import org.jdbi.v3.core.Handle; +import org.jdbi.v3.core.Jdbi; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Throwables.throwIfInstanceOf; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; + +public class StatisticsManagerImpl + implements StatisticsManager +{ + private final ConnectionFactory connectionFactory; + private final boolean statisticsEnabled; + + @Inject + public StatisticsManagerImpl(ConnectionFactory connectionFactory, + JdbcStatisticsConfig statisticsConfig) + { + this.connectionFactory = connectionFactory; + statisticsEnabled = statisticsConfig.isEnabled(); + } + + @Override + public TableStatistics getTableStatistics(ConnectorSession session, + JdbcTableHandle handle, + List columns) + { + if (!statisticsEnabled) { + return TableStatistics.empty(); + } + if (!handle.isNamedRelation()) { + return TableStatistics.empty(); + } + try { + return readTableStatistics(session, handle, columns); + } + catch (SQLException | RuntimeException e) { + throwIfInstanceOf(e, TrinoException.class); + throw new TrinoException(JDBC_ERROR, "Failed fetching statistics for table: " + handle, e); + } + } + + private TableStatistics readTableStatistics(ConnectorSession session, JdbcTableHandle table, List columns) + throws SQLException + { + checkArgument(table.isNamedRelation(), "Relation is not a table: %s", table); + try (Connection connection = connectionFactory.openConnection(session); + Handle handle = Jdbi.open(connection)) { + StatisticsDao statisticsDao = new StatisticsDao(handle); + + Optional optionalRowCount = readRowCountTableStat(statisticsDao, table); + if (optionalRowCount.isEmpty()) { + // Table not found + return TableStatistics.empty(); + } + long rowCount = optionalRowCount.get(); + if (rowCount == -1) { + // Table has never yet been vacuumed or analyzed + return TableStatistics.empty(); + } + TableStatistics.Builder tableStatistics = TableStatistics.builder(); + tableStatistics.setRowCount(Estimate.of(rowCount)); + + if (rowCount == 0) { + return tableStatistics.build(); + } + + RemoteTableName remoteTableName = table.getRequiredNamedRelation().getRemoteTableName(); + Map columnStatistics = statisticsDao.getColumnStatistics(remoteTableName.getSchemaName().orElse(null), remoteTableName.getTableName()).stream() + .collect(toImmutableMap(ColumnStatisticsResult::columnName, identity())); + + for (JdbcColumnHandle column : columns) { + ColumnStatisticsResult result = columnStatistics.get(column.getColumnName()); + if (result == null) { + continue; + } + + ColumnStatistics statistics = ColumnStatistics.builder() + .setNullsFraction(result.nullsFraction() + .map(Estimate::of) + .orElseGet(Estimate::unknown)) + .setDistinctValuesCount(result.distinctValuesIndicator() + .map(distinctValuesIndicator -> { + if (distinctValuesIndicator >= 0.0) { + return distinctValuesIndicator; + } + return -distinctValuesIndicator * rowCount; + }) + .map(Estimate::of) + .orElseGet(Estimate::unknown)) + .setDataSize(result.averageColumnLength() + .flatMap(averageColumnLength -> + result.nullsFraction().map(nullsFraction -> + Estimate.of(1.0 * averageColumnLength * rowCount * (1 - nullsFraction)))) + .orElseGet(Estimate::unknown)) + .build(); + + tableStatistics.setColumnStatistics(column, statistics); + } + + return tableStatistics.build(); + } + } + + private static Optional readRowCountTableStat(StatisticsDao statisticsDao, JdbcTableHandle table) + { + RemoteTableName remoteTableName = table.getRequiredNamedRelation().getRemoteTableName(); + String schemaName = remoteTableName.getSchemaName().orElse(null); + Optional rowCount = statisticsDao.getRowCountFromPgClass(schemaName, remoteTableName.getTableName()); + if (rowCount.isEmpty()) { + // Table not found + return Optional.empty(); + } + if (statisticsDao.isPartitionedTable(schemaName, remoteTableName.getTableName())) { + Optional partitionedTableRowCount = statisticsDao.getRowCountPartitionedTableFromPgClass(schemaName, remoteTableName.getTableName()); + if (partitionedTableRowCount.isPresent()) { + return partitionedTableRowCount; + } + + return statisticsDao.getRowCountPartitionedTableFromPgStats(schemaName, remoteTableName.getTableName()); + } + if (rowCount.get() == 0) { + rowCount = statisticsDao.getRowCountFromPgStat(schemaName, remoteTableName.getTableName()); + } + return rowCount; + } + + private static class StatisticsDao + { + private final Handle handle; + + public StatisticsDao(Handle handle) + { + this.handle = requireNonNull(handle, "handle is null"); + } + + Optional getRowCountFromPgClass(String schema, String tableName) + { + return handle.createQuery("SELECT reltuples " + + "FROM pg_class " + + "WHERE relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = :schema) " + + "AND relname = :table_name") + .bind("schema", schema) + .bind("table_name", tableName) + .mapTo(Long.class) + .findOne(); + } + + Optional getRowCountFromPgStat(String schema, String tableName) + { + return handle.createQuery("SELECT n_live_tup FROM pg_stat_all_tables WHERE schemaname = :schema AND relname = :table_name") + .bind("schema", schema) + .bind("table_name", tableName) + .mapTo(Long.class) + .findOne(); + } + + Optional getRowCountPartitionedTableFromPgClass(String schema, String tableName) + { + return handle.createQuery("SELECT SUM(child.reltuples) " + + "FROM pg_inherits " + + "JOIN pg_class parent ON pg_inherits.inhparent = parent.oid " + + "JOIN pg_class child ON pg_inherits.inhrelid = child.oid " + + "JOIN pg_namespace parent_ns ON parent_ns.oid = parent.relnamespace " + + "JOIN pg_namespace child_ns ON child_ns.oid = child.relnamespace " + + "WHERE parent.oid = :schema_table_name::regclass") + .bind("schema_table_name", format("%s.%s", schema, tableName)) + .mapTo(Long.class) + .findOne(); + } + + Optional getRowCountPartitionedTableFromPgStats(String schema, String tableName) + { + return handle.createQuery("SELECT SUM(stat.n_live_tup) " + + "FROM pg_inherits " + + "JOIN pg_class parent ON pg_inherits.inhparent = parent.oid " + + "JOIN pg_class child ON pg_inherits.inhrelid = child.oid " + + "JOIN pg_namespace parent_ns ON parent_ns.oid = parent.relnamespace " + + "JOIN pg_namespace child_ns ON child_ns.oid = child.relnamespace " + + "JOIN pg_stat_all_tables stat ON stat.schemaname = child_ns.nspname AND stat.relname = child.relname " + + "WHERE parent.oid = :schema_table_name::regclass") + .bind("schema_table_name", format("%s.%s", schema, tableName)) + .mapTo(Long.class) + .findOne(); + } + + List getColumnStatistics(String schema, String tableName) + { + return handle.createQuery("SELECT attname, null_frac, n_distinct, avg_width FROM pg_stats WHERE schemaname = :schema AND tablename = :table_name") + .bind("schema", schema) + .bind("table_name", tableName) + .map((rs, ctx) -> new ColumnStatisticsResult( + requireNonNull(rs.getString("attname"), "attname is null"), + Optional.ofNullable(rs.getObject("null_frac", Float.class)), + Optional.ofNullable(rs.getObject("n_distinct", Float.class)), + Optional.ofNullable(rs.getObject("avg_width", Integer.class)))) + .list(); + } + + boolean isPartitionedTable(String schema, String tableName) + { + return handle.createQuery("SELECT true " + + "FROM pg_class " + + "WHERE relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = :schema) " + + "AND relname = :table_name " + + "AND relkind = 'p'") + .bind("schema", schema) + .bind("table_name", tableName) + .mapTo(Boolean.class) + .findOne() + .orElse(false); + } + } + + private record ColumnStatisticsResult(String columnName, Optional nullsFraction, + Optional distinctValuesIndicator, + Optional averageColumnLength) + { + } +} diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index 60d21a787243..ebd3b1198f49 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -26,6 +26,7 @@ query.client.timeout=5m query.min-expire-age=30m plugin.bundles=\ + ../../plugin/trino-adb/pom.xml, \ ../../plugin/trino-resource-group-managers/pom.xml,\ ../../plugin/trino-password-authenticators/pom.xml, \ ../../plugin/trino-iceberg/pom.xml,\ diff --git a/testing/trino-server-dev/etc/log.properties b/testing/trino-server-dev/etc/log.properties index b615d661c74a..40f0b939e26c 100644 --- a/testing/trino-server-dev/etc/log.properties +++ b/testing/trino-server-dev/etc/log.properties @@ -5,7 +5,7 @@ # in production. For example configuration, see the Trino documentation. # -io.trino=INFO +io.trino=DEBUG # show classpath for plugins io.trino.server.PluginManager=DEBUG