You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "Fokko (via GitHub)" <gi...@apache.org> on 2023/03/08 15:04:35 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #7044: Flink: Add REST catalog shorthand

Fokko opened a new pull request, #7044:
URL: https://github.com/apache/iceberg/pull/7044

   Currently I'm using the REST Catalog using a `catalog-impl`:
   
   ```sql
   CREATE CATALOG my_catalog WITH (
     'type'='iceberg',
     'catalog-impl'='org.apache.iceberg.rest.RESTCatalog',
     'uri'='http://localhost:8181',
     's3.access-key-id'='admin',
     's3.secret-access-key'='password',
     's3.endpoint'='http://localhost:9000'
   );
   ```
   
   But it would be nicer to have the `catalog-type` shorthand:
   
   ```sql
   CREATE CATALOG my_catalog WITH (
     'type'='iceberg',
     'catalog-type'='rest',
     'uri'='http://localhost:8181',
     's3.access-key-id'='admin',
     's3.secret-access-key'='password',
     's3.endpoint'='http://localhost:9000'
   );
   ```
   
   This PR adds this


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on pull request #7044: Flink: Add REST catalog shorthand

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#issuecomment-1460941320

   Thanks @stevenzwu, @nastra, and @rdblue for the prompt review!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129591127


##########
docs/flink-getting-started.md:
##########
@@ -260,6 +260,24 @@ The following properties can be set if using the Hadoop catalog:
 
 We could execute the sql command `USE CATALOG hive_catalog` to set the current catalog.
 
+### REST catalog
+
+This creates an iceberg catalog named `rest_catalog` that can be configured using `'catalog-type'='rest'`, which loads tables from a REST catalog:
+
+```sql
+CREATE CATALOG rest_catalog WITH (
+  'type'='iceberg',
+  'catalog-type'='rest',
+  'uri'='https://localhost/'
+);
+```
+
+The following properties can be set if using the REST catalog:
+
+* `uri`: The URL to the REST Catalog (Required)
+* `credential`: A credential to authenticate against the REST catalog (Optional)
+* `token`: A token to interact with the REST catalog (Optional)

