Skip to content

Commit 88e3c6e

Browse files
feat: DH-18664: Add support to read from uri's with scheme "s3a" (deephaven#6640)
1 parent 0e5a012 commit 88e3c6e

15 files changed

+443
-99
lines changed

extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/S3WarehouseSqliteCatalogBase.java

+138-4
Original file line numberDiff line numberDiff line change
@@ -3,22 +3,42 @@
33
//
44
package io.deephaven.iceberg.util;
55

6+
import io.deephaven.engine.table.Table;
7+
import io.deephaven.engine.util.TableTools;
68
import io.deephaven.extensions.s3.S3Instructions;
9+
import io.deephaven.iceberg.base.IcebergUtils;
710
import io.deephaven.iceberg.junit5.SqliteCatalogBase;
11+
import io.deephaven.iceberg.sqlite.SqliteHelper;
812
import org.apache.iceberg.CatalogProperties;
13+
import org.apache.iceberg.DataFile;
14+
import org.apache.iceberg.DataFiles;
915
import org.apache.iceberg.aws.s3.S3FileIO;
1016
import org.junit.jupiter.api.TestInfo;
17+
import org.junit.jupiter.api.io.TempDir;
18+
import org.junit.jupiter.api.Test;
19+
import org.apache.iceberg.catalog.TableIdentifier;
1120
import software.amazon.awssdk.services.s3.S3AsyncClient;
1221
import software.amazon.awssdk.services.s3.model.CreateBucketRequest;
22+
import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
23+
import software.amazon.awssdk.services.s3.model.NoSuchBucketException;
1324

1425
import java.nio.file.Path;
26+
import java.util.HashMap;
27+
import java.util.List;
1528
import java.util.Locale;
1629
import java.util.Map;
1730
import java.util.concurrent.ExecutionException;
1831
import java.util.concurrent.TimeUnit;
1932
import java.util.concurrent.TimeoutException;
33+
import java.util.stream.Collectors;
2034

35+
import static io.deephaven.engine.testutil.TstUtils.assertTableEquals;
36+
import static io.deephaven.engine.util.TableTools.doubleCol;
37+
import static io.deephaven.engine.util.TableTools.intCol;
2138
import static io.deephaven.extensions.s3.testlib.S3Helper.TIMEOUT_SECONDS;
39+
import static io.deephaven.iceberg.base.IcebergUtils.dataFileUri;
40+
import static io.deephaven.iceberg.base.IcebergUtils.locationUri;
41+
import static org.assertj.core.api.Assertions.assertThat;
2242

