Skip to content

Commit

Permalink
Start to refactor format from CatalogFile
Browse files Browse the repository at this point in the history
  • Loading branch information
cdouglas committed Jan 31, 2025
1 parent 30aa4cf commit 1b8aa02
Show file tree
Hide file tree
Showing 15 changed files with 269 additions and 158 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ enum Strategy {
* Generate a token to replace the InputFile with the specified content.
*
* @param source Invoked to obtain an InputStream for the future output.
* @param howto
* @param howto append/CAS
*/
T prepare(Supplier<InputStream> source, Strategy howto) throws IOException;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.UUID;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.catalog.CatalogTests;
import org.apache.iceberg.io.CASCatalogFormat;
import org.apache.iceberg.io.FileIOCatalog;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -72,7 +73,8 @@ public void before(TestInfo info) {
final S3FileIO io = new S3FileIO(); // () -> s3);
io.initialize(Maps.newHashMap());
final String location = warehouseLocation + "/catalog";
catalog = new FileIOCatalog("test", location, null, io, Maps.newHashMap());
catalog =
new FileIOCatalog("test", location, null, new CASCatalogFormat(), io, Maps.newHashMap());

final Map<String, String> properties = Maps.newHashMap();
properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.UUID;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.catalog.CatalogTransactionTests;
import org.apache.iceberg.io.CASCatalogFormat;
import org.apache.iceberg.io.FileIOCatalog;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -70,7 +71,8 @@ public void before(TestInfo info) {
final S3FileIO io = new S3FileIO(); // () -> s3);
io.initialize(Maps.newHashMap());
final String location = warehouseLocation + "/catalog";
catalog = new FileIOCatalog("test", location, null, io, Maps.newHashMap());
catalog =
new FileIOCatalog("test", location, null, new CASCatalogFormat(), io, Maps.newHashMap());

final Map<String, String> properties = Maps.newHashMap();
properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ public InputFile toInputFile() {
@Override
public CAS prepare(Supplier<InputStream> source, Strategy howto) {
final ADLSChecksum checksum = new ADLSChecksum(howto);
final byte[] buffer = new byte[8192];
try (InputStream in = source.get();
FileChecksumOutputStream chk =
new FileChecksumOutputStream(ByteStreams.nullOutputStream(), checksum)) {
Expand All @@ -119,6 +118,7 @@ public ADLSInputFile writeAtomic(CAS checksum, Supplier<InputStream> source) thr
}
}

@SuppressWarnings("deprecation") // not clear how else to support this
private ADLSInputFile appendDestObj(ADLSChecksum checksum, Supplier<InputStream> source)
throws IOException {
try (InputStream src = source.get()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
import org.apache.iceberg.azure.AzureProperties;
import org.apache.iceberg.catalog.CatalogTests;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.io.CASCatalogFormat;
import org.apache.iceberg.io.CatalogFormat;
import org.apache.iceberg.io.FileIOCatalog;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.junit.jupiter.api.AfterAll;
Expand Down Expand Up @@ -140,7 +142,8 @@ public void before(TestInfo info) throws IOException {
final Map<String, String> properties = Maps.newHashMap();
properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation);
final String location = warehouseLocation + "/catalog";
catalog = new FileIOCatalog("test", location, null, io, Maps.newHashMap());
CatalogFormat format = new CASCatalogFormat();
catalog = new FileIOCatalog("test", location, null, format, io, Maps.newHashMap());
catalog.initialize(testName, properties);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.azure.AzureProperties;
import org.apache.iceberg.catalog.CatalogTransactionTests;
import org.apache.iceberg.io.CASCatalogFormat;
import org.apache.iceberg.io.FileIOCatalog;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.junit.jupiter.api.AfterAll;
Expand Down Expand Up @@ -106,7 +107,8 @@ public void before(TestInfo info) {
final Map<String, String> properties = Maps.newHashMap();
properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation);
final String location = warehouseLocation + "/catalog";
catalog = new FileIOCatalog("test", location, null, io, Maps.newHashMap());
catalog =
new FileIOCatalog("test", location, null, new CASCatalogFormat(), io, Maps.newHashMap());
catalog.initialize(testName, properties);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import static org.mockito.Mockito.when;

import com.azure.core.exception.UnexpectedLengthException;
import com.azure.core.http.HttpHeader;
import com.azure.core.http.rest.PagedIterable;
import com.azure.core.http.rest.Response;
import com.azure.core.util.Context;
Expand All @@ -47,17 +46,14 @@
import com.azure.storage.file.datalake.models.PathHttpHeaders;
import com.azure.storage.file.datalake.models.PathInfo;
import com.azure.storage.file.datalake.models.PathItem;
import com.azure.storage.file.datalake.options.DataLakeFileAppendOptions;
import com.azure.storage.file.datalake.options.DataLakeFileFlushOptions;
import com.azure.storage.file.datalake.options.FileParallelUploadOptions;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.OffsetDateTime;
import java.util.Iterator;
import java.util.Map;
Expand Down Expand Up @@ -313,41 +309,45 @@ private byte[] randBytes(int len) {
return bytes;
}

private static Response<PathInfo> writeBytes(DataLakeFileClient client, byte[] bytes, DataLakeRequestConditions cond) {
@SuppressWarnings("deprecation") // how else?
private static Response<PathInfo> writeBytes(
DataLakeFileClient client, byte[] bytes, DataLakeRequestConditions cond) {
final FileChecksum chk = new ADLSChecksum(AtomicOutputFile.Strategy.CAS);
chk.update(bytes, 0, bytes.length);
final Response<PathInfo> resp =
client.uploadWithResponse(
new FileParallelUploadOptions(new ByteArrayInputStream(bytes), bytes.length)
.setRequestConditions(cond)
.setHeaders(new PathHttpHeaders()
.setContentMd5(chk.contentChecksumBytes())
.setContentType("binary")),
null, // no timeout
Context.NONE);
return resp;
// no timeout
return client.uploadWithResponse(
new FileParallelUploadOptions(new ByteArrayInputStream(bytes), bytes.length)
.setRequestConditions(cond)
.setHeaders(
new PathHttpHeaders()
.setContentMd5(chk.contentChecksumBytes())
.setContentType("binary")),
null, // no timeout
Context.NONE);
}

private static Response<PathInfo> appendBytes(DataLakeFileClient client, long origLen, byte[] bytes, DataLakeRequestConditions cond) {
private static Response<PathInfo> appendBytes(
DataLakeFileClient client, long origLen, byte[] bytes, DataLakeRequestConditions cond) {
final FileChecksum chk = new ADLSChecksum(AtomicOutputFile.Strategy.CAS);
chk.update(bytes, 0, bytes.length);
client.appendWithResponse(
new ByteArrayInputStream(bytes),
origLen, // client.getProperties().getFileSize(),
bytes.length,
chk.contentChecksumBytes(),
null,
null,
Context.NONE);
final DataLakeFileFlushOptions flushOpts = new DataLakeFileFlushOptions()
new ByteArrayInputStream(bytes),
origLen, // client.getProperties().getFileSize(),
bytes.length,
chk.contentChecksumBytes(),
null,
null,
Context.NONE);
final DataLakeFileFlushOptions flushOpts =
new DataLakeFileFlushOptions()
.setClose(true)
.setRequestConditions(cond)
.setUncommittedDataRetained(false);
return client.flushWithResponse(origLen + bytes.length,
flushOpts, null, Context.NONE);
return client.flushWithResponse(origLen + bytes.length, flushOpts, null, Context.NONE);
}

@Test
@SuppressWarnings("deprecation") // getHeaderValue; InvalidFlush not in SDK
public void scratchADLS() {
// ADLSFileIO io = createFileIO();
// ADLSLocation loc = new ADLSLocation(AZURITE_CONTAINER.location("path/to/file.txt"));
Expand All @@ -365,35 +365,68 @@ public void scratchADLS() {
DataLakeFileClient client = dirClient.getFileClient(objId);

final byte[] originalBytes = randBytes(1024 + random.nextInt(1024));
final Response<PathInfo> origResp = writeBytes(client, originalBytes, new DataLakeRequestConditions());
System.out.println("DEBUG0: " + client.getProperties().getFileSize() + " + " + origResp.getValue().getETag());
final Response<PathInfo> origResp =
writeBytes(client, originalBytes, new DataLakeRequestConditions());
System.out.println(
"DEBUG0: " + client.getProperties().getFileSize() + " + " + origResp.getValue().getETag());

final byte[] overBytes = randBytes(1024 + random.nextInt(1024));
final Response<PathInfo> overResp = writeBytes(client, overBytes, new DataLakeRequestConditions().setIfMatch(origResp.getValue().getETag()));
System.out.println("DEBUG1: " + client.getProperties().getFileSize() + " + " + overResp.getValue().getETag());

final Response<PathInfo> overResp =
writeBytes(
client,
overBytes,
new DataLakeRequestConditions().setIfMatch(origResp.getValue().getETag()));
System.out.println(
"DEBUG1: " + client.getProperties().getFileSize() + " + " + overResp.getValue().getETag());

// TODO update length to be known offset + length of new data
final byte[] appendBytes = randBytes(1024 + random.nextInt(1024));
final Response<PathInfo> appendResp = appendBytes(client, overBytes.length, appendBytes, new DataLakeRequestConditions().setIfMatch(overResp.getValue().getETag()));
System.out.println("DEBUG2: " + client.getProperties().getFileSize() + " + " + appendResp.getValue().getETag());
final Response<PathInfo> appendResp =
appendBytes(
client,
overBytes.length,
appendBytes,
new DataLakeRequestConditions().setIfMatch(overResp.getValue().getETag()));
System.out.println(
"DEBUG2: "
+ client.getProperties().getFileSize()
+ " + "
+ appendResp.getValue().getETag());

final byte[] appendBytes2 = randBytes(1024 + random.nextInt(1024));
// fail, off by 1
final Response<PathInfo> appendResp2;
try {
// final Response<PathInfo> appendResp2 = appendBytes(client, overBytes.length + appendBytes.length - 1, appendBytes2, new DataLakeRequestConditions().setIfMatch(appendResp.getValue().getETag()));
appendResp2 = appendBytes(client, overBytes.length + appendBytes.length - 1, appendBytes2, new DataLakeRequestConditions().setIfMatch(appendResp.getValue().getETag()));
System.out.println("DEBUG3: " + client.getProperties().getFileSize() + " + " + appendResp2.getValue().getETag());
// final Response<PathInfo> appendResp2 = appendBytes(client, overBytes.length +
// appendBytes.length - 1, appendBytes2, new
// DataLakeRequestConditions().setIfMatch(appendResp.getValue().getETag()));
appendResp2 =
appendBytes(
client,
overBytes.length + appendBytes.length - 1,
appendBytes2,
new DataLakeRequestConditions().setIfMatch(appendResp.getValue().getETag()));
System.out.println(
"DEBUG3: "
+ client.getProperties().getFileSize()
+ " + "
+ appendResp2.getValue().getETag());
} catch (DataLakeStorageException e) {
assertThat(e.getResponse().getHeaderValue("x-ms-error-code")).isEqualTo("x-ms-error-code:InvalidFlushPosition");
assertThat(e.getResponse().getHeaderValue("x-ms-error-code"))
.isEqualTo("InvalidFlushPosition");
}

// final byte[] overBytes2 = randBytes(1024 + random.nextInt(1024));
// final Response<PathInfo> overResp2 = writeBytes(client, overBytes2, new DataLakeRequestConditions().setIfMatch(appendResp2.getValue().getETag()));
// System.out.println("DEBUG4: " + client.getProperties().getFileSize() + " + " + overResp2.getValue().getETag());
// final Response<PathInfo> overResp2 = writeBytes(client, overBytes2, new
// DataLakeRequestConditions().setIfMatch(appendResp2.getValue().getETag()));
// System.out.println("DEBUG4: " + client.getProperties().getFileSize() + " + " +
// overResp2.getValue().getETag());

// final byte[] failBytes = randBytes(1024 + random.nextInt(1024));
// final Response<PathInfo> failResp = writeBytes(client, failBytes, new DataLakeRequestConditions().setIfMatch(origResp.getValue().getETag()));
// System.out.println("DEBUG3: " + client.getProperties().getFileSize() + " + " + failResp.getValue().getETag());
// final Response<PathInfo> failResp = writeBytes(client, failBytes, new
// DataLakeRequestConditions().setIfMatch(origResp.getValue().getETag()));
// System.out.println("DEBUG3: " + client.getProperties().getFileSize() + " + " +
// failResp.getValue().getETag());
}

@Test
Expand Down
112 changes: 112 additions & 0 deletions core/src/main/java/org/apache/iceberg/io/CASCatalogFormat.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.iceberg.io;

import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.UncheckedIOException;
import java.util.Collections;
import java.util.Map;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;

public class CASCatalogFormat extends CatalogFormat {

@Override
public CatalogFile.MutCatalogFile empty() {
// TODO record format
return new CatalogFile.MutCatalogFile();
}

@Override
public CatalogFile.MutCatalogFile from(CatalogFile other) {
// TODO
return null;
}

@Override
public CatalogFile read(InputFile catalogLocation) {
final Map<TableIdentifier, CatalogFile.TableInfo> fqti = Maps.newHashMap();
final Map<Namespace, Map<String, String>> namespaces = Maps.newHashMap();
try (InputStream in = catalogLocation.newStream();
DataInputStream din = new DataInputStream(in)) {
int nNamespaces = din.readInt();
for (int i = 0; i < nNamespaces; ++i) {
Namespace namespace = readNamespace(din);
Map<String, String> props = readProperties(din);
namespaces.put(namespace, props);
}
int nTables = din.readInt();
for (int i = 0; i < nTables; i++) {
int tableVersion = din.readInt();
Namespace namespace = readNamespace(din);
TableIdentifier tid = TableIdentifier.of(namespace, din.readUTF());
fqti.put(tid, new CatalogFile.TableInfo(tableVersion, din.readUTF()));
}
int seqno = din.readInt();
long msb = din.readLong();
long lsb = din.readLong();
return new CatalogFile(new UUID(msb, lsb), seqno, namespaces, fqti, catalogLocation);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}

private static Map<String, String> readProperties(DataInputStream in) throws IOException {
int nprops = in.readInt();
Map<String, String> props = nprops > 0 ? Maps.newHashMap() : Collections.emptyMap();
for (int j = 0; j < nprops; j++) {
props.put(in.readUTF(), in.readUTF());
}
return props;
}

private static Namespace readNamespace(DataInputStream in) throws IOException {
int nlen = in.readInt();
String[] levels = new String[nlen];
for (int j = 0; j < nlen; j++) {
levels[j] = in.readUTF();
}
return Namespace.of(levels);
}

static void writeNamespace(DataOutputStream out, Namespace namespace) throws IOException {
out.writeInt(namespace.length());
for (String n : namespace.levels()) {
out.writeUTF(n);
}
}

static void writeProperties(DataOutputStream out, Map<String, String> props) throws IOException {
Map<String, String> writeProps =
props.entrySet().stream()
.filter(e -> e.getValue() != null)
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
out.writeInt(writeProps.size());
for (Map.Entry<String, String> p : writeProps.entrySet()) {
out.writeUTF(p.getKey());
out.writeUTF(p.getValue());
}
}
}
Loading

0 comments on commit 1b8aa02

Please sign in to comment.