You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mb...@apache.org on 2021/10/08 10:22:00 UTC

[hive] branch master updated: HIVE-25604: Iceberg should implement the authorization storage handler (Marton Bod, reviewed by Peter Vary)

This is an automated email from the ASF dual-hosted git repository.

mbod pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 38771b1  HIVE-25604: Iceberg should implement the authorization storage handler (Marton Bod, reviewed by Peter Vary)
38771b1 is described below

commit 38771b18af70ae1ac71f4a88f64a54d099625173
Author: Marton Bod <ma...@gmail.com>
AuthorDate: Fri Oct 8 12:21:43 2021 +0200

    HIVE-25604: Iceberg should implement the authorization storage handler (Marton Bod, reviewed by Peter Vary)
    
    Iceberg's StorageHandler should implement the HiveStorageAuthorizationHandler interface for authorization purposes. We'll use the iceberg table root location as the basis for permission handling.
---
 .../iceberg/mr/hive/HiveIcebergStorageHandler.java | 13 ++++++-
 .../mr/hive/TestHiveIcebergStorageHandler.java     | 44 ++++++++++++++++++++++
 2 files changed, 56 insertions(+), 1 deletion(-)

diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
index 7a357af..a4392b4 100644
--- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
+++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
@@ -21,6 +21,8 @@ package org.apache.iceberg.mr.hive;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -43,6 +45,7 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveStorageAuthorizationHandler;
 import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler;
 import org.apache.hadoop.hive.ql.metadata.HiveStoragePredicateHandler;
 import org.apache.hadoop.hive.ql.parse.PartitionTransformSpec;
@@ -84,9 +87,11 @@ import org.apache.iceberg.util.SerializationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, HiveStorageHandler {
+public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, HiveStorageHandler,
+    HiveStorageAuthorizationHandler {
   private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergStorageHandler.class);
 
+  private static final String ICEBERG_URI_PREFIX = "iceberg://";
   private static final Splitter TABLE_NAME_SPLITTER = Splitter.on("..");
   private static final String TABLE_NAME_SEPARATOR = "..";
 
@@ -360,6 +365,12 @@ public class HiveIcebergStorageHandler implements HiveStoragePredicateHandler, H
     return IcebergMetadataTables.isValidMetaTable(metaTableName);
   }
 
+  @Override
+  public URI getURIForAuth(Map<String, String> tableProperties) throws URISyntaxException {
+    String tableLocation = tableProperties.get(Catalogs.LOCATION);
+    return new URI(ICEBERG_URI_PREFIX + tableLocation);
+  }
+
   private void setCommonJobConf(JobConf jobConf) {
     jobConf.set("tez.mrreader.config.update.properties", "hive.io.file.readcolumn.names,hive.io.file.readcolumn.ids");
   }
diff --git a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java
new file mode 100644
index 0000000..3d9c5ce
--- /dev/null
+++ b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandler.java
@@ -0,0 +1,44 @@
+/*
+ * 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.mr.hive;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestHiveIcebergStorageHandler {
+
+  @Test
+  public void testAuthzURI() throws URISyntaxException {
+    Map<String, String> props = ImmutableMap.of(
+        Catalogs.LOCATION, "hdfs://abcd/汉字123/"
+    );
+
+    HiveIcebergStorageHandler storageHandler = new HiveIcebergStorageHandler();
+    URI uriForAuth = storageHandler.getURIForAuth(props);
+
+    Assert.assertEquals("iceberg://hdfs://abcd/汉字123/", uriForAuth.toString());
+  }
+
+}