2343
abstract class S3WarehouseSqliteCatalogBase extends SqliteCatalogBase {
2444

@@ -32,17 +52,131 @@ public final Object dataInstructions() {
3252
}
3353

3454
@Override
35-
protected IcebergCatalogAdapter catalogAdapter(TestInfo testInfo, Path rootDir, Map<String, String> properties)
55+
protected IcebergCatalogAdapter catalogAdapter(
56+
final TestInfo testInfo,
57+
final Path rootDir,
58+
final Map<String, String> properties)
59+
throws ExecutionException, InterruptedException, TimeoutException {
60+
return catalogAdapterForScheme(testInfo, properties, "s3");
61+
}
62+
63+
private IcebergCatalogAdapter catalogAdapterForScheme(
64+
final TestInfo testInfo,
65+
final Map<String, String> properties,
66+
final String scheme)
3667
throws ExecutionException, InterruptedException, TimeoutException {
3768
final String methodName = testInfo.getTestMethod().orElseThrow().getName();
3869
final String catalogName = methodName + "-catalog";
3970
final String bucket = methodName.toLowerCase(Locale.US) + "-bucket";
4071
try (final S3AsyncClient client = s3AsyncClient()) {
41-
client.createBucket(CreateBucketRequest.builder().bucket(bucket).build())
42-
.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
72+
if (!doesBucketExist(client, bucket)) {
73+
client.createBucket(CreateBucketRequest.builder().bucket(bucket).build())
74+
.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
75+
}
4376
}
44-
properties.put(CatalogProperties.WAREHOUSE_LOCATION, "s3://" + bucket + "/warehouse");
77+
properties.put(CatalogProperties.WAREHOUSE_LOCATION, scheme + "://" + bucket + "/warehouse");
4578
properties.put(CatalogProperties.FILE_IO_IMPL, S3FileIO.class.getName());
4679
return IcebergToolsS3.createAdapter(catalogName, properties, Map.of(), s3Instructions());
4780
}
81+
82+
private boolean doesBucketExist(final S3AsyncClient client, final String bucketName)
83+
throws ExecutionException, InterruptedException, TimeoutException {
84+
try {
85+
client.headBucket(HeadBucketRequest.builder().bucket(bucketName).build())
86+
.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
87+
return true;
88+
} catch (ExecutionException e) {
89+
if (e.getCause() instanceof NoSuchBucketException) {
90+
return false;
91+
}
92+
throw e;
93+
}
94+
}
95+
96+
@Test
97+
void testIcebergTablesWithS3AScheme(TestInfo testInfo, @TempDir Path rootDir)
98+
throws ExecutionException, InterruptedException, TimeoutException {
99+
testIcebergTablesWithCustomScheme("s3a", testInfo, rootDir);
100+
}
101+
102+
@Test
103+
void testIcebergTablesWithS3NScheme(TestInfo testInfo, @TempDir Path rootDir)
104+
throws ExecutionException, InterruptedException, TimeoutException {
105+
testIcebergTablesWithCustomScheme("s3n", testInfo, rootDir);
106+
}
107+
108+
private void testIcebergTablesWithCustomScheme(final String scheme, TestInfo testInfo, @TempDir Path rootDir)
109+
throws ExecutionException, InterruptedException, TimeoutException {
110+
final Map<String, String> properties = new HashMap<>();
111+
SqliteHelper.setJdbcCatalogProperties(properties, rootDir);
112+
final IcebergCatalogAdapter catalogAdapter = catalogAdapterForScheme(testInfo, properties, scheme);
113+
114+
final TableIdentifier tableIdentifier = TableIdentifier.parse("MyNamespace.MyTable");
115+
116+
final Table data = TableTools.newTable(
117+
intCol("intCol", 2, 4, 6, 8, 10),
118+
doubleCol("doubleCol", 2.5, 5.0, 7.5, 10.0, 12.5));
119+
120+
// Create a new iceberg table
121+
final IcebergTableAdapter tableAdapter = catalogAdapter.createTable(tableIdentifier, data.getDefinition());
122+
final org.apache.iceberg.Table icebergTable = tableAdapter.icebergTable();
123+
124+
// Verify that the table location has the right scheme
125+
assertThat(locationUri(icebergTable).getScheme()).isEqualTo(scheme);
126+
127+
// Add data to the table
128+
final IcebergTableWriter tableWriter = tableAdapter.tableWriter(writerOptionsBuilder()
129+
.tableDefinition(data.getDefinition())
130+
.build());
131+
tableWriter.append(IcebergWriteInstructions.builder()
132+
.addTables(data, data)
133+
.build());
134+
135+
// Verify all data files have the right scheme
136+
final List<DataFile> dataFiles = IcebergUtils.allDataFiles(icebergTable, icebergTable.currentSnapshot())
137+
.collect(Collectors.toList());
138+
assertThat(dataFiles).hasSize(2);
139+
assertThat(dataFiles).allMatch(dataFile -> dataFileUri(icebergTable, dataFile).getScheme().equals(scheme));
140+
141+
// Verify the data is correct
142+
Table fromIceberg = tableAdapter.table();
143+
Table expected = TableTools.merge(data, data);
144+
assertTableEquals(expected, fromIceberg);
145+
146+
// Create a new data file but with s3 scheme
147+
final DataFile existingDataFile = dataFiles.get(0);
148+
final String existingDataFileLocation = existingDataFile.location();
149+
assertThat(existingDataFileLocation).startsWith(scheme);
150+
final String newLocation = existingDataFileLocation.replace(scheme + "://", "s3://");
151+
final DataFile newDataFile = DataFiles.builder(icebergTable.spec())
152+
.withPath(newLocation)
153+
.withFormat(existingDataFile.format())
154+
.withRecordCount(existingDataFile.recordCount())
155+
.withFileSizeInBytes(existingDataFile.fileSizeInBytes())
156+
.build();
157+
158+
// Append the new data files to the table
159+
icebergTable.newAppend().appendFile(newDataFile).commit();
160+
161+
// Verify the new data file has the right scheme
162+
final List<DataFile> newDataFiles = IcebergUtils.allDataFiles(icebergTable, icebergTable.currentSnapshot())
163+
.collect(Collectors.toList());
164+
int s3DataFiles = 0;
165+
int nonS3DataFiles = 0;
166+
for (final DataFile dataFile : newDataFiles) {
167+
if (dataFileUri(icebergTable, dataFile).getScheme().equals(scheme)) {
168+
nonS3DataFiles++;
169+
} else {
170+
assertThat(dataFileUri(icebergTable, dataFile).getScheme()).isEqualTo("s3");
171+
s3DataFiles++;
172+
}
173+
}
174+
assertThat(s3DataFiles).isEqualTo(1);
175+
assertThat(nonS3DataFiles).isEqualTo(2);
176+
177+
// Verify the data is correct
178+
fromIceberg = tableAdapter.table();
179+
expected = TableTools.merge(expected, data);
180+
assertTableEquals(expected, fromIceberg);
181+
}
48182
}

