Skip to content

Commit 87e0531

Browse files
committed
return iceberg metadata fields
1 parent 485f5b8 commit 87e0531

File tree

2 files changed

+26
-3
lines changed

2 files changed

+26
-3
lines changed

xtable-service/README.md

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,11 +24,14 @@ See XTable's `spec` module for more details: https://github.com/apache/incubator
2424

2525
## How to run the service locally
2626

27+
#### Before running the service, ensure that you have the required credentials set in your enviroment needed to read and write to cloud storage.
28+
2729
To run the service locally, first ensure you have built the project with
2830
```sh
2931
mvn clean install -DskipTests
3032
```
3133

34+
3235
Then you can run start the quarkus service using the following command:
3336
```sh
3437
mvn quarkus:dev -pl xtable-service

xtable-service/src/main/java/org/apache/xtable/service/ConversionService.java

Lines changed: 23 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,10 +26,17 @@
2626
import java.util.Collections;
2727
import java.util.List;
2828

29+
import org.apache.commons.lang3.tuple.Pair;
30+
import org.apache.hadoop.conf.Configuration;
2931
import org.apache.hudi.common.table.timeline.HoodieInstant;
3032

33+
import org.apache.iceberg.BaseTable;
3134
import org.apache.iceberg.Snapshot;
3235

36+
import org.apache.iceberg.Table;
37+
import org.apache.iceberg.TableMetadata;
38+
import org.apache.iceberg.TableOperations;
39+
import org.apache.iceberg.hadoop.HadoopTables;
3340
import org.apache.xtable.conversion.ConversionConfig;
3441
import org.apache.xtable.conversion.ConversionController;
3542
import org.apache.xtable.conversion.ConversionSourceProvider;
@@ -79,9 +86,13 @@ public ConvertTableResponse runSync(ConvertTableRequest request) {
7986
getConversionSourceProvider(request.getSourceFormat());
8087
conversionController.sync(conversionConfig, conversionSourceProvider);
8188

82-
InternalTable internalTable = new InternalTable("ICEBERG", request.getSourceTablePath() + "/metadata", null);
83-
ConvertTableResponse response = new ConvertTableResponse(Collections.singletonList(internalTable));
84-
return response;
89+
Pair<String, String> responseFields = getIcebergSchemaAndMetadataPath(request.getSourceTablePath(), sparkHolder.jsc().hadoopConfiguration());
90+
91+
InternalTable internalTable =
92+
new InternalTable(
93+
"ICEBERG",
94+
responseFields.getLeft(), responseFields.getRight());
95+
return new ConvertTableResponse(Collections.singletonList(internalTable));
8596
}
8697

8798
private ConversionSourceProvider<?> getConversionSourceProvider(String sourceTableFormat) {
@@ -104,4 +115,13 @@ private ConversionSourceProvider<?> getConversionSourceProvider(String sourceTab
104115
throw new IllegalArgumentException("Unsupported source format: " + sourceTableFormat);
105116
}
106117
}
118+
119+
public static Pair<String, String> getIcebergSchemaAndMetadataPath(String tableLocation, Configuration conf) {
120+
HadoopTables tables = new HadoopTables(conf);
121+
Table table = tables.load(tableLocation);
122+
TableOperations ops = ((BaseTable) table).operations();
123+
TableMetadata current = ops.current();
124+
return Pair.of(current.metadataFileLocation(), current.schema().toString());
125+
}
126+
107127
}

0 commit comments

Comments
 (0)