Review Comment:
   ```suggestion
   * `token`: A token which will be used to interact with the server (Optional)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on pull request #7044: Flink: Add REST catalog shorthand

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#issuecomment-1461044224

   @Fokko when you got time, can you also backport it to 1.14 and 1.15?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "stevenzwu (via GitHub)" <gi...@apache.org>.
stevenzwu commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129876803


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test for {@link CatalogLoader}. */
+public class TestCatalogLoader extends FlinkTestBase {
+
+  private static File warehouse = null;
+  private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table");
+  private static final Schema SCHEMA =
+      new Schema(Types.NestedField.required(1, "f1", Types.StringType.get()));
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+    hiveConf.set("my_key", "my_value");
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Test
+  public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException {
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse);
+    CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties);
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testHiveCatalogLoader() throws IOException, ClassNotFoundException {
+    CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap());
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testRESTCatalogLoader() throws IOException, ClassNotFoundException {
+    CatalogLoader loader = CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap());
+    validateCatalogLoader(loader);
+  }
+
+  private static void validateCatalogLoader(CatalogLoader loader)
+      throws IOException, ClassNotFoundException {
+    Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA);
+    validateHadoopConf(table);
+  }
+
+  private static void validateHadoopConf(Table table) {
+    FileIO io = table.io();
+    Assertions.assertThat(io)
+        .as("FileIO should be a HadoopFileIO")
+        .isInstanceOf(HadoopFileIO.class);
+    HadoopFileIO hadoopIO = (HadoopFileIO) io;
+    Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException {

Review Comment:
   nit: maybe just `javaSerdes` is enough as method name?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "nastra (via GitHub)" <gi...@apache.org>.
nastra commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129589332


##########
docs/flink-getting-started.md:
##########
@@ -260,6 +260,24 @@ The following properties can be set if using the Hadoop catalog:
 
 We could execute the sql command `USE CATALOG hive_catalog` to set the current catalog.
 
+### REST catalog
+
+This creates an iceberg catalog named `rest_catalog` that can be configured using `'catalog-type'='rest'`, which loads tables from a REST catalog:
+
+```sql
+CREATE CATALOG rest_catalog WITH (
+  'type'='iceberg',
+  'catalog-type'='rest',
+  'uri'='https://localhost/'
+);
+```
+
+The following properties can be set if using the REST catalog:
+
+* `uri`: The URL to the REST Catalog (Required)
+* `credential`: A credential to authenticate against the REST catalog (Optional)

Review Comment:
   ```suggestion
   * `credential`: A credential to exchange for a token in the OAuth2 client credentials flow (Optional)
   ```



##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test for {@link CatalogLoader}. */
+public class TestCatalogLoader extends FlinkTestBase {
+
+  private static File warehouse = null;
+  private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table");
+  private static final Schema SCHEMA =
+      new Schema(Types.NestedField.required(1, "f1", Types.StringType.get()));
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+    hiveConf.set("my_key", "my_value");
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Test
+  public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException {
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse);
+    CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties);
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testHiveCatalogLoader() throws IOException, ClassNotFoundException {

Review Comment:
   it's probably ok to move those tests from `TestCatalogTableLoader` to `TestCatalogLoader`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129950182


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test for {@link CatalogLoader}. */
+public class TestCatalogLoader extends FlinkTestBase {
+
+  private static File warehouse = null;
+  private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table");
+  private static final Schema SCHEMA =
+      new Schema(Types.NestedField.required(1, "f1", Types.StringType.get()));
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+    hiveConf.set("my_key", "my_value");
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Test
+  public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException {
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse);
+    CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties);
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testHiveCatalogLoader() throws IOException, ClassNotFoundException {
+    CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap());
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testRESTCatalogLoader() throws IOException, ClassNotFoundException {
+    CatalogLoader loader = CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap());
+    validateCatalogLoader(loader);
+  }
+
+  private static void validateCatalogLoader(CatalogLoader loader)
+      throws IOException, ClassNotFoundException {
+    Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA);
+    validateHadoopConf(table);
+  }
+
+  private static void validateHadoopConf(Table table) {
+    FileIO io = table.io();
+    Assertions.assertThat(io)
+        .as("FileIO should be a HadoopFileIO")
+        .isInstanceOf(HadoopFileIO.class);
+    HadoopFileIO hadoopIO = (HadoopFileIO) io;
+    Assert.assertEquals("my_value", hadoopIO.conf().get("my_key"));
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException {

Review Comment:
   Good one, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #7044: Flink: Add REST catalog shorthand

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#issuecomment-1460676075

   Looks like there are failing tests.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129948921


##########
docs/flink-getting-started.md:
##########
@@ -260,6 +260,24 @@ The following properties can be set if using the Hadoop catalog:
 
 We could execute the sql command `USE CATALOG hive_catalog` to set the current catalog.
 
+### REST catalog
+
+This creates an iceberg catalog named `rest_catalog` that can be configured using `'catalog-type'='rest'`, which loads tables from a REST catalog:
+
+```sql
+CREATE CATALOG rest_catalog WITH (
+  'type'='iceberg',
+  'catalog-type'='rest',
+  'uri'='https://localhost/'
+);
+```
+
+The following properties can be set if using the REST catalog:
+
+* `uri`: The URL to the REST Catalog (Required)
+* `credential`: A credential to authenticate against the REST catalog (Optional)

Review Comment:
   I like it, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko merged pull request #7044: Flink: Add REST catalog shorthand

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko merged PR #7044:
URL: https://github.com/apache/iceberg/pull/7044


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Fokko commented on a diff in pull request #7044: Flink: Add REST catalog shorthand

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on code in PR #7044:
URL: https://github.com/apache/iceberg/pull/7044#discussion_r1129949289


##########
flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.flink;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Map;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.assertj.core.api.Assertions;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/** Test for {@link CatalogLoader}. */
+public class TestCatalogLoader extends FlinkTestBase {
+
+  private static File warehouse = null;
+  private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table");
+  private static final Schema SCHEMA =
+      new Schema(Types.NestedField.required(1, "f1", Types.StringType.get()));
+
+  @BeforeClass
+  public static void createWarehouse() throws IOException {
+    warehouse = File.createTempFile("warehouse", null);
+    Assert.assertTrue(warehouse.delete());
+    hiveConf.set("my_key", "my_value");
+  }
+
+  @AfterClass
+  public static void dropWarehouse() throws IOException {
+    if (warehouse != null && warehouse.exists()) {
+      Path warehousePath = new Path(warehouse.getAbsolutePath());
+      FileSystem fs = warehousePath.getFileSystem(hiveConf);
+      Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true));
+    }
+  }
+
+  @Test
+  public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException {
+    Map<String, String> properties = Maps.newHashMap();
+    properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse);
+    CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties);
+    validateCatalogLoader(loader);
+  }
+
+  @Test
+  public void testHiveCatalogLoader() throws IOException, ClassNotFoundException {

Review Comment:
   I like the 1:1 mapping of classes and their tests



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org