extensions/iceberg/src/main/java/io/deephaven/iceberg/base/IcebergUtils.java

+16
Original file line numberDiff line numberDiff line change
@@ -3,9 +3,11 @@
33
//
44
package io.deephaven.iceberg.base;
55

6+
import io.deephaven.base.FileUtils;
67
import io.deephaven.engine.table.ColumnDefinition;
78
import io.deephaven.engine.table.TableDefinition;
89
import io.deephaven.engine.table.impl.locations.TableDataException;
10+
import io.deephaven.iceberg.relative.RelativeFileIO;
911
import io.deephaven.iceberg.util.IcebergReadInstructions;
1012
import org.apache.iceberg.DataFile;
1113
import org.apache.iceberg.ManifestContent;
@@ -21,6 +23,7 @@
2123
import org.apache.iceberg.catalog.SupportsNamespaces;
2224
import org.apache.iceberg.exceptions.AlreadyExistsException;
2325
import org.apache.iceberg.exceptions.NamespaceNotEmptyException;
26+
import org.apache.iceberg.io.FileIO;
2427
import org.apache.iceberg.types.Type;
2528
import org.apache.iceberg.types.Types;
2629
import org.jetbrains.annotations.NotNull;
@@ -29,6 +32,7 @@
2932
import java.io.IOException;
3033
import java.io.UncheckedIOException;
3134
import java.math.BigDecimal;
35+
import java.net.URI;
3236
import java.time.Instant;
3337
import java.time.LocalDate;
3438
import java.time.LocalDateTime;
@@ -127,6 +131,18 @@ public static <T> Stream<T> toStream(final org.apache.iceberg.io.CloseableIterab
127131
});
128132
}
129133

