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
2 changes: 1 addition & 1 deletion .github/trigger_files/beam_PostCommit_SQL.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run ",
"modification": 3
"modification": 0
}
2 changes: 1 addition & 1 deletion .github/trigger_files/beam_PreCommit_SQL.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 3
"modification": 0
}
2 changes: 2 additions & 0 deletions sdks/java/extensions/sql/iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ dependencies {
implementation project(":sdks:java:core")
implementation project(":sdks:java:managed")
implementation project(":sdks:java:io:iceberg")
implementation library.java.jackson_databind
implementation library.java.jackson_core
runtimeOnly project(":sdks:java:io:iceberg:bqms")
runtimeOnly project(":sdks:java:io:iceberg:hive")
// TODO(https://github.com/apache/beam/issues/21156): Determine how to build without this dependency
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.beam.sdk.extensions.sql.meta.provider.iceberg;

import java.util.Collection;
import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.extensions.sql.meta.provider.AlterTableOps;
import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig.IcebergTableInfo;
import org.apache.beam.sdk.schemas.Schema;

/** {@link AlterTableOps} for Iceberg tables. */
public class IcebergAlterTableOps implements AlterTableOps {
private final IcebergTableInfo table;

IcebergAlterTableOps(IcebergTableInfo table) {
this.table = table;
}

@Override
public void updateTableProperties(Map<String, String> setProps, List<String> resetProps) {
table.updateTableProps(setProps, resetProps);
}

@Override
public void updateSchema(List<Schema.Field> columnsToAdd, Collection<String> columnsToDrop) {
table.updateSchema(columnsToAdd, columnsToDrop);
}

@Override
public void updatePartitionSpec(
List<String> partitionsToAdd, Collection<String> partitionsToDrop) {
table.updatePartitionSpec(partitionsToAdd, partitionsToDrop);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,28 +32,11 @@ public class IcebergCatalog extends InMemoryCatalog {
// other SDKs can make use of it too
private static final String BEAM_HADOOP_PREFIX = "beam.catalog.hadoop";
private final Map<String, IcebergMetastore> metaStores = new HashMap<>();
@VisibleForTesting final IcebergCatalogConfig catalogConfig;
@VisibleForTesting IcebergCatalogConfig catalogConfig;

public IcebergCatalog(String name, Map<String, String> properties) {
super(name, properties);

ImmutableMap.Builder<String, String> catalogProps = ImmutableMap.builder();
ImmutableMap.Builder<String, String> hadoopProps = ImmutableMap.builder();

for (Map.Entry<String, String> entry : properties.entrySet()) {
if (entry.getKey().startsWith(BEAM_HADOOP_PREFIX)) {
hadoopProps.put(entry.getKey(), entry.getValue());
} else {
catalogProps.put(entry.getKey(), entry.getValue());
}
}

catalogConfig =
IcebergCatalogConfig.builder()
.setCatalogName(name)
.setCatalogProperties(catalogProps.build())
.setConfigProperties(hadoopProps.build())
.build();
public IcebergCatalog(String name, Map<String, String> props) {
super(name, props);
catalogConfig = initConfig(name, props);
}

@Override
Expand All @@ -67,6 +50,12 @@ public String type() {
return "iceberg";
}

@Override
public void updateProperties(Map<String, String> setProps, Collection<String> resetProps) {
super.updateProperties(setProps, resetProps);
catalogConfig = initConfig(name(), properties());
}

@Override
public boolean createDatabase(String database) {
return catalogConfig.createNamespace(database);
Expand Down Expand Up @@ -97,4 +86,23 @@ public boolean dropDatabase(String database, boolean cascade) {
}
return removed;
}

private static IcebergCatalogConfig initConfig(String name, Map<String, String> properties) {
ImmutableMap.Builder<String, String> catalogProps = ImmutableMap.builder();
ImmutableMap.Builder<String, String> hadoopProps = ImmutableMap.builder();

for (Map.Entry<String, String> entry : properties.entrySet()) {
if (entry.getKey().startsWith(BEAM_HADOOP_PREFIX)) {
hadoopProps.put(entry.getKey(), entry.getValue());
} else {
catalogProps.put(entry.getKey(), entry.getValue());
}
}

return IcebergCatalogConfig.builder()
.setCatalogName(name)
.setCatalogProperties(catalogProps.build())
.setConfigProperties(hadoopProps.build())
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;

import com.fasterxml.jackson.core.type.TypeReference;
import java.util.HashMap;
import java.util.Map;
import org.apache.beam.sdk.extensions.sql.TableUtils;
Expand Down Expand Up @@ -60,7 +61,12 @@ public void createTable(Table table) {
} else {
String identifier = getIdentifier(table);
try {
catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields());
Map<String, String> properties =
TableUtils.getObjectMapper()
.convertValue(table.getProperties(), new TypeReference<Map<String, String>>() {});
;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

There's a redundant semicolon at the end of this line.

                .convertValue(table.getProperties(), new TypeReference<Map<String, String>>() {});

catalogConfig.createTable(
identifier, table.getSchema(), table.getPartitionFields(), properties);
} catch (TableAlreadyExistsException e) {
LOG.info(
"Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier);
Expand Down Expand Up @@ -147,6 +153,15 @@ public boolean supportsPartitioning(Table table) {
return getProvider(table.getType()).supportsPartitioning(table);
}

@Override
public IcebergAlterTableOps alterTable(String name) {
IcebergTableInfo table =
checkStateNotNull(
catalogConfig.loadTable(getIdentifier(name)), "Could not find table '%s'", name);

return new IcebergAlterTableOps(table);
}

@Override
public void registerProvider(TableProvider provider) {
super.registerProvider(provider);
Expand Down
Loading
Loading