Skip to content

Commit 73e55f4

Browse files
authored
Support catalog type property and setting catalog name (#41)
1 parent 06e6cec commit 73e55f4

File tree

10 files changed

+40
-25
lines changed

10 files changed

+40
-25
lines changed

README.md

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ The zip archive will be found under `./kafka-connect-runtime/build/distributions
3333
| iceberg.control.commitIntervalMs | Commit interval in msec, default is 300,000 (5 min) |
3434
| iceberg.control.commitTimeoutMs | Commit timeout interval in msec, default is 30,000 (30 sec) |
3535
| iceberg.control.commitThreads | Number of threads to use for commits, default is (cores * 2) |
36+
| iceberg.catalog | Name of the catalog, default is `iceberg` |
3637
| iceberg.catalog.* | Properties passed through to Iceberg catalog initialization |
3738
| iceberg.kafka.* | Properties passed through to control topic Kafka client initialization |
3839

@@ -59,7 +60,7 @@ otherwise you will need to include that yourself.
5960

6061
### REST example
6162
```
62-
"iceberg.catalog": "org.apache.iceberg.rest.RESTCatalog",
63+
"iceberg.catalog.type": "rest",
6364
"iceberg.catalog.uri": "https://catalog-service",
6465
"iceberg.catalog.credential": "<credential>",
6566
"iceberg.catalog.warehouse": "<warehouse>",
@@ -69,7 +70,7 @@ otherwise you will need to include that yourself.
6970
NOTE: Use the distribution that includes the HMS client (or include the HMS client yourself). Use `S3FileIO` when
7071
using S3 for storage (the default is `HadoopFileIO` with `HiveCatalog`).
7172
```
72-
"iceberg.catalog":"org.apache.iceberg.hive.HiveCatalog",
73+
"iceberg.catalog.tyoe":"hive",
7374
"iceberg.catalog.uri":"thrift://hive:9083",
7475
"iceberg.catalog.io-impl":"org.apache.iceberg.aws.s3.S3FileIO",
7576
"iceberg.catalog.warehouse":"s3a://bucket/warehouse",
@@ -131,7 +132,7 @@ This example config connects to a Iceberg REST catalog.
131132
"tasks.max": "2",
132133
"topics": "events",
133134
"iceberg.tables": "default.events",
134-
"iceberg.catalog": "org.apache.iceberg.rest.RESTCatalog",
135+
"iceberg.catalog.type": "rest",
135136
"iceberg.catalog.uri": "https://localhost",
136137
"iceberg.catalog.credential": "<credential>",
137138
"iceberg.catalog.warehouse": "<warehouse name>"
@@ -173,7 +174,7 @@ PARTITIONED BY (hours(ts));
173174
"iceberg.tables.routeField": "type",
174175
"iceberg.table.default.events_list.routeRegex": "list",
175176
"iceberg.table.default.events_create.routeRegex": "create",
176-
"iceberg.catalog": "org.apache.iceberg.rest.RESTCatalog",
177+
"iceberg.catalog.type": "rest",
177178
"iceberg.catalog.uri": "https://localhost",
178179
"iceberg.catalog.credential": "<credential>",
179180
"iceberg.catalog.warehouse": "<warehouse name>"
@@ -199,7 +200,7 @@ See above for creating two tables.
199200
"topics": "events",
200201
"iceberg.tables.dynamic.enabled": "true",
201202
"iceberg.tables.routeField": "db_table",
202-
"iceberg.catalog": "org.apache.iceberg.rest.RESTCatalog",
203+
"iceberg.catalog.type": "rest",
203204
"iceberg.catalog.uri": "https://localhost",
204205
"iceberg.catalog.credential": "<credential>",
205206
"iceberg.catalog.warehouse": "<warehouse name>"
@@ -226,7 +227,7 @@ See above for creating the table
226227
"topics": "events",
227228
"iceberg.tables": "default.events",
228229
"iceberg.tables.cdcField": "_cdc_op",
229-
"iceberg.catalog": "org.apache.iceberg.rest.RESTCatalog",
230+
"iceberg.catalog.type": "rest",
230231
"iceberg.catalog.uri": "https://localhost",
231232
"iceberg.catalog.credential": "<credential>",
232233
"iceberg.catalog.warehouse": "<warehouse name>"

build.gradle

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@ subprojects {
1313
apply plugin: "maven-publish"
1414

1515
group "io.tabular.connect"
16-
version "0.4.3-SNAPSHOT"
16+
version "0.4.3"
1717

1818
repositories {
1919
mavenCentral()

kafka-connect-runtime/src/test/java/io/tabular/iceberg/connect/IntegrationCdcTest.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import java.util.UUID;
3232
import java.util.concurrent.TimeUnit;
3333
import org.apache.iceberg.CatalogProperties;
34+
import org.apache.iceberg.CatalogUtil;
3435
import org.apache.iceberg.DataFile;
3536
import org.apache.iceberg.DeleteFile;
3637
import org.apache.iceberg.PartitionSpec;
@@ -44,7 +45,6 @@
4445
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
4546
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
4647
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
47-
import org.apache.iceberg.rest.RESTCatalog;
4848
import org.apache.iceberg.types.Types;
4949
import org.awaitility.Awaitility;
5050
import org.junit.jupiter.api.AfterEach;
@@ -105,7 +105,9 @@ public void testIcebergSink() throws Exception {
105105
.config("iceberg.tables.cdcField", "op")
106106
.config("iceberg.control.commitIntervalMs", 1000)
107107
.config("iceberg.control.commitTimeoutMs", Integer.MAX_VALUE)
108-
.config("iceberg.catalog", RESTCatalog.class.getName())
108+
.config(
109+
"iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE,
110+
CatalogUtil.ICEBERG_CATALOG_TYPE_REST)
109111
.config("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:8181")
110112
.config("iceberg.catalog." + S3FileIOProperties.ENDPOINT, "http://minio:9000")
111113
.config("iceberg.catalog." + S3FileIOProperties.ACCESS_KEY_ID, AWS_ACCESS_KEY)

kafka-connect-runtime/src/test/java/io/tabular/iceberg/connect/IntegrationDynamicTableTest.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import java.util.UUID;
3030
import java.util.concurrent.TimeUnit;
3131
import org.apache.iceberg.CatalogProperties;
32+
import org.apache.iceberg.CatalogUtil;
3233
import org.apache.iceberg.DataFile;
3334
import org.apache.iceberg.PartitionSpec;
3435
import org.apache.iceberg.Schema;
@@ -38,7 +39,6 @@
3839
import org.apache.iceberg.catalog.Namespace;
3940
import org.apache.iceberg.catalog.TableIdentifier;
4041
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
41-
import org.apache.iceberg.rest.RESTCatalog;
4242
import org.apache.iceberg.types.Types;
4343
import org.awaitility.Awaitility;
4444
import org.junit.jupiter.api.AfterEach;
@@ -99,7 +99,9 @@ public void testIcebergSink() {
9999
.config("iceberg.tables.routeField", "payload")
100100
.config("iceberg.control.commitIntervalMs", 1000)
101101
.config("iceberg.control.commitTimeoutMs", Integer.MAX_VALUE)
102-
.config("iceberg.catalog", RESTCatalog.class.getName())
102+
.config(
103+
"iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE,
104+
CatalogUtil.ICEBERG_CATALOG_TYPE_REST)
103105
.config("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:8181")
104106
.config("iceberg.catalog." + S3FileIOProperties.ENDPOINT, "http://minio:9000")
105107
.config("iceberg.catalog." + S3FileIOProperties.ACCESS_KEY_ID, AWS_ACCESS_KEY)

kafka-connect-runtime/src/test/java/io/tabular/iceberg/connect/IntegrationMultiTableTest.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import java.util.UUID;
3030
import java.util.concurrent.TimeUnit;
3131
import org.apache.iceberg.CatalogProperties;
32+
import org.apache.iceberg.CatalogUtil;
3233
import org.apache.iceberg.DataFile;
3334
import org.apache.iceberg.PartitionSpec;
3435
import org.apache.iceberg.Schema;
@@ -38,7 +39,6 @@
3839
import org.apache.iceberg.catalog.Namespace;
3940
import org.apache.iceberg.catalog.TableIdentifier;
4041
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
41-
import org.apache.iceberg.rest.RESTCatalog;
4242
import org.apache.iceberg.types.Types;
4343
import org.awaitility.Awaitility;
4444
import org.junit.jupiter.api.AfterEach;
@@ -103,7 +103,9 @@ public void testIcebergSink() {
103103
.config(format("iceberg.table.%s.%s.routeRegex", TEST_DB, TEST_TABLE2), "type2")
104104
.config("iceberg.control.commitIntervalMs", 1000)
105105
.config("iceberg.control.commitTimeoutMs", Integer.MAX_VALUE)
106-
.config("iceberg.catalog", RESTCatalog.class.getName())
106+
.config(
107+
"iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE,
108+
CatalogUtil.ICEBERG_CATALOG_TYPE_REST)
107109
.config("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:8181")
108110
.config("iceberg.catalog." + S3FileIOProperties.ENDPOINT, "http://minio:9000")
109111
.config("iceberg.catalog." + S3FileIOProperties.ACCESS_KEY_ID, AWS_ACCESS_KEY)

kafka-connect-runtime/src/test/java/io/tabular/iceberg/connect/IntegrationTest.java

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import java.util.UUID;
3131
import java.util.concurrent.TimeUnit;
3232
import org.apache.iceberg.CatalogProperties;
33+
import org.apache.iceberg.CatalogUtil;
3334
import org.apache.iceberg.DataFile;
3435
import org.apache.iceberg.PartitionSpec;
3536
import org.apache.iceberg.Schema;
@@ -39,7 +40,6 @@
3940
import org.apache.iceberg.catalog.Namespace;
4041
import org.apache.iceberg.catalog.TableIdentifier;
4142
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
42-
import org.apache.iceberg.rest.RESTCatalog;
4343
import org.apache.iceberg.types.Types;
4444
import org.awaitility.Awaitility;
4545
import org.junit.jupiter.api.AfterEach;
@@ -96,7 +96,9 @@ public void testIcebergSink() throws Exception {
9696
.config("iceberg.tables", format("%s.%s", TEST_DB, TEST_TABLE))
9797
.config("iceberg.control.commitIntervalMs", 1000)
9898
.config("iceberg.control.commitTimeoutMs", Integer.MAX_VALUE)
99-
.config("iceberg.catalog", RESTCatalog.class.getName())
99+
.config(
100+
"iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE,
101+
CatalogUtil.ICEBERG_CATALOG_TYPE_REST)
100102
.config("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:8181")
101103
.config("iceberg.catalog." + S3FileIOProperties.ENDPOINT, "http://minio:9000")
102104
.config("iceberg.catalog." + S3FileIOProperties.ACCESS_KEY_ID, AWS_ACCESS_KEY)

kafka-connect/src/main/java/io/tabular/iceberg/connect/IcebergSinkConfig.java

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ public class IcebergSinkConfig extends AbstractConfig {
5858
private static final String KAFKA_PROP_PREFIX = "iceberg.kafka.";
5959
private static final String TABLE_PROP_PREFIX = "iceberg.table.";
6060

61-
private static final String CATALOG_IMPL_PROP = "iceberg.catalog";
61+
private static final String CATALOG_NAME_PROP = "iceberg.catalog";
6262
private static final String TABLES_PROP = "iceberg.tables";
6363
private static final String TABLES_DYNAMIC_PROP = "iceberg.tables.dynamic.enabled";
6464
private static final String TABLES_ROUTE_FIELD_PROP = "iceberg.tables.routeField";
@@ -75,6 +75,7 @@ public class IcebergSinkConfig extends AbstractConfig {
7575
private static final String NAME_PROP = "name";
7676
private static final String BOOTSTRAP_SERVERS_PROP = "bootstrap.servers";
7777

78+
private static final String DEFAULT_CATALOG_NAME = "iceberg";
7879
private static final String DEFAULT_CONTROL_TOPIC = "control-iceberg";
7980
public static final String DEFAULT_CONTROL_GROUP_PREFIX = "cg-control-";
8081

@@ -125,7 +126,12 @@ private static ConfigDef newConfigDef() {
125126
false,
126127
Importance.MEDIUM,
127128
"Set to true to treat all appends as upserts, false otherwise");
128-
configDef.define(CATALOG_IMPL_PROP, Type.STRING, Importance.HIGH, "Iceberg catalog class name");
129+
configDef.define(
130+
CATALOG_NAME_PROP,
131+
Type.STRING,
132+
DEFAULT_CATALOG_NAME,
133+
Importance.MEDIUM,
134+
"Iceberg catalog name");
129135
configDef.define(
130136
CONTROL_TOPIC_PROP,
131137
Type.STRING,
@@ -186,6 +192,7 @@ public IcebergSinkConfig(Map<String, String> originalProps) {
186192
}
187193

188194
private void validate() {
195+
checkState(!getCatalogProps().isEmpty(), "Must specify Iceberg catalog properties");
189196
if (getTables() != null) {
190197
checkState(!getDynamicTablesEnabled(), "Cannot specify both static and dynamic table names");
191198
} else if (getDynamicTablesEnabled()) {
@@ -223,8 +230,8 @@ public Map<String, String> getKafkaProps() {
223230
return kafkaProps;
224231
}
225232

226-
public String getCatalogImpl() {
227-
return getString(CATALOG_IMPL_PROP);
233+
public String getCatalogName() {
234+
return getString(CATALOG_NAME_PROP);
228235
}
229236

230237
public List<String> getTables() {

kafka-connect/src/main/java/io/tabular/iceberg/connect/data/Utilities.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -48,8 +48,8 @@ public class Utilities {
4848
private static final Logger LOG = LoggerFactory.getLogger(Utilities.class.getName());
4949

5050
public static Catalog loadCatalog(IcebergSinkConfig config) {
51-
return CatalogUtil.loadCatalog(
52-
config.getCatalogImpl(), "iceberg", config.getCatalogProps(), getHadoopConfig());
51+
return CatalogUtil.buildIcebergCatalog(
52+
config.getCatalogName(), config.getCatalogProps(), getHadoopConfig());
5353
}
5454

5555
private static Object getHadoopConfig() {

kafka-connect/src/test/java/io/tabular/iceberg/connect/IcebergSinkConfigTest.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@
2424

2525
import java.util.Map;
2626
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
27-
import org.apache.iceberg.rest.RESTCatalog;
2827
import org.apache.kafka.common.config.ConfigException;
2928
import org.junit.jupiter.api.Test;
3029

@@ -47,7 +46,7 @@ public void testInvalid() {
4746
Map<String, String> props =
4847
ImmutableMap.of(
4948
"topics", "source-topic",
50-
"iceberg.catalog", RESTCatalog.class.getName(),
49+
"iceberg.catalog.type", "rest",
5150
"iceberg.tables", "db.landing",
5251
"iceberg.tables.dynamic.enabled", "true");
5352
assertThatExceptionOfType(ConfigException.class).isThrownBy(() -> new IcebergSinkConfig(props));
@@ -57,7 +56,7 @@ public void testInvalid() {
5756
public void testGetDefault() {
5857
Map<String, String> props =
5958
ImmutableMap.of(
60-
"iceberg.catalog", RESTCatalog.class.getName(),
59+
"iceberg.catalog.type", "rest",
6160
"topics", "source-topic",
6261
"iceberg.tables", "db.landing");
6362
IcebergSinkConfig config = new IcebergSinkConfig(props);

kafka-connect/src/test/java/io/tabular/iceberg/connect/IcebergSinkTaskTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public void testIsLeader() {
3535
task.start(
3636
ImmutableMap.of(
3737
"topics", "topic1, topic2",
38-
"iceberg.catalog", "catalog",
38+
"iceberg.catalog.type", "rest",
3939
"iceberg.tables", "table"));
4040

4141
List<TopicPartition> assignments =

0 commit comments

Comments
 (0)