134+
private static String path(String path, FileIO io) {
135+
return io instanceof RelativeFileIO ? ((RelativeFileIO) io).absoluteLocation(path) : path;
136+
}
137+
138+
public static URI locationUri(Table table) {
139+
return FileUtils.convertToURI(path(table.location(), table.io()), true);
140+
}
141+
142+
public static URI dataFileUri(Table table, DataFile dataFile) {
143+
return FileUtils.convertToURI(path(dataFile.path().toString(), table.io()), false);
144+
}
145+
130146
/**
131147
* Convert an Iceberg data type to a Deephaven type.
132148
*

extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java

+35-24
Original file line numberDiff line numberDiff line change
@@ -3,14 +3,12 @@
33
//
44
package io.deephaven.iceberg.layout;
55

6-
import io.deephaven.base.FileUtils;
76
import io.deephaven.engine.table.TableDefinition;
87
import io.deephaven.engine.table.impl.locations.TableDataException;
98
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
109
import io.deephaven.iceberg.base.IcebergUtils;
1110
import io.deephaven.iceberg.location.IcebergTableLocationKey;
1211
import io.deephaven.iceberg.location.IcebergTableParquetLocationKey;
13-
import io.deephaven.iceberg.relative.RelativeFileIO;
1412
import io.deephaven.iceberg.util.IcebergReadInstructions;
1513
import io.deephaven.iceberg.util.IcebergTableAdapter;
1614
import io.deephaven.parquet.table.ParquetInstructions;
@@ -20,26 +18,37 @@
2018
import org.apache.iceberg.*;
2119
import org.apache.iceberg.catalog.Catalog;
2220
import org.apache.iceberg.catalog.TableIdentifier;
23-
import org.apache.iceberg.io.FileIO;
2421
import org.jetbrains.annotations.NotNull;
2522
import org.jetbrains.annotations.Nullable;
2623

2724
import java.net.URI;
25+
import java.util.HashMap;
2826
import java.util.List;
2927
import java.util.Map;
30-
import java.util.Objects;
3128
import java.util.UUID;
3229
import java.util.function.Consumer;
3330
import java.util.stream.Stream;
3431

3532
import static io.deephaven.iceberg.base.IcebergUtils.allManifestFiles;
33+
import static io.deephaven.iceberg.base.IcebergUtils.dataFileUri;
34+
import static io.deephaven.iceberg.base.IcebergUtils.locationUri;
3635

3736
public abstract class IcebergBaseLayout implements TableLocationKeyFinder<IcebergTableLocationKey> {
3837
/**
3938
* The {@link IcebergTableAdapter} that will be used to access the table.
4039
*/
4140
final IcebergTableAdapter tableAdapter;
4241

42+
/**
43+
* The instructions for customizations while reading.
44+
*/
45+
final IcebergReadInstructions instructions;
46+
47+
/**
48+
* The instructions for customizations while reading.
49+
*/
50+
final DataInstructionsProviderLoader dataInstructionsProvider;
51+
4352
/**
4453
* The UUID of the table, if available.
4554
*/
@@ -81,7 +90,7 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder<Iceber
8190
* The {@link SeekableChannelsProvider} object that will be used for {@link IcebergTableParquetLocationKey}
8291
* creation.
8392
*/
84-
private final SeekableChannelsProvider channelsProvider;
93+
private final Map<String, SeekableChannelsProvider> uriSchemeTochannelsProviders;
8594

8695

