Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.beam.sdk.extensions.sql.TableUtils;
import org.apache.beam.sdk.extensions.sql.impl.TableName;
import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
Expand Down Expand Up @@ -60,12 +61,14 @@ public void createTable(Table table) {
getProvider(table.getType()).createTable(table);
} else {
String identifier = getIdentifier(table);
Map<String, String> props =
TableUtils.getObjectMapper()
.convertValue(table.getProperties(), new TypeReference<Map<String, String>>() {})
.entrySet().stream()
.filter(p -> !p.getKey().startsWith("beam."))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
try {
Map<String, String> properties =
TableUtils.getObjectMapper()
.convertValue(table.getProperties(), new TypeReference<Map<String, String>>() {});
catalogConfig.createTable(
identifier, table.getSchema(), table.getPartitionFields(), properties);
catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields(), props);
} catch (TableAlreadyExistsException e) {
LOG.info(
"Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import java.util.HashMap;
import java.util.List;
Expand Down Expand Up @@ -56,6 +57,8 @@ class IcebergTable extends SchemaBaseBeamTable {
@VisibleForTesting static final String CATALOG_PROPERTIES_FIELD = "catalog_properties";
@VisibleForTesting static final String HADOOP_CONFIG_PROPERTIES_FIELD = "config_properties";
@VisibleForTesting static final String CATALOG_NAME_FIELD = "catalog_name";
static final String BEAM_WRITE_PROPERTY = "beam.write.";
static final String BEAM_READ_PROPERTY = "beam.read.";

@VisibleForTesting
static final String TRIGGERING_FREQUENCY_FIELD = "triggering_frequency_seconds";
Expand All @@ -71,9 +74,21 @@ class IcebergTable extends SchemaBaseBeamTable {
this.tableIdentifier = tableIdentifier;
this.catalogConfig = catalogConfig;
ObjectNode properties = table.getProperties();
if (properties.has(TRIGGERING_FREQUENCY_FIELD)) {
this.triggeringFrequency = properties.get(TRIGGERING_FREQUENCY_FIELD).asInt();
for (Map.Entry<String, JsonNode> property : properties.properties()) {
String name = property.getKey();
if (name.startsWith(BEAM_WRITE_PROPERTY)) {
String prop = name.substring(BEAM_WRITE_PROPERTY.length());
if (prop.equals(TRIGGERING_FREQUENCY_FIELD)) {
this.triggeringFrequency = property.getValue().asInt();
} else {
throw new IllegalArgumentException("Unknown Beam write property: " + name);
}
} else if (name.startsWith(BEAM_READ_PROPERTY)) {
// none supported yet
throw new IllegalArgumentException("Unknown Beam read property: " + name);
}
}

this.partitionFields = table.getPartitionFields();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ public void testSimpleInsertWithPartitionedFields() throws Exception {
+ ") \n"
+ "TYPE 'iceberg' \n"
+ "PARTITIONED BY('id', 'truncate(name, 3)') \n"
+ "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'";
+ "TBLPROPERTIES '{ \"beam.write.triggering_frequency_seconds\" : 10 }'";
String insertStatement =
format("INSERT INTO %s \n", tableIdentifier)
+ "SELECT \n"
Expand Down Expand Up @@ -211,7 +211,7 @@ public void testSimpleInsertFlat() throws Exception {
+ " name VARCHAR \n "
+ ") \n"
+ "TYPE 'iceberg' \n"
+ "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'";
+ "TBLPROPERTIES '{ \"beam.write.triggering_frequency_seconds\" : 10 }'";
String insertStatement =
format("INSERT INTO %s \n", tableIdentifier)
+ "SELECT \n"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ public void createTable(
String tableIdentifier,
Schema tableSchema,
@Nullable List<String> partitionFields,
Map<String, String> properties) {
@Nullable Map<String, String> properties) {
TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier);
org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(tableSchema);
PartitionSpec icebergSpec = PartitionUtils.toPartitionSpec(partitionFields, tableSchema);
Expand All @@ -162,7 +162,11 @@ public void createTable(
icebergIdentifier,
icebergSchema,
icebergSpec);
catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec, properties);
if (properties != null) {
catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec, properties);
} else {
catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec);
}
LOG.info("Successfully created table '{}'.", icebergIdentifier);
} catch (AlreadyExistsException e) {
throw new TableAlreadyExistsException(e);
Expand Down
1 change: 1 addition & 0 deletions website/www/site/assets/js/language-switch-v2.js
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,7 @@ $(document).ready(function() {
}).render();

Switcher({"name": "runner", "default": "direct"}).render();
Switcher({"name": "tab"}).render();
Switcher({"name": "shell", "default": "unix"}).render();
Switcher({"name": "version"}).render();
});
Loading
Loading