8796
/**
@@ -100,7 +109,8 @@ protected IcebergTableLocationKey locationKey(
100109
@NotNull final ManifestFile manifestFile,
101110
@NotNull final DataFile dataFile,
102111
@NotNull final URI fileUri,
103-
@Nullable final Map<String, Comparable<?>> partitions) {
112+
@Nullable final Map<String, Comparable<?>> partitions,
113+
@NotNull final SeekableChannelsProvider channelsProvider) {
104114
final org.apache.iceberg.FileFormat format = dataFile.format();
105115
if (format == org.apache.iceberg.FileFormat.PARQUET) {
106116
return new IcebergTableParquetLocationKey(catalogName, tableUuid, tableIdentifier, manifestFile, dataFile,
@@ -119,6 +129,8 @@ public IcebergBaseLayout(
119129
@NotNull final IcebergReadInstructions instructions,
120130
@NotNull final DataInstructionsProviderLoader dataInstructionsProvider) {
121131
this.tableAdapter = tableAdapter;
132+
this.instructions = instructions;
133+
this.dataInstructionsProvider = dataInstructionsProvider;
122134
{
123135
UUID uuid;
124136
try {
@@ -158,22 +170,26 @@ public IcebergBaseLayout(
158170
}
159171
this.parquetInstructions = builder.build();
160172
}
161-
this.channelsProvider = SeekableChannelsProviderLoader.getInstance().load(uriScheme, specialInstructions);
162-
}
163173

164-
abstract IcebergTableLocationKey keyFromDataFile(ManifestFile manifestFile, DataFile dataFile, URI fileUri);
165-
166-
private static String path(String path, FileIO io) {
167-
return io instanceof RelativeFileIO ? ((RelativeFileIO) io).absoluteLocation(path) : path;
174+
uriSchemeTochannelsProviders = new HashMap<>();
175+
uriSchemeTochannelsProviders.put(uriScheme,
176+
SeekableChannelsProviderLoader.getInstance().load(uriScheme, specialInstructions));
168177
}
169178

170-
private static URI locationUri(Table table) {
171-
return FileUtils.convertToURI(path(table.location(), table.io()), true);
179+
private SeekableChannelsProvider getChannelsProvider(final String scheme) {
180+
return uriSchemeTochannelsProviders.computeIfAbsent(scheme,
181+
scheme2 -> {
182+
final Object specialInstructions = instructions.dataInstructions()
183+
.orElseGet(() -> dataInstructionsProvider.load(scheme2));
184+
return SeekableChannelsProviderLoader.getInstance().load(scheme2, specialInstructions);
185+
});
172186
}
173187

174-
private static URI dataFileUri(Table table, DataFile dataFile) {
175-
return FileUtils.convertToURI(path(dataFile.path().toString(), table.io()), false);
176-
}
188+
abstract IcebergTableLocationKey keyFromDataFile(
189+
ManifestFile manifestFile,
190+
DataFile dataFile,
191+
URI fileUri,
192+
SeekableChannelsProvider channelsProvider);
177193

178194
@Override
179195
public synchronized void findKeys(@NotNull final Consumer<IcebergTableLocationKey> locationKeyObserver) {
@@ -187,13 +203,8 @@ public synchronized void findKeys(@NotNull final Consumer<IcebergTableLocationKe
187203
IcebergUtils.toStream(reader)
188204
.map(dataFile -> {
189205
final URI fileUri = dataFileUri(table, dataFile);
190-
if (!uriScheme.equals(fileUri.getScheme())) {
191-
throw new TableDataException(String.format(
192-
"%s:%d - multiple URI schemes are not currently supported. uriScheme=%s, " +
193-
"fileUri=%s",
194-
table, snapshot.snapshotId(), uriScheme, fileUri));
195-
}
196-
return keyFromDataFile(manifestFile, dataFile, fileUri);
206+
return keyFromDataFile(manifestFile, dataFile, fileUri,
207+
getChannelsProvider(fileUri.getScheme()));
197208
})
198209
.forEach(locationKeyObserver);
199210
});

extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergFlatLayout.java

+4-2
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@
88
import io.deephaven.iceberg.util.IcebergReadInstructions;
99
import io.deephaven.iceberg.internal.DataInstructionsProviderLoader;
1010
import io.deephaven.iceberg.util.IcebergTableAdapter;
11+
import io.deephaven.util.channel.SeekableChannelsProvider;
1112
import org.apache.iceberg.*;
1213
import org.jetbrains.annotations.NotNull;
1314

@@ -38,7 +39,8 @@ public String toString() {
3839
IcebergTableLocationKey keyFromDataFile(
3940
@NotNull final ManifestFile manifestFile,
4041
@NotNull final DataFile dataFile,
41-
@NotNull final URI fileUri) {
42-
return locationKey(manifestFile, dataFile, fileUri, null);
42+
@NotNull final URI fileUri,
43+
@NotNull final SeekableChannelsProvider channelsProvider) {
44+
return locationKey(manifestFile, dataFile, fileUri, null, channelsProvider);
4345
}
4446
}

0 commit comments

Comments
 (0)