From f17e72ba535a925a1996c3adbecb7af87a295acb Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Mon, 21 Mar 2016 14:13:48 +0100 Subject: [PATCH 01/20] Update hive-apache-jdbc dependency New version contains `hive-shims` classes within. `hive-shims` is required by presto-product-tests to access kerberized hive. --- pom.xml | 2 +- .../presto/tests/hive/TestHiveStorageFormats.java | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java diff --git a/pom.xml b/pom.xml index 814df1be13bfd..ba9abba26a5d7 100644 --- a/pom.xml +++ b/pom.xml @@ -735,7 +735,7 @@ com.facebook.presto.hive hive-apache-jdbc - 0.13.1-1 + 0.13.1-3 diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java new file mode 100644 index 0000000000000..4ba4109700d15 --- /dev/null +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -0,0 +1,8 @@ +package com.facebook.presto.tests.hive; + +/** + * Created by andrii on 21.03.16. + */ +public class TestHiveStorageFormats +{ +} From c7fdd6b50ba9afe39f7f953caf8d11917f253d4d Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Mon, 21 Mar 2016 14:17:28 +0100 Subject: [PATCH 02/20] Add TestHiveStorageFormats product test Test INSERT and SELECT paths for all the storage formats supported by hive connector. This test is going to be used for kerberized HDFS access verification. Initally all the formats are commented out because no one of them is supported yet. We will uncomment format by format together with the kerberos support implmenetation pathches. So we can better track what changes are needed in order to ensure that some particular format works for Kerberized Hadoop. --- .../com/facebook/presto/tests/TestGroups.java | 1 + .../tests/hive/TestHiveStorageFormats.java | 282 +++++++++++++++++- 2 files changed, 280 insertions(+), 3 deletions(-) diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java index d7ed096b966dd..1bb3f6177b010 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java @@ -42,6 +42,7 @@ public final class TestGroups public static final String REGEX_FUNCTIONS = "regex_functions"; public static final String STRING_FUNCTIONS = "string_functions"; public static final String MATH_FUNCTIONS = "math_functions"; + public static final String STORAGE_FORMATS = "storage_formats"; private TestGroups() {} } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index 4ba4109700d15..12a59d715c850 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -1,8 +1,284 @@ +/* + * Licensed 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 com.facebook.presto.tests.hive; -/** - * Created by andrii on 21.03.16. - */ +import com.facebook.presto.jdbc.PrestoConnection; +import com.google.common.base.MoreObjects; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.teradata.tempto.ProductTest; +import com.teradata.tempto.assertions.QueryAssert.Row; +import com.teradata.tempto.query.QueryResult; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.tests.TestGroups.STORAGE_FORMATS; +import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; +import static com.teradata.tempto.assertions.QueryAssert.Row.row; +import static com.teradata.tempto.assertions.QueryAssert.assertThat; +import static com.teradata.tempto.query.QueryExecutor.defaultQueryExecutor; +import static com.teradata.tempto.query.QueryExecutor.query; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + public class TestHiveStorageFormats + extends ProductTest { + private static final String TPCH_SCHEMA = "tiny"; + + @DataProvider(name = "storage_formats") + public static Object[][] storageFormats() + { + return new StorageFormat[][] { + //{storageFormat("ORC")}, + //{storageFormat("DWRF")}, + //{storageFormat("PARQUET")}, + //{storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, + //{storageFormat("RCBINARY")}, + //{storageFormat("RCTEXT")}, + //{storageFormat("SEQUENCEFILE")}, + //{storageFormat("TEXTFILE")} + }; + } + + @Test(dataProvider = "storage_formats", groups = {STORAGE_FORMATS}) + public void testInsertIntoTable(StorageFormat storageFormat) + { + setSessionProperties(storageFormat); + + String tableName = "storage_formats_test_insert_into_" + storageFormat.getName().toLowerCase(); + + // DROP TABLE + query(format("DROP TABLE IF EXISTS %s", tableName)); + + // CREATE TABLE + String createTable = format( + "CREATE TABLE %s(" + + " orderkey BIGINT," + + " partkey BIGINT," + + " suppkey BIGINT," + + " linenumber INTEGER," + + " quantity DOUBLE," + + " extendedprice DOUBLE," + + " discount DOUBLE," + + " tax DOUBLE," + + " linestatus VARCHAR," + + " shipinstruct VARCHAR," + + " shipmode VARCHAR," + + " comment VARCHAR," + + " returnflag VARCHAR" + + ") WITH (format='%s')", + tableName, + storageFormat.getName()); + query(createTable); + + // INSERT INTO TABLE + String insertInto = format("INSERT INTO %s " + + "SELECT " + + "orderkey, partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, " + + "linestatus, shipinstruct, shipmode, comment, returnflag " + + "FROM tpch.%s.lineitem", tableName, TPCH_SCHEMA); + query(insertInto); + + // SELECT FROM TABLE + assertSelect("select sum(tax), sum(discount), sum(linenumber) from %s", tableName); + + // DROP TABLE + query(format("DROP TABLE %s", tableName)); + } + + @Test(dataProvider = "storage_formats", groups = {STORAGE_FORMATS}) + public void testCreateTableAs(StorageFormat storageFormat) + { + setSessionProperties(storageFormat); + + String tableName = "storage_formats_test_create_table_as_select_" + storageFormat.getName().toLowerCase(); + + // DROP TABLE + query(format("DROP TABLE IF EXISTS %s", tableName)); + + // CREATE TABLE AS SELECT + String createTableAsSelect = format( + "CREATE TABLE %s WITH (format='%s') AS " + + "SELECT " + + "orderkey, partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, " + + "linestatus, shipinstruct, shipmode, comment, returnflag " + + "FROM tpch.%s.lineitem", + tableName, + storageFormat.getName(), + TPCH_SCHEMA); + query(createTableAsSelect); + + // SELECT FROM TABLE + assertSelect("select sum(extendedprice), sum(suppkey), count(partkey) from %s", tableName); + + // DROP TABLE + query(format("DROP TABLE %s", tableName)); + } + + @Test(dataProvider = "storage_formats", groups = {STORAGE_FORMATS}) + public void testInsertIntoPartitionedTable(StorageFormat storageFormat) + { + setSessionProperties(storageFormat); + + String tableName = "storage_formats_test_insert_into_partitioned_" + storageFormat.getName().toLowerCase(); + + // DROP TABLE + query(format("DROP TABLE IF EXISTS %s", tableName)); + + // CREATE TABLE + String createTable = format( + "CREATE TABLE %s(" + + " orderkey BIGINT," + + " partkey BIGINT," + + " suppkey BIGINT," + + " linenumber INTEGER," + + " quantity DOUBLE," + + " extendedprice DOUBLE," + + " discount DOUBLE," + + " tax DOUBLE," + + " linestatus VARCHAR," + + " shipinstruct VARCHAR," + + " shipmode VARCHAR," + + " comment VARCHAR," + + " returnflag VARCHAR" + + ") WITH (format='%s', partitioned_by = ARRAY['returnflag'])", + tableName, + storageFormat.getName()); + query(createTable); + + // INSERT INTO TABLE + String insertInto = format("INSERT INTO %s " + + "SELECT " + + "orderkey, partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, " + + "linestatus, shipinstruct, shipmode, comment, returnflag " + + "FROM tpch.%s.lineitem", tableName, TPCH_SCHEMA); + query(insertInto); + + // SELECT FROM TABLE + assertSelect("select sum(tax), sum(discount), sum(length(returnflag)) from %s", tableName); + + // DROP TABLE + query(format("DROP TABLE %s", tableName)); + } + + @Test(dataProvider = "storage_formats", groups = {STORAGE_FORMATS}) + public void testCreatePartitionedTableAs(StorageFormat storageFormat) + { + setSessionProperties(storageFormat); + + String tableName = "storage_formats_test_create_table_as_select_partitioned_" + storageFormat.getName().toLowerCase(); + + // DROP TABLE + query(format("DROP TABLE IF EXISTS %s", tableName)); + + // CREATE TABLE AS SELECT + String createTableAsSelect = format( + "CREATE TABLE %s WITH (format='%s', partitioned_by = ARRAY['returnflag']) AS " + + "SELECT " + + "orderkey, partkey, suppkey, linenumber, quantity, extendedprice, discount, tax, " + + "linestatus, shipinstruct, shipmode, comment, returnflag " + + "FROM tpch.%s.lineitem", + tableName, + storageFormat.getName(), + TPCH_SCHEMA); + query(createTableAsSelect); + + // SELECT FROM TABLE + assertSelect("select sum(tax), sum(discount), sum(length(returnflag)) from %s", tableName); + + // DROP TABLE + query(format("DROP TABLE %s", tableName)); + } + + private static void assertSelect(String query, String tableName) + { + QueryResult expected = query(format(query, "tpch." + TPCH_SCHEMA + ".lineitem")); + List expectedRows = expected.rows().stream() + .map((columns) -> row(columns.toArray())) + .collect(toImmutableList()); + QueryResult actual = query(format(query, tableName)); + assertThat(actual) + .hasColumns(expected.getColumnTypes()) + .containsExactly(expectedRows); + } + + private static void setSessionProperties(StorageFormat storageFormat) + { + setSessionProperties(storageFormat.getSessionProperties()); + } + + private static void setSessionProperties(Map sessionProperties) + { + Connection connection = defaultQueryExecutor().getConnection(); + try { + PrestoConnection prestoConnection = connection.unwrap(PrestoConnection.class); + // create more than one split + prestoConnection.setSessionProperty("task_writer_count", "4"); + prestoConnection.setSessionProperty("redistribute_writes", "false"); + for (Map.Entry sessionProperty : sessionProperties.entrySet()) { + prestoConnection.setSessionProperty(sessionProperty.getKey(), sessionProperty.getValue()); + } + } + catch (SQLException e) { + throw Throwables.propagate(e); + } + } + + private static StorageFormat storageFormat(String name) + { + return storageFormat(name, ImmutableMap.of()); + } + + private static StorageFormat storageFormat(String name, Map sessionProperties) + { + return new StorageFormat(name, sessionProperties); + } + + private static class StorageFormat + { + private final String name; + private final Map sessionProperties; + + private StorageFormat(String name, Map sessionProperties) + { + this.name = requireNonNull(name, "name is null"); + this.sessionProperties = requireNonNull(sessionProperties, "sessionProperties is null"); + } + + public String getName() + { + return name; + } + + public Map getSessionProperties() + { + return sessionProperties; + } + + @Override + public String toString() + { + return MoreObjects.toStringHelper(this) + .add("name", name) + .add("sessionProperties", sessionProperties) + .toString(); + } + } } From e039914411704f00119d5b353535da28a3c6a0de Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Mon, 21 Mar 2016 18:25:13 +0100 Subject: [PATCH 03/20] Make HiveHdfsConfiguration immutable Make INITIAL_CONFIGURATION in HiveHdfsConfiguration immutable. Based on the speficic of the Configuration implementation it can be modified during the additional hadoop modules loading, such as DistributedFileSystem, MapReduce, etc. Let's consider the next flow 1. Client1 calls HdfsConfiguration.getConfiguration() 2. Client2 calls FileSystem.getFileSystem() which implicitly loads the DistributedFileSystem 3. Client3 calls HdfsConfiguration.getConfiguration() In such case Client1 and Client3 will obtain the Configuration with the different property set. In order to solve this issue we must load the hdfs related configuration during the HiveHdfsConfiguration initialization and store thoose values in unmodifiable INITIAL_CONFIGURATION. --- .../presto/hive/HiveHdfsConfiguration.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java index 8ad5acc65c9f0..71caf2cdae55b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveHdfsConfiguration.java @@ -25,7 +25,20 @@ public class HiveHdfsConfiguration implements HdfsConfiguration { - private static final Configuration DEFAULT_CONFIGURATION = new Configuration(); + private static final Configuration INITIAL_CONFIGURATION; + + static { + Configuration.addDefaultResource("hdfs-default.xml"); + Configuration.addDefaultResource("hdfs-site.xml"); + + // must not be transitively reloaded during the future loading of various Hadoop modules + // all the required default resources must be declared above + INITIAL_CONFIGURATION = new Configuration(false); + Configuration defaultConfiguration = new Configuration(); + for (Map.Entry entry : defaultConfiguration) { + INITIAL_CONFIGURATION.set(entry.getKey(), entry.getValue()); + } + } @SuppressWarnings("ThreadLocalNotStaticFinal") private final ThreadLocal hadoopConfiguration = new ThreadLocal() @@ -34,7 +47,7 @@ public class HiveHdfsConfiguration protected Configuration initialValue() { Configuration config = new Configuration(false); - for (Map.Entry entry : DEFAULT_CONFIGURATION) { + for (Map.Entry entry : INITIAL_CONFIGURATION) { config.set(entry.getKey(), entry.getValue()); } updater.updateConfiguration(config); From be6808f75741606afa12cc47f51923474e2ee7ed Mon Sep 17 00:00:00 2001 From: arhimondr Date: Tue, 29 Mar 2016 17:06:49 +0200 Subject: [PATCH 04/20] Add UserGroupInformationShim Instead of using the reflection to acces the private methods from the UserGroupInformation we are going to leverage the thin Shim. This commit is going to be replaced with the updated versions of Hadoop libraries once they released. --- .../security/UserGroupInformationShim.java | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java diff --git a/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java b/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java new file mode 100644 index 0000000000000..0e828c51fa9e7 --- /dev/null +++ b/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java @@ -0,0 +1,52 @@ +/* + * Licensed 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.hadoop.security; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; + +import java.util.Set; + +import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS; + +public final class UserGroupInformationShim +{ + private UserGroupInformationShim() {} + + public static Subject getSubject(UserGroupInformation userGroupInformation) + { + return userGroupInformation.getSubject(); + } + + public static UserGroupInformation createUserGroupInformationForSubject(Subject subject) + { + if (subject == null) { + throw new NullPointerException("subject is null"); + } + Set kerberosPrincipals = subject.getPrincipals(KerberosPrincipal.class); + if (kerberosPrincipals.isEmpty()) { + throw new IllegalArgumentException("subject must contain a KerberosPrincipal"); + } + if (kerberosPrincipals.size() != 1) { + throw new IllegalArgumentException("subject must contain only a single KerberosPrincipal"); + } + + KerberosPrincipal principal = kerberosPrincipals.iterator().next(); + User user = new User(principal.getName(), KERBEROS, null); + subject.getPrincipals().add(user); + UserGroupInformation userGroupInformation = new UserGroupInformation(subject); + userGroupInformation.setAuthenticationMethod(KERBEROS); + return userGroupInformation; + } +} From 6bb741b7302eedefc1f91f2b60f9d77c5f01cc88 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Mon, 21 Mar 2016 18:52:50 +0100 Subject: [PATCH 05/20] Support kerberized HiveMetastore in Hive connector --- .../presto/hive/HiveClientConfig.java | 63 +++++++++++ .../presto/hive/HiveConnectorFactory.java | 10 ++ .../hive/HiveMetastoreClientFactory.java | 13 ++- .../authentication/AuthenticationModules.java | 51 +++++++++ .../CachingKerberosHadoopAuthentication.java | 72 ++++++++++++ .../authentication/HadoopAuthentication.java | 21 ++++ .../HiveMetastoreAuthentication.java | 23 ++++ .../KerberosAuthentication.java | 107 ++++++++++++++++++ .../KerberosHadoopAuthentication.java | 54 +++++++++ .../KerberosHiveMetastoreAuthentication.java | 85 ++++++++++++++ .../authentication/KerberosTicketUtils.java | 75 ++++++++++++ .../NoHiveMetastoreAuthentication.java | 28 +++++ .../presto/hive/thrift/Transport.java | 21 ++-- .../hive/MockHiveMetastoreClientFactory.java | 3 +- .../presto/hive/TestHiveClientConfig.java | 16 ++- .../presto/hive/TestingHiveCluster.java | 3 +- 16 files changed, 625 insertions(+), 20 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/CachingKerberosHadoopAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/HadoopAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/HiveMetastoreAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHadoopAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosTicketUtils.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHiveMetastoreAuthentication.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java index d8c26f635e5df..e9e326458cced 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java @@ -117,6 +117,11 @@ public class HiveClientConfig private DataSize orcMaxBufferSize = new DataSize(8, MEGABYTE); private DataSize orcStreamBufferSize = new DataSize(8, MEGABYTE); + private HiveMetastoreAuthenticationType hiveMetastoreAuthenticationType = HiveMetastoreAuthenticationType.NONE; + private String hiveMetastoreServicePrincipal; + private String hiveMetastoreClientPrincipal; + private String hiveMetastoreClientKeytab; + public int getMaxInitialSplits() { return maxInitialSplits; @@ -874,4 +879,62 @@ public HiveClientConfig setUseParquetColumnNames(boolean useParquetColumnNames) this.useParquetColumnNames = useParquetColumnNames; return this; } + + public enum HiveMetastoreAuthenticationType + { + NONE, + KERBEROS + } + + public HiveMetastoreAuthenticationType getHiveMetastoreAuthenticationType() + { + return hiveMetastoreAuthenticationType; + } + + @Config("hive.metastore.authentication.type") + @ConfigDescription("Hive Metastore authentication type") + public HiveClientConfig setHiveMetastoreAuthenticationType(HiveMetastoreAuthenticationType hiveMetastoreAuthenticationType) + { + this.hiveMetastoreAuthenticationType = hiveMetastoreAuthenticationType; + return this; + } + + public String getHiveMetastoreServicePrincipal() + { + return hiveMetastoreServicePrincipal; + } + + @Config("hive.metastore.service.principal") + @ConfigDescription("Hive Metastore service principal") + public HiveClientConfig setHiveMetastoreServicePrincipal(String hiveMetastoreServicePrincipal) + { + this.hiveMetastoreServicePrincipal = hiveMetastoreServicePrincipal; + return this; + } + + public String getHiveMetastoreClientPrincipal() + { + return hiveMetastoreClientPrincipal; + } + + @Config("hive.metastore.client.principal") + @ConfigDescription("Hive Metastore client principal") + public HiveClientConfig setHiveMetastoreClientPrincipal(String hiveMetastoreClientPrincipal) + { + this.hiveMetastoreClientPrincipal = hiveMetastoreClientPrincipal; + return this; + } + + public String getHiveMetastoreClientKeytab() + { + return hiveMetastoreClientKeytab; + } + + @Config("hive.metastore.client.keytab") + @ConfigDescription("Hive Metastore client keytab location") + public HiveClientConfig setHiveMetastoreClientKeytab(String hiveMetastoreClientKeytab) + { + this.hiveMetastoreClientKeytab = hiveMetastoreClientKeytab; + return this; + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java index c686118c7d184..8fdaffd14d6e5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java @@ -46,6 +46,8 @@ import static com.facebook.presto.hive.ConditionalModule.installModuleIf; import static com.facebook.presto.hive.SecurityConfig.ALLOW_ALL_ACCESS_CONTROL; +import static com.facebook.presto.hive.authentication.AuthenticationModules.kerberosHiveMetastoreAuthenticationModule; +import static com.facebook.presto.hive.authentication.AuthenticationModules.noHiveMetastoreAuthenticationModule; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; @@ -115,6 +117,14 @@ public Connector create(String connectorId, Map config) SecurityConfig.class, security -> "sql-standard".equalsIgnoreCase(security.getSecuritySystem()), new SqlStandardSecurityModule()), + installModuleIf( + HiveClientConfig.class, + hiveClientConfig -> hiveClientConfig.getHiveMetastoreAuthenticationType() == HiveClientConfig.HiveMetastoreAuthenticationType.NONE, + noHiveMetastoreAuthenticationModule()), + installModuleIf( + HiveClientConfig.class, + hiveClientConfig -> hiveClientConfig.getHiveMetastoreAuthenticationType() == HiveClientConfig.HiveMetastoreAuthenticationType.KERBEROS, + kerberosHiveMetastoreAuthenticationModule()), binder -> { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); binder.bind(MBeanServer.class).toInstance(new RebindSafeMBeanServer(platformMBeanServer)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetastoreClientFactory.java index 83605e3d97e04..1668896be9222 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetastoreClientFactory.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.facebook.presto.hive.thrift.Transport; import com.google.common.net.HostAndPort; @@ -23,26 +24,30 @@ import javax.annotation.Nullable; import javax.inject.Inject; +import static java.util.Objects.requireNonNull; + public class HiveMetastoreClientFactory { private final HostAndPort socksProxy; private final int timeoutMillis; + private final HiveMetastoreAuthentication metastoreAuthentication; - public HiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout) + public HiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication) { this.socksProxy = socksProxy; this.timeoutMillis = Ints.checkedCast(timeout.toMillis()); + this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } @Inject - public HiveMetastoreClientFactory(HiveClientConfig config) + public HiveMetastoreClientFactory(HiveClientConfig config, HiveMetastoreAuthentication metastoreAuthentication) { - this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout()); + this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout(), metastoreAuthentication); } public HiveMetastoreClient create(String host, int port) throws TTransportException { - return new ThriftHiveMetastoreClient(Transport.create(host, port, socksProxy, timeoutMillis)); + return new ThriftHiveMetastoreClient(Transport.create(host, port, socksProxy, timeoutMillis, metastoreAuthentication)); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java new file mode 100644 index 0000000000000..11c41645f9928 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java @@ -0,0 +1,51 @@ +package com.facebook.presto.hive.authentication; + +import com.facebook.presto.hive.ForHiveMetastore; +import com.facebook.presto.hive.HiveClientConfig; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Singleton; + +import static com.google.inject.Scopes.SINGLETON; + +public final class AuthenticationModules +{ + private AuthenticationModules() {} + + public static Module noHiveMetastoreAuthenticationModule() + { + return binder -> binder + .bind(HiveMetastoreAuthentication.class) + .to(NoHiveMetastoreAuthentication.class) + .in(SINGLETON); + } + + public static Module kerberosHiveMetastoreAuthenticationModule() + { + return new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(HiveMetastoreAuthentication.class) + .to(KerberosHiveMetastoreAuthentication.class) + .in(SINGLETON); + } + + @Inject + @Provides + @Singleton + @ForHiveMetastore + HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig) + { + String principal = hiveClientConfig.getHiveMetastoreClientPrincipal(); + String keytabLocation = hiveClientConfig.getHiveMetastoreClientKeytab(); + KerberosAuthentication kerberosAuthentication = new KerberosAuthentication(principal, keytabLocation); + KerberosHadoopAuthentication kerberosHadoopAuthentication = new KerberosHadoopAuthentication(kerberosAuthentication); + return new CachingKerberosHadoopAuthentication(kerberosHadoopAuthentication); + } + }; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/CachingKerberosHadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/CachingKerberosHadoopAuthentication.java new file mode 100644 index 0000000000000..1a17fea680e23 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/CachingKerberosHadoopAuthentication.java @@ -0,0 +1,72 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.hadoop.security.UserGroupInformation; + +import javax.annotation.concurrent.GuardedBy; +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosTicket; + +import static com.facebook.presto.hive.authentication.KerberosTicketUtils.getTicketGrantingTicket; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.security.UserGroupInformationShim.getSubject; + +public class CachingKerberosHadoopAuthentication + implements HadoopAuthentication +{ + private final KerberosHadoopAuthentication delegate; + + private final Object lock = new Object(); + @GuardedBy("lock") + private UserGroupInformation userGroupInformation = null; + @GuardedBy("lock") + private long nextRefreshTime = Long.MIN_VALUE; + + public CachingKerberosHadoopAuthentication(KerberosHadoopAuthentication delegate) + { + this.delegate = requireNonNull(delegate, "hadoopAuthentication is null"); + } + + @Override + public UserGroupInformation getUserGroupInformation() + { + synchronized (lock) { + if (refreshIsNeeded()) { + refreshUgi(); + } + } + return userGroupInformation; + } + + private void refreshUgi() + { + userGroupInformation = delegate.getUserGroupInformation(); + nextRefreshTime = calculateNextRefreshTime(userGroupInformation); + } + + private boolean refreshIsNeeded() + { + return nextRefreshTime < System.currentTimeMillis() || userGroupInformation == null; + } + + private long calculateNextRefreshTime(UserGroupInformation userGroupInformation) + { + Subject subject = getSubject(userGroupInformation); + checkArgument(subject != null, "subject must be present in kerberos based UGI"); + KerberosTicket tgtTicket = getTicketGrantingTicket(subject); + return KerberosTicketUtils.getRefreshTime(tgtTicket); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HadoopAuthentication.java new file mode 100644 index 0000000000000..412645d5f0f77 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HadoopAuthentication.java @@ -0,0 +1,21 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.hadoop.security.UserGroupInformation; + +public interface HadoopAuthentication +{ + UserGroupInformation getUserGroupInformation(); +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HiveMetastoreAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HiveMetastoreAuthentication.java new file mode 100644 index 0000000000000..1e61d0eb6c958 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HiveMetastoreAuthentication.java @@ -0,0 +1,23 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +public interface HiveMetastoreAuthentication +{ + TTransport authenticate(TTransport rawTransport, String hiveMetastoreHost) + throws TTransportException; +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosAuthentication.java new file mode 100644 index 0000000000000..975faf42c4f99 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosAuthentication.java @@ -0,0 +1,107 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; + +import java.io.IOException; +import java.net.InetAddress; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.nio.file.Files.exists; +import static java.nio.file.Files.isReadable; +import static java.util.Collections.emptySet; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.security.SecurityUtil.getServerPrincipal; + +public class KerberosAuthentication +{ + private static final String KERBEROS_LOGIN_MODULE = "com.sun.security.auth.module.Krb5LoginModule"; + + private final KerberosPrincipal principal; + private final Configuration configuration; + + public KerberosAuthentication(String principal, String keytabLocation) + { + requireNonNull(principal, "principal is null"); + requireNonNull(keytabLocation, "keytabLocation is null"); + Path keytabPath = Paths.get(keytabLocation); + checkArgument(exists(keytabPath), "keytab does not exist: " + keytabLocation); + checkArgument(isReadable(keytabPath), "keytab is not readable: " + keytabLocation); + this.principal = createKerberosPrincipal(principal); + this.configuration = createConfiguration(principal, keytabLocation); + } + + public Subject getSubject() + { + Subject subject = new Subject(false, ImmutableSet.of(principal), emptySet(), emptySet()); + try { + LoginContext loginContext = new LoginContext("", subject, null, configuration); + loginContext.login(); + return loginContext.getSubject(); + } + catch (LoginException e) { + throw Throwables.propagate(e); + } + } + + private static KerberosPrincipal createKerberosPrincipal(String principal) + { + try { + return new KerberosPrincipal(getServerPrincipal(principal, InetAddress.getLocalHost().getHostName())); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private static Configuration createConfiguration(String principal, String keytabLocation) + { + Map options = ImmutableMap.builder() + .put("useKeyTab", "true") + .put("storeKey", "true") + .put("doNotPrompt", "true") + .put("isInitiator", "true") + .put("principal", principal) + .put("keyTab", keytabLocation) + .build(); + + return new Configuration() + { + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) + { + return new AppConfigurationEntry[] { + new AppConfigurationEntry( + KERBEROS_LOGIN_MODULE, + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options + ) + }; + } + }; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHadoopAuthentication.java new file mode 100644 index 0000000000000..abd7904d6c1f6 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHadoopAuthentication.java @@ -0,0 +1,54 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authentication.util.KerberosName; + +import javax.security.auth.Subject; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.security.UserGroupInformationShim.createUserGroupInformationForSubject; + +public class KerberosHadoopAuthentication + implements HadoopAuthentication +{ + static { + // In order to enable KERBEROS authentication method for HDFS + // UserGroupInformation.authenticationMethod static field must be set to KERBEROS + // It is further used in many places in DfsClient + Configuration configuration = new Configuration(false); + configuration.set("hadoop.security.authentication", "kerberos"); + UserGroupInformation.setConfiguration(configuration); + + // KerberosName#rules static field must be initialized + // It is used in KerberosName#getShortName which is used in User constructor invoked by UserGroupInformation#getUGIFromSubject + KerberosName.setRules("DEFAULT"); + } + + private final KerberosAuthentication kerberosAuthentication; + + public KerberosHadoopAuthentication(KerberosAuthentication kerberosAuthentication) + { + this.kerberosAuthentication = requireNonNull(kerberosAuthentication, "kerberosAuthentication is null"); + } + + @Override + public UserGroupInformation getUserGroupInformation() + { + Subject subject = kerberosAuthentication.getSubject(); + return createUserGroupInformationForSubject(subject); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java new file mode 100644 index 0000000000000..838b5a71ccdef --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosHiveMetastoreAuthentication.java @@ -0,0 +1,85 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import com.facebook.presto.hive.ForHiveMetastore; +import com.facebook.presto.hive.HiveClientConfig; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport; +import org.apache.hadoop.security.SaslRpcServer; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import javax.security.sasl.Sasl; + +import java.io.IOException; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS; +import static org.apache.hadoop.security.SecurityUtil.getServerPrincipal; + +public class KerberosHiveMetastoreAuthentication + implements HiveMetastoreAuthentication +{ + private final String hiveMetastoreServicePrincipal; + private final HadoopAuthentication authentication; + + @Inject + public KerberosHiveMetastoreAuthentication(HiveClientConfig hiveClientConfig, @ForHiveMetastore HadoopAuthentication authentication) + { + this(hiveClientConfig.getHiveMetastoreServicePrincipal(), authentication); + } + + public KerberosHiveMetastoreAuthentication(String hiveMetastoreServicePrincipal, HadoopAuthentication authentication) + { + this.hiveMetastoreServicePrincipal = requireNonNull(hiveMetastoreServicePrincipal, "hiveMetastoreServicePrincipal is null"); + this.authentication = requireNonNull(authentication, "authentication is null"); + } + + @Override + public TTransport authenticate(TTransport rawTransport, String hiveMetastoreHost) + throws TTransportException + { + try { + String serverPrincipal = getServerPrincipal(hiveMetastoreServicePrincipal, hiveMetastoreHost); + String[] names = SaslRpcServer.splitKerberosName(serverPrincipal); + checkState(names.length == 3, + "Kerberos principal name does NOT have the expected hostname part: %s", serverPrincipal); + + Map saslProps = ImmutableMap.of( + Sasl.QOP, "auth", + Sasl.SERVER_AUTH, "true" + ); + + TTransport saslTransport = new TSaslClientTransport( + KERBEROS.getMechanismName(), + null, + names[0], + names[1], + saslProps, + null, + rawTransport); + + return new TUGIAssumingTransport(saslTransport, authentication.getUserGroupInformation()); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosTicketUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosTicketUtils.java new file mode 100644 index 0000000000000..0764c257f1dea --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/KerberosTicketUtils.java @@ -0,0 +1,75 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.kerberos.KerberosTicket; + +import java.util.Set; + +final class KerberosTicketUtils +{ + private static final float TICKET_RENEW_WINDOW = 0.80f; + + private KerberosTicketUtils() + { + } + + static KerberosTicket getTicketGrantingTicket(Subject subject) + { + Set tickets = subject.getPrivateCredentials(KerberosTicket.class); + for (KerberosTicket ticket : tickets) { + if (isOriginalTicketGrantingTicket(ticket)) { + return ticket; + } + } + throw new IllegalArgumentException("kerberos ticket not found in " + subject); + } + + static long getRefreshTime(KerberosTicket ticket) + { + long start = ticket.getStartTime().getTime(); + long end = ticket.getEndTime().getTime(); + return start + (long) ((end - start) * TICKET_RENEW_WINDOW); + } + + /** + * Check whether the server principal is the TGS's principal + * + * @param ticket the original TGT (the ticket that is obtained when a + * kinit is done) + * @return true or false + */ + static boolean isOriginalTicketGrantingTicket(KerberosTicket ticket) + { + return isTicketGrantingServerPrincipal(ticket.getServer()); + } + + /** + * TGS must have the server principal of the form "krbtgt/FOO@FOO". + * + * @return true or false + */ + private static boolean isTicketGrantingServerPrincipal(KerberosPrincipal principal) + { + if (principal == null) { + return false; + } + if (principal.getName().equals("krbtgt/" + principal.getRealm() + "@" + principal.getRealm())) { + return true; + } + return false; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHiveMetastoreAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHiveMetastoreAuthentication.java new file mode 100644 index 0000000000000..3342c4af988c0 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHiveMetastoreAuthentication.java @@ -0,0 +1,28 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +public class NoHiveMetastoreAuthentication + implements HiveMetastoreAuthentication +{ + @Override + public TTransport authenticate(TTransport rawTransport, String hiveMetastoreHost) + throws TTransportException + { + return rawTransport; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java b/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java index ce03b7a089d96..b9bdd521bdae4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/thrift/Transport.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.thrift; +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; import com.google.common.net.HostAndPort; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; @@ -27,11 +28,16 @@ public final class Transport { - public static TTransport create(String host, int port, HostAndPort socksProxy, int timeoutMillis) + public static TTransport create(String host, int port, HostAndPort socksProxy, int timeoutMillis, HiveMetastoreAuthentication authentication) throws TTransportException { try { - return new TTransportWrapper(createRaw(host, port, socksProxy, timeoutMillis), host); + TTransport rawTransport = createRaw(host, port, socksProxy, timeoutMillis); + TTransport authenticatedTransport = authentication.authenticate(rawTransport, host); + if (!authenticatedTransport.isOpen()) { + authenticatedTransport.open(); + } + return new TTransportWrapper(authenticatedTransport, host); } catch (TTransportException e) { throw rewriteException(e, host); @@ -44,16 +50,7 @@ private static TTransport createRaw(String host, int port, HostAndPort socksProx throws TTransportException { if (socksProxy == null) { - TTransport transport = new TSocket(host, port, timeoutMillis); - - try { - transport.open(); - return transport; - } - catch (Throwable t) { - transport.close(); - throw t; - } + return new TSocket(host, port, timeoutMillis); } Socket socks = createSocksSocket(socksProxy); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/MockHiveMetastoreClientFactory.java b/presto-hive/src/test/java/com/facebook/presto/hive/MockHiveMetastoreClientFactory.java index 47b257ff2f773..a4db721ff29d9 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/MockHiveMetastoreClientFactory.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/MockHiveMetastoreClientFactory.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.authentication.NoHiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.google.common.net.HostAndPort; import io.airlift.units.Duration; @@ -33,7 +34,7 @@ public class MockHiveMetastoreClientFactory public MockHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, List clients) { - super(socksProxy, timeout); + super(socksProxy, timeout, new NoHiveMetastoreAuthentication()); this.clients = new ArrayList<>(requireNonNull(clients, "clients is null")); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java index 56a2c896bc857..d7fa6c708e89d 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java @@ -92,7 +92,11 @@ public void testDefaults() .setAssumeCanonicalPartitionKeys(false) .setOrcMaxMergeDistance(new DataSize(1, Unit.MEGABYTE)) .setOrcMaxBufferSize(new DataSize(8, Unit.MEGABYTE)) - .setOrcStreamBufferSize(new DataSize(8, Unit.MEGABYTE))); + .setOrcStreamBufferSize(new DataSize(8, Unit.MEGABYTE)) + .setHiveMetastoreAuthenticationType(HiveClientConfig.HiveMetastoreAuthenticationType.NONE) + .setHiveMetastoreServicePrincipal(null) + .setHiveMetastoreClientPrincipal(null) + .setHiveMetastoreClientKeytab(null)); } @Test @@ -156,6 +160,10 @@ public void testExplicitPropertyMappings() .put("hive.orc.max-merge-distance", "22kB") .put("hive.orc.max-buffer-size", "44kB") .put("hive.orc.stream-buffer-size", "55kB") + .put("hive.metastore.authentication.type", "KERBEROS") + .put("hive.metastore.service.principal", "hive/_HOST@EXAMPLE.COM") + .put("hive.metastore.client.principal", "metastore@EXAMPLE.COM") + .put("hive.metastore.client.keytab", "/tmp/metastore.keytab") .build(); HiveClientConfig expected = new HiveClientConfig() @@ -215,7 +223,11 @@ public void testExplicitPropertyMappings() .setAssumeCanonicalPartitionKeys(true) .setOrcMaxMergeDistance(new DataSize(22, Unit.KILOBYTE)) .setOrcMaxBufferSize(new DataSize(44, Unit.KILOBYTE)) - .setOrcStreamBufferSize(new DataSize(55, Unit.KILOBYTE)); + .setOrcStreamBufferSize(new DataSize(55, Unit.KILOBYTE)) + .setHiveMetastoreAuthenticationType(HiveClientConfig.HiveMetastoreAuthenticationType.KERBEROS) + .setHiveMetastoreServicePrincipal("hive/_HOST@EXAMPLE.COM") + .setHiveMetastoreClientPrincipal("metastore@EXAMPLE.COM") + .setHiveMetastoreClientKeytab("/tmp/metastore.keytab"); ConfigAssertions.assertFullMapping(properties, expected); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java index 6cc928a3c74f6..c35f682a03c8b 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.authentication.NoHiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.google.common.base.Throwables; import org.apache.thrift.transport.TTransportException; @@ -39,7 +40,7 @@ public TestingHiveCluster(HiveClientConfig config, String host, int port) public HiveMetastoreClient createMetastoreClient() { try { - return new HiveMetastoreClientFactory(config).create(host, port); + return new HiveMetastoreClientFactory(config, new NoHiveMetastoreAuthentication()).create(host, port); } catch (TTransportException e) { throw Throwables.propagate(e); From f6c74810e1843a9fe6295751d8f4d883c056b6ce Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 22 Mar 2016 14:45:20 +0100 Subject: [PATCH 06/20] Introduce different HDFS authentication types Support both KERBEROS and SIMPLE hadoop authentications with impersonation and without. --- .../com/facebook/presto/hive/ForHdfs.java | 35 +++++++ .../presto/hive/HiveClientConfig.java | 63 ++++++++++++ .../presto/hive/HiveConnectorFactory.java | 24 +++++ .../authentication/AuthenticationModules.java | 95 ++++++++++++++++++- .../DirectHdfsAuthentication.java | 39 ++++++++ .../GenericExceptionAction.java | 20 ++++ .../authentication/HdfsAuthentication.java | 28 ++++++ .../ImpersonatingHdfsAuthentication.java | 45 +++++++++ .../authentication/NoHdfsAuthentication.java | 25 +++++ .../SimpleHadoopAuthentication.java | 34 +++++++ .../UserGroupInformationUtils.java | 64 +++++++++++++ .../presto/hive/TestHiveClientConfig.java | 16 +++- 12 files changed, 483 insertions(+), 5 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ForHdfs.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/DirectHdfsAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/GenericExceptionAction.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/ImpersonatingHdfsAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHdfsAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/authentication/UserGroupInformationUtils.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ForHdfs.java b/presto-hive/src/main/java/com/facebook/presto/hive/ForHdfs.java new file mode 100644 index 0000000000000..d0bfa7424abb2 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ForHdfs.java @@ -0,0 +1,35 @@ +/* + * Copyright 2016, Teradata Corp. All rights reserved. + */ + +/* + * Licensed 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 com.facebook.presto.hive; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForHdfs +{ +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java index e9e326458cced..a4251b043353f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveClientConfig.java @@ -122,6 +122,11 @@ public class HiveClientConfig private String hiveMetastoreClientPrincipal; private String hiveMetastoreClientKeytab; + private HdfsAuthenticationType hdfsAuthenticationType = HdfsAuthenticationType.NONE; + private boolean hdfsImpersonationEnabled; + private String hdfsPrestoPrincipal; + private String hdfsPrestoKeytab; + public int getMaxInitialSplits() { return maxInitialSplits; @@ -937,4 +942,62 @@ public HiveClientConfig setHiveMetastoreClientKeytab(String hiveMetastoreClientK this.hiveMetastoreClientKeytab = hiveMetastoreClientKeytab; return this; } + + public enum HdfsAuthenticationType + { + NONE, + KERBEROS, + } + + public HdfsAuthenticationType getHdfsAuthenticationType() + { + return hdfsAuthenticationType; + } + + @Config("hive.hdfs.authentication.type") + @ConfigDescription("HDFS authentication type") + public HiveClientConfig setHdfsAuthenticationType(HdfsAuthenticationType hdfsAuthenticationType) + { + this.hdfsAuthenticationType = hdfsAuthenticationType; + return this; + } + + public boolean isHdfsImpersonationEnabled() + { + return hdfsImpersonationEnabled; + } + + @Config("hive.hdfs.impersonation.enabled") + @ConfigDescription("Should Presto user be impersonated when communicating with HDFS") + public HiveClientConfig setHdfsImpersonationEnabled(boolean hdfsImpersonationEnabled) + { + this.hdfsImpersonationEnabled = hdfsImpersonationEnabled; + return this; + } + + public String getHdfsPrestoPrincipal() + { + return hdfsPrestoPrincipal; + } + + @Config("hive.hdfs.presto.principal") + @ConfigDescription("Presto principal used to access HDFS") + public HiveClientConfig setHdfsPrestoPrincipal(String hdfsPrestoPrincipal) + { + this.hdfsPrestoPrincipal = hdfsPrestoPrincipal; + return this; + } + + public String getHdfsPrestoKeytab() + { + return hdfsPrestoKeytab; + } + + @Config("hive.hdfs.presto.keytab") + @ConfigDescription("Presto keytab used to access HDFS") + public HiveClientConfig setHdfsPrestoKeytab(String hdfsPrestoKeytab) + { + this.hdfsPrestoKeytab = hdfsPrestoKeytab; + return this; + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java index 8fdaffd14d6e5..b38f4dd38ad44 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java @@ -46,8 +46,12 @@ import static com.facebook.presto.hive.ConditionalModule.installModuleIf; import static com.facebook.presto.hive.SecurityConfig.ALLOW_ALL_ACCESS_CONTROL; +import static com.facebook.presto.hive.authentication.AuthenticationModules.kerberosHdfsAuthenticationModule; import static com.facebook.presto.hive.authentication.AuthenticationModules.kerberosHiveMetastoreAuthenticationModule; +import static com.facebook.presto.hive.authentication.AuthenticationModules.kerberosImpersonatingHdfsAuthenticationModule; +import static com.facebook.presto.hive.authentication.AuthenticationModules.noHdfsAuthenticationModule; import static com.facebook.presto.hive.authentication.AuthenticationModules.noHiveMetastoreAuthenticationModule; +import static com.facebook.presto.hive.authentication.AuthenticationModules.simpleImpersonatingHdfsAuthenticationModule; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; @@ -125,6 +129,26 @@ public Connector create(String connectorId, Map config) HiveClientConfig.class, hiveClientConfig -> hiveClientConfig.getHiveMetastoreAuthenticationType() == HiveClientConfig.HiveMetastoreAuthenticationType.KERBEROS, kerberosHiveMetastoreAuthenticationModule()), + installModuleIf( + HiveClientConfig.class, + configuration -> configuration.getHdfsAuthenticationType() == HiveClientConfig.HdfsAuthenticationType.NONE && + !configuration.isHdfsImpersonationEnabled(), + noHdfsAuthenticationModule()), + installModuleIf( + HiveClientConfig.class, + configuration -> configuration.getHdfsAuthenticationType() == HiveClientConfig.HdfsAuthenticationType.NONE && + configuration.isHdfsImpersonationEnabled(), + simpleImpersonatingHdfsAuthenticationModule()), + installModuleIf( + HiveClientConfig.class, + configuration -> configuration.getHdfsAuthenticationType() == HiveClientConfig.HdfsAuthenticationType.KERBEROS && + !configuration.isHdfsImpersonationEnabled(), + kerberosHdfsAuthenticationModule()), + installModuleIf( + HiveClientConfig.class, + configuration -> configuration.getHdfsAuthenticationType() == HiveClientConfig.HdfsAuthenticationType.KERBEROS && + configuration.isHdfsImpersonationEnabled(), + kerberosImpersonatingHdfsAuthenticationModule()), binder -> { MBeanServer platformMBeanServer = ManagementFactory.getPlatformMBeanServer(); binder.bind(MBeanServer.class).toInstance(new RebindSafeMBeanServer(platformMBeanServer)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java index 11c41645f9928..dd97ed2cd20de 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/AuthenticationModules.java @@ -1,9 +1,24 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; +import com.facebook.presto.hive.ForHdfs; import com.facebook.presto.hive.ForHiveMetastore; import com.facebook.presto.hive.HiveClientConfig; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.Singleton; @@ -42,10 +57,84 @@ HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfi { String principal = hiveClientConfig.getHiveMetastoreClientPrincipal(); String keytabLocation = hiveClientConfig.getHiveMetastoreClientKeytab(); - KerberosAuthentication kerberosAuthentication = new KerberosAuthentication(principal, keytabLocation); - KerberosHadoopAuthentication kerberosHadoopAuthentication = new KerberosHadoopAuthentication(kerberosAuthentication); - return new CachingKerberosHadoopAuthentication(kerberosHadoopAuthentication); + return createCachingKerberosHadoopAuthentication(principal, keytabLocation); } }; } + + public static Module noHdfsAuthenticationModule() + { + return binder -> binder + .bind(HdfsAuthentication.class) + .to(NoHdfsAuthentication.class) + .in(SINGLETON); + } + + public static Module simpleImpersonatingHdfsAuthenticationModule() + { + return binder -> { + binder.bind(Key.get(HadoopAuthentication.class, ForHdfs.class)) + .to(SimpleHadoopAuthentication.class); + binder.bind(HdfsAuthentication.class) + .to(ImpersonatingHdfsAuthentication.class) + .in(SINGLETON); + }; + } + + public static Module kerberosHdfsAuthenticationModule() + { + return new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(HdfsAuthentication.class) + .to(DirectHdfsAuthentication.class) + .in(SINGLETON); + } + + @Inject + @Provides + @Singleton + @ForHdfs + HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig) + { + String principal = hiveClientConfig.getHdfsPrestoPrincipal(); + String keytabLocation = hiveClientConfig.getHdfsPrestoKeytab(); + return createCachingKerberosHadoopAuthentication(principal, keytabLocation); + } + }; + } + + public static Module kerberosImpersonatingHdfsAuthenticationModule() + { + return new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(HdfsAuthentication.class) + .to(ImpersonatingHdfsAuthentication.class) + .in(SINGLETON); + } + + @Inject + @Provides + @Singleton + @ForHdfs + HadoopAuthentication createHadoopAuthentication(HiveClientConfig hiveClientConfig) + { + String principal = hiveClientConfig.getHdfsPrestoPrincipal(); + String keytabLocation = hiveClientConfig.getHdfsPrestoKeytab(); + return createCachingKerberosHadoopAuthentication(principal, keytabLocation); + } + }; + } + + private static HadoopAuthentication createCachingKerberosHadoopAuthentication(String principal, String keytabLocation) + { + KerberosAuthentication kerberosAuthentication = new KerberosAuthentication(principal, keytabLocation); + KerberosHadoopAuthentication kerberosHadoopAuthentication = new KerberosHadoopAuthentication(kerberosAuthentication); + return new CachingKerberosHadoopAuthentication(kerberosHadoopAuthentication); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/DirectHdfsAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/DirectHdfsAuthentication.java new file mode 100644 index 0000000000000..c108e90ff62b0 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/DirectHdfsAuthentication.java @@ -0,0 +1,39 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import com.facebook.presto.hive.ForHdfs; +import com.google.inject.Inject; + +import static com.facebook.presto.hive.authentication.UserGroupInformationUtils.executeActionInDoAs; +import static java.util.Objects.requireNonNull; + +public class DirectHdfsAuthentication + implements HdfsAuthentication +{ + private final HadoopAuthentication hadoopAuthentication; + + @Inject + public DirectHdfsAuthentication(@ForHdfs HadoopAuthentication hadoopAuthentication) + { + this.hadoopAuthentication = requireNonNull(hadoopAuthentication); + } + + @Override + public R doAs(String user, GenericExceptionAction action) + throws E + { + return executeActionInDoAs(hadoopAuthentication.getUserGroupInformation(), action); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/GenericExceptionAction.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/GenericExceptionAction.java new file mode 100644 index 0000000000000..a4b250b0f5539 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/GenericExceptionAction.java @@ -0,0 +1,20 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +public interface GenericExceptionAction +{ + R run() + throws E; +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsAuthentication.java new file mode 100644 index 0000000000000..41b8820cec750 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/HdfsAuthentication.java @@ -0,0 +1,28 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +public interface HdfsAuthentication +{ + R doAs(String user, GenericExceptionAction action) + throws E; + + default void doAs(String user, Runnable action) + { + doAs(user, () -> { + action.run(); + return null; + }); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/ImpersonatingHdfsAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/ImpersonatingHdfsAuthentication.java new file mode 100644 index 0000000000000..059b7dd0b2c7c --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/ImpersonatingHdfsAuthentication.java @@ -0,0 +1,45 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import com.facebook.presto.hive.ForHdfs; +import com.google.inject.Inject; +import org.apache.hadoop.security.UserGroupInformation; + +import static com.facebook.presto.hive.authentication.UserGroupInformationUtils.executeActionInDoAs; +import static java.util.Objects.requireNonNull; + +public class ImpersonatingHdfsAuthentication + implements HdfsAuthentication +{ + private final HadoopAuthentication hadoopAuthentication; + + @Inject + public ImpersonatingHdfsAuthentication(@ForHdfs HadoopAuthentication hadoopAuthentication) + { + this.hadoopAuthentication = requireNonNull(hadoopAuthentication); + } + + @Override + public R doAs(String user, GenericExceptionAction action) + throws E + { + return executeActionInDoAs(createProxyUser(user), action); + } + + private UserGroupInformation createProxyUser(String user) + { + return UserGroupInformation.createProxyUser(user, hadoopAuthentication.getUserGroupInformation()); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHdfsAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHdfsAuthentication.java new file mode 100644 index 0000000000000..7d135e67ae7c6 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/NoHdfsAuthentication.java @@ -0,0 +1,25 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +public class NoHdfsAuthentication + implements HdfsAuthentication +{ + @Override + public R doAs(String user, GenericExceptionAction action) + throws E + { + return action.run(); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java new file mode 100644 index 0000000000000..663d852e43992 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java @@ -0,0 +1,34 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import com.google.common.base.Throwables; +import org.apache.hadoop.security.UserGroupInformation; + +import java.io.IOException; + +public class SimpleHadoopAuthentication + implements HadoopAuthentication +{ + @Override + public UserGroupInformation getUserGroupInformation() + { + try { + return UserGroupInformation.getCurrentUser(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/UserGroupInformationUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/UserGroupInformationUtils.java new file mode 100644 index 0000000000000..5656fbc736683 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/UserGroupInformationUtils.java @@ -0,0 +1,64 @@ +/* + * Licensed 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 com.facebook.presto.hive.authentication; + +import org.apache.hadoop.security.UserGroupInformation; + +import java.security.PrivilegedAction; + +final class UserGroupInformationUtils +{ + private UserGroupInformationUtils() {} + + static R executeActionInDoAs(UserGroupInformation userGroupInformation, GenericExceptionAction action) + throws E + { + return userGroupInformation.doAs((PrivilegedAction>) () -> { + try { + return new ResultOrException<>(action.run(), null); + } + catch (Throwable e) { + return new ResultOrException<>(null, e); + } + }).get(); + } + + private static class ResultOrException + { + private final T result; + private final Throwable exception; + + public ResultOrException(T result, Throwable exception) + { + this.result = result; + this.exception = exception; + } + + @SuppressWarnings("unchecked") + public T get() + throws E + { + if (exception != null) { + if (exception instanceof Error) { + throw (Error) exception; + } + if (exception instanceof RuntimeException) { + throw (RuntimeException) exception; + } + throw (E) exception; + } + return result; + } + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java index d7fa6c708e89d..8a1dae03f6ad9 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveClientConfig.java @@ -96,7 +96,11 @@ public void testDefaults() .setHiveMetastoreAuthenticationType(HiveClientConfig.HiveMetastoreAuthenticationType.NONE) .setHiveMetastoreServicePrincipal(null) .setHiveMetastoreClientPrincipal(null) - .setHiveMetastoreClientKeytab(null)); + .setHiveMetastoreClientKeytab(null) + .setHdfsAuthenticationType(HiveClientConfig.HdfsAuthenticationType.NONE) + .setHdfsImpersonationEnabled(false) + .setHdfsPrestoPrincipal(null) + .setHdfsPrestoKeytab(null)); } @Test @@ -164,6 +168,10 @@ public void testExplicitPropertyMappings() .put("hive.metastore.service.principal", "hive/_HOST@EXAMPLE.COM") .put("hive.metastore.client.principal", "metastore@EXAMPLE.COM") .put("hive.metastore.client.keytab", "/tmp/metastore.keytab") + .put("hive.hdfs.authentication.type", "KERBEROS") + .put("hive.hdfs.impersonation.enabled", "true") + .put("hive.hdfs.presto.principal", "presto@EXAMPLE.COM") + .put("hive.hdfs.presto.keytab", "/tmp/presto.keytab") .build(); HiveClientConfig expected = new HiveClientConfig() @@ -227,7 +235,11 @@ public void testExplicitPropertyMappings() .setHiveMetastoreAuthenticationType(HiveClientConfig.HiveMetastoreAuthenticationType.KERBEROS) .setHiveMetastoreServicePrincipal("hive/_HOST@EXAMPLE.COM") .setHiveMetastoreClientPrincipal("metastore@EXAMPLE.COM") - .setHiveMetastoreClientKeytab("/tmp/metastore.keytab"); + .setHiveMetastoreClientKeytab("/tmp/metastore.keytab") + .setHdfsAuthenticationType(HiveClientConfig.HdfsAuthenticationType.KERBEROS) + .setHdfsImpersonationEnabled(true) + .setHdfsPrestoPrincipal("presto@EXAMPLE.COM") + .setHdfsPrestoKeytab("/tmp/presto.keytab"); ConfigAssertions.assertFullMapping(properties, expected); } From f6d832a0fcf84cb0951755d06ba09f6c024e4d28 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 12:34:37 +0100 Subject: [PATCH 07/20] Create authenticated file system in HdfsEnvironment Pass session user as a parameter to HdfsEnvironment.getFileSystem It is enough to just create FileSystem within the UserGroupInformation.doAs to make it authenticate the HDFS requests with Kerberos. --- .../hive/BackgroundHiveSplitLoader.java | 13 ++-- .../facebook/presto/hive/HdfsEnvironment.java | 19 ++++-- .../presto/hive/HiveLocationService.java | 20 +++--- .../facebook/presto/hive/HiveMetadata.java | 64 ++++++++++--------- .../facebook/presto/hive/HivePageSink.java | 10 ++- .../presto/hive/HivePageSinkProvider.java | 9 +-- .../facebook/presto/hive/HiveWriteUtils.java | 38 ++++++----- .../facebook/presto/hive/LocationService.java | 4 +- .../presto/hive/AbstractTestHiveClient.java | 9 +-- .../presto/hive/AbstractTestHiveClientS3.java | 9 +-- .../presto/hive/TestHivePageSink.java | 3 +- 11 files changed, 107 insertions(+), 91 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java b/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java index 1677d878dd252..90030b62c386d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/BackgroundHiveSplitLoader.java @@ -242,7 +242,7 @@ private CompletableFuture loadSplits() } } else { - boolean splittable = isSplittable(files.getInputFormat(), hdfsEnvironment.getFileSystem(file.getPath()), file.getPath()); + boolean splittable = isSplittable(files.getInputFormat(), hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()), file.getPath()); CompletableFuture future = hiveSplitSource.addToQueue(createHiveSplits( files.getPartitionName(), @@ -278,6 +278,7 @@ private void loadPartition(HivePartitionMetadata partition) Path path = new Path(getPartitionLocation(table, partition.getPartition())); Configuration configuration = hdfsEnvironment.getConfiguration(path); InputFormat inputFormat = getInputFormat(configuration, schema, false); + FileSystem fs = hdfsEnvironment.getFileSystem(session.getUser(), path); if (inputFormat instanceof SymlinkTextInputFormat) { if (bucketHandle.isPresent()) { @@ -285,7 +286,7 @@ private void loadPartition(HivePartitionMetadata partition) } // TODO: This should use an iterator like the HiveFileIterator - for (Path targetPath : getTargetPathsFromSymlink(configuration, path)) { + for (Path targetPath : getTargetPathsFromSymlink(fs, path)) { // The input should be in TextInputFormat. TextInputFormat targetInputFormat = new TextInputFormat(); // get the configuration for the target path -- it may be a different hdfs instance @@ -298,7 +299,7 @@ private void loadPartition(HivePartitionMetadata partition) for (InputSplit inputSplit : targetSplits) { FileSplit split = (FileSplit) inputSplit; - FileSystem targetFilesystem = split.getPath().getFileSystem(targetConfiguration); + FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), split.getPath()); FileStatus file = targetFilesystem.getFileStatus(split.getPath()); hiveSplitSource.addToQueue(createHiveSplits( partitionName, @@ -321,7 +322,6 @@ private void loadPartition(HivePartitionMetadata partition) } // If only one bucket could match: load that one file - FileSystem fs = hdfsEnvironment.getFileSystem(path); HiveFileIterator iterator = new HiveFileIterator(path, fs, directoryLister, namenodeStats, partitionName, inputFormat, schema, partitionKeys, effectivePredicate); if (bucket.isPresent()) { List locatedFileStatuses = listAndSortBucketFiles(iterator, bucket.get().getBucketCount()); @@ -352,7 +352,7 @@ private void loadPartition(HivePartitionMetadata partition) for (int bucketIndex = 0; bucketIndex < bucketCount; bucketIndex++) { LocatedFileStatus file = list.get(bucketIndex); - boolean splittable = isSplittable(iterator.getInputFormat(), hdfsEnvironment.getFileSystem(file.getPath()), file.getPath()); + boolean splittable = isSplittable(iterator.getInputFormat(), hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()), file.getPath()); hiveSplitSource.addToQueue(createHiveSplits( iterator.getPartitionName(), @@ -396,10 +396,9 @@ private static List listAndSortBucketFiles(HiveFileIterator h return list; } - private static List getTargetPathsFromSymlink(Configuration conf, Path symlinkDir) + private static List getTargetPathsFromSymlink(FileSystem fileSystem, Path symlinkDir) { try { - FileSystem fileSystem = symlinkDir.getFileSystem(conf); FileStatus[] symlinks = fileSystem.listStatus(symlinkDir, HIDDEN_FILES_PATH_FILTER); List targets = new ArrayList<>(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java index 1412c39b137ba..2eda6669d8fee 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java @@ -15,6 +15,7 @@ import com.facebook.presto.hadoop.HadoopFileSystemCache; import com.facebook.presto.hadoop.HadoopNative; +import com.facebook.presto.hive.authentication.HdfsAuthentication; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -33,13 +34,18 @@ public class HdfsEnvironment } private final HdfsConfiguration hdfsConfiguration; + private final HdfsAuthentication hdfsAuthentication; private final boolean verifyChecksum; @Inject - public HdfsEnvironment(HdfsConfiguration hdfsConfiguration, HiveClientConfig config) + public HdfsEnvironment( + HdfsConfiguration hdfsConfiguration, + HiveClientConfig config, + HdfsAuthentication hdfsAuthentication) { this.hdfsConfiguration = requireNonNull(hdfsConfiguration, "hdfsConfiguration is null"); this.verifyChecksum = requireNonNull(config, "config is null").isVerifyChecksum(); + this.hdfsAuthentication = requireNonNull(hdfsAuthentication, "hdfsAuthentication is null"); } public Configuration getConfiguration(Path path) @@ -47,12 +53,13 @@ public Configuration getConfiguration(Path path) return hdfsConfiguration.getConfiguration(path.toUri()); } - public FileSystem getFileSystem(Path path) + public FileSystem getFileSystem(String user, Path path) throws IOException { - FileSystem fileSystem = path.getFileSystem(getConfiguration(path)); - fileSystem.setVerifyChecksum(verifyChecksum); - - return fileSystem; + return hdfsAuthentication.doAs(user, () -> { + FileSystem fileSystem = path.getFileSystem(getConfiguration(path)); + fileSystem.setVerifyChecksum(verifyChecksum); + return fileSystem; + }); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveLocationService.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveLocationService.java index 728fbc6af4b5f..55c6116bdd2f1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveLocationService.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveLocationService.java @@ -46,18 +46,18 @@ public HiveLocationService(HiveMetastore metastore, HdfsEnvironment hdfsEnvironm } @Override - public LocationHandle forNewTable(String queryId, String schemaName, String tableName) + public LocationHandle forNewTable(String user, String queryId, String schemaName, String tableName) { - Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName); + Path targetPath = getTableDefaultLocation(user, metastore, hdfsEnvironment, schemaName, tableName); // verify the target directory for the table - if (pathExists(hdfsEnvironment, targetPath)) { + if (pathExists(user, hdfsEnvironment, targetPath)) { throw new PrestoException(HIVE_PATH_ALREADY_EXISTS, format("Target directory for table '%s.%s' already exists: %s", schemaName, tableName, targetPath)); } Path writePath; - if (shouldUseTemporaryDirectory(targetPath)) { - writePath = createTemporaryPath(hdfsEnvironment, targetPath); + if (shouldUseTemporaryDirectory(user, targetPath)) { + writePath = createTemporaryPath(user, hdfsEnvironment, targetPath); } else { writePath = targetPath; @@ -67,13 +67,13 @@ public LocationHandle forNewTable(String queryId, String schemaName, String tabl } @Override - public LocationHandle forExistingTable(String queryId, Table table) + public LocationHandle forExistingTable(String user, String queryId, Table table) { Path targetPath = new Path(table.getSd().getLocation()); Optional writePath; - if (shouldUseTemporaryDirectory(targetPath)) { - writePath = Optional.of(createTemporaryPath(hdfsEnvironment, targetPath)); + if (shouldUseTemporaryDirectory(user, targetPath)) { + writePath = Optional.of(createTemporaryPath(user, hdfsEnvironment, targetPath)); } else { writePath = Optional.empty(); @@ -82,11 +82,11 @@ public LocationHandle forExistingTable(String queryId, Table table) return new LocationHandle(targetPath, writePath, true); } - private boolean shouldUseTemporaryDirectory(Path path) + private boolean shouldUseTemporaryDirectory(String user, Path path) { try { // skip using temporary directory for S3 - return !(hdfsEnvironment.getFileSystem(path) instanceof PrestoS3FileSystem); + return !(hdfsEnvironment.getFileSystem(user, path) instanceof PrestoS3FileSystem); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java index 210962105ce53..5d0a50e54612a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java @@ -376,9 +376,9 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe HiveStorageFormat hiveStorageFormat = getHiveStorageFormat(tableMetadata.getProperties()); Map additionalTableParameters = tableParameterCodec.encode(tableMetadata.getProperties()); - LocationHandle locationHandle = locationService.forNewTable(session.getQueryId(), schemaName, tableName); + LocationHandle locationHandle = locationService.forNewTable(session.getUser(), session.getQueryId(), schemaName, tableName); Path targetPath = locationService.targetPathRoot(locationHandle); - createDirectory(hdfsEnvironment, targetPath); + createDirectory(session.getUser(), hdfsEnvironment, targetPath); createTable(schemaName, tableName, tableMetadata.getOwner(), columnHandles, hiveStorageFormat, partitionedBy, bucketProperty, additionalTableParameters, targetPath); } @@ -549,7 +549,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto tableName, columnHandles, session.getQueryId(), - locationService.forNewTable(session.getQueryId(), schemaName, tableName), + locationService.forNewTable(session.getUser(), session.getQueryId(), schemaName, tableName), tableStorageFormat, respectTableFormat ? tableStorageFormat : defaultStorageFormat, partitionedBy, @@ -557,7 +557,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto tableMetadata.getOwner(), additionalTableParameters); - setRollback(() -> rollbackCreateTable(result)); + setRollback(() -> rollbackCreateTable(session.getUser(), result)); return result; } @@ -577,14 +577,14 @@ public void finishCreateTable(ConnectorSession session, ConnectorOutputTableHand // rename if using a temporary directory if (!targetPath.equals(writePath)) { // verify no one raced us to create the target directory - if (pathExists(hdfsEnvironment, targetPath)) { + if (pathExists(session.getUser(), hdfsEnvironment, targetPath)) { throw new PrestoException(HIVE_PATH_ALREADY_EXISTS, format("Target directory for table '%s.%s' already exists: %s", handle.getSchemaName(), handle.getTableName(), targetPath)); } // rename the temporary directory to the target - renameDirectory(hdfsEnvironment, handle.getSchemaName(), handle.getTableName(), writePath, targetPath); + renameDirectory(session.getUser(), hdfsEnvironment, handle.getSchemaName(), handle.getTableName(), writePath, targetPath); } PartitionCommitter partitionCommitter = new PartitionCommitter(handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); @@ -614,17 +614,17 @@ public void finishCreateTable(ConnectorSession session, ConnectorOutputTableHand } catch (Throwable throwable) { partitionCommitter.abort(); - rollbackPartitionUpdates(partitionUpdates, "table creation"); + rollbackPartitionUpdates(session.getUser(), partitionUpdates, "table creation"); throw throwable; } clearRollback(); } - private void rollbackCreateTable(ConnectorOutputTableHandle tableHandle) + private void rollbackCreateTable(String user, ConnectorOutputTableHandle tableHandle) { HiveOutputTableHandle handle = checkType(tableHandle, HiveOutputTableHandle.class, "tableHandle"); - cleanupTempDirectory(locationService.writePathRoot(handle.getLocationHandle()).get().toString(), handle.getFilePrefix(), "create table"); + cleanupTempDirectory(user, locationService.writePathRoot(handle.getLocationHandle()).get().toString(), handle.getFilePrefix(), "create table"); // Note: there is no need to cleanup the target directory as it will only be written // to during the commit call and the commit call cleans up after failures. } @@ -662,12 +662,12 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl tableName.getTableName(), handles, session.getQueryId(), - locationService.forExistingTable(session.getQueryId(), table.get()), + locationService.forExistingTable(session.getUser(), session.getQueryId(), table.get()), HiveBucketProperty.fromStorageDescriptor(table.get().getSd(), table.get().getTableName()), tableStorageFormat, respectTableFormat ? tableStorageFormat : defaultStorageFormat); - setRollback(() -> rollbackInsert(result)); + setRollback(() -> rollbackInsert(session.getUser(), result)); return result; } @@ -758,7 +758,9 @@ public void finishInsert(ConnectorSession session, ConnectorInsertTableHandle in if (!partitionUpdate.getName().isEmpty() && partitionUpdate.isNew()) { // move data to final location if (!partitionUpdate.getWritePath().equals(partitionUpdate.getTargetPath())) { - renameDirectory(hdfsEnvironment, + renameDirectory( + session.getUser(), + hdfsEnvironment, table.get().getDbName(), table.get().getTableName(), new Path(partitionUpdate.getWritePath()), @@ -779,7 +781,7 @@ public void finishInsert(ConnectorSession session, ConnectorInsertTableHandle in FileSystem fileSystem; try { - fileSystem = hdfsEnvironment.getFileSystem(targetDir); + fileSystem = hdfsEnvironment.getFileSystem(session.getUser(), targetDir); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, e); @@ -807,7 +809,7 @@ public void finishInsert(ConnectorSession session, ConnectorInsertTableHandle in } catch (Throwable t) { partitionCommitter.abort(); - rollbackPartitionUpdates(partitionUpdates, "insert"); + rollbackPartitionUpdates(session.getUser(), partitionUpdates, "insert"); throw t; } @@ -838,14 +840,14 @@ private Partition createPartition(Table table, PartitionUpdate partitionUpdate) return partition; } - private void rollbackInsert(ConnectorInsertTableHandle insertHandle) + private void rollbackInsert(String user, ConnectorInsertTableHandle insertHandle) { HiveInsertTableHandle handle = checkType(insertHandle, HiveInsertTableHandle.class, "invalid insertHandle"); // if there is a temp directory, we only need to cleanup temp files in this directory Optional writePath = locationService.writePathRoot(handle.getLocationHandle()); if (writePath.isPresent()) { - cleanupTempDirectory(writePath.get().toString(), handle.getFilePrefix(), "insert"); + cleanupTempDirectory(user, writePath.get().toString(), handle.getFilePrefix(), "insert"); // Note: in this case there is no need to cleanup the target directory as it will only // be written to during the commit call and the commit call cleans up after failures. return; @@ -879,7 +881,7 @@ private void rollbackInsert(ConnectorInsertTableHandle insertHandle) // delete any file that starts with the unique prefix of this query List notDeletedFiles = new ArrayList<>(); for (String location : locationsToClean) { - notDeletedFiles.addAll(recursiveDeleteFilesStartingWith(location, handle.getFilePrefix())); + notDeletedFiles.addAll(recursiveDeleteFilesStartingWith(user, location, handle.getFilePrefix())); } if (!notDeletedFiles.isEmpty()) { log.error("Cannot delete insert data files %s", notDeletedFiles); @@ -888,23 +890,23 @@ private void rollbackInsert(ConnectorInsertTableHandle insertHandle) // Note: we can not delete any of these locations since we do not know who created them } - private void cleanupTempDirectory(String location, String filePrefix, String actionName) + private void cleanupTempDirectory(String user, String location, String filePrefix, String actionName) { // to be safe only delete files that start with the unique prefix for this query - List notDeletedFiles = recursiveDeleteFilesStartingWith(location, filePrefix); + List notDeletedFiles = recursiveDeleteFilesStartingWith(user, location, filePrefix); if (!notDeletedFiles.isEmpty()) { log.warn("Error rolling back " + actionName + " temporary data files %s", notDeletedFiles.stream() .collect(joining(", "))); } // try to delete the temp directory - if (!deleteIfExists(location)) { + if (!deleteIfExists(user, location)) { // this is temp data so an error isn't a big problem log.debug("Error deleting " + actionName + " temp data in %s", location); } } - private void rollbackPartitionUpdates(List partitionUpdates, String actionName) + private void rollbackPartitionUpdates(String user, List partitionUpdates, String actionName) { for (PartitionUpdate partitionUpdate : partitionUpdates) { String targetPath = partitionUpdate.getTargetPath(); @@ -913,21 +915,21 @@ private void rollbackPartitionUpdates(List partitionUpdates, St // delete temp data if we used a temp dir if (!writePath.equals(targetPath)) { // to be safe only delete the files we know we created in the temp directory - List notDeletedFiles = deleteFilesFrom(writePath, partitionUpdate.getFileNames()); + List notDeletedFiles = deleteFilesFrom(user, writePath, partitionUpdate.getFileNames()); if (!notDeletedFiles.isEmpty()) { log.warn("Error rolling back " + actionName + " temporary data files %s", notDeletedFiles.stream() .collect(joining(", "))); } // try to delete the temp directory - if (!deleteIfExists(writePath)) { + if (!deleteIfExists(user, writePath)) { // this is temp data so an error isn't a big problem log.debug("Error deleting " + actionName + " temp data in %s", writePath); } } // delete data from target directory - List notDeletedFiles = deleteFilesFrom(targetPath, partitionUpdate.getFileNames()); + List notDeletedFiles = deleteFilesFrom(user, targetPath, partitionUpdate.getFileNames()); if (!notDeletedFiles.isEmpty()) { log.error("Error rolling back " + actionName + " data files %s", notDeletedFiles.stream() .collect(joining(", "))); @@ -935,7 +937,7 @@ private void rollbackPartitionUpdates(List partitionUpdates, St // only try to delete directory if the partition is new if (partitionUpdate.isNew()) { - if (!deleteIfExists(targetPath)) { + if (!deleteIfExists(user, targetPath)) { log.debug("Cannot delete " + actionName + " directory %s", targetPath); } } @@ -946,13 +948,13 @@ private void rollbackPartitionUpdates(List partitionUpdates, St * Attempts to remove the file or empty directory. * @return true if the location no longer exists */ - public boolean deleteIfExists(String location) + public boolean deleteIfExists(String user, String location) { Path path = new Path(location); FileSystem fileSystem; try { - fileSystem = hdfsEnvironment.getFileSystem(path); + fileSystem = hdfsEnvironment.getFileSystem(user, path); } catch (IOException ignored) { return false; @@ -990,12 +992,12 @@ private static boolean deleteIfExists(FileSystem fileSystem, Path path) * Attempt to remove the {@code fileNames} files within {@code location}. * @return the files that could not be removed */ - private List deleteFilesFrom(String location, List fileNames) + private List deleteFilesFrom(String user, String location, List fileNames) { Path directory = new Path(location); FileSystem fileSystem; try { - fileSystem = hdfsEnvironment.getFileSystem(directory); + fileSystem = hdfsEnvironment.getFileSystem(user, directory); } catch (IOException e) { return fileNames; @@ -1015,12 +1017,12 @@ private List deleteFilesFrom(String location, List fileNames) * Attempt to remove all files in all directories within {@code location} that start with the {@code filePrefix}. * @return the files starting with the {@code filePrefix} that could not be removed */ - private List recursiveDeleteFilesStartingWith(String location, String filePrefix) + private List recursiveDeleteFilesStartingWith(String user, String location, String filePrefix) { FileSystem fileSystem; try { Path directory = new Path(location); - fileSystem = hdfsEnvironment.getFileSystem(directory); + fileSystem = hdfsEnvironment.getFileSystem(user, directory); } catch (IOException e) { return ImmutableList.of(location + "/" + filePrefix + "*"); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java index 159040d8143d0..07684a46d2c50 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java @@ -16,6 +16,7 @@ import com.facebook.presto.hive.HiveWriteUtils.FieldSetter; import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.spi.ConnectorPageSink; +import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PageIndexer; import com.facebook.presto.spi.PageIndexerFactory; @@ -147,6 +148,8 @@ public class HivePageSink private final List> bucketWriters; private int bucketWriterCount = 0; + private final ConnectorSession session; + public HivePageSink( String schemaName, String tableName, @@ -165,7 +168,8 @@ public HivePageSink( int maxOpenPartitions, boolean immutablePartitions, boolean compress, - JsonCodec partitionUpdateCodec) + JsonCodec partitionUpdateCodec, + ConnectorSession session) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -273,6 +277,8 @@ public HivePageSink( Path hdfsEnvironmentPath = locationService.writePathRoot(locationHandle).orElseGet(() -> locationService.targetPathRoot(locationHandle)); conf = new JobConf(hdfsEnvironment.getConfiguration(hdfsEnvironmentPath)); } + + this.session = requireNonNull(session, "session is null"); } @Override @@ -435,7 +441,7 @@ private HiveRecordWriter createWriter(List partitionRow, String fileName if (partitionName.isPresent() && !target.equals(write)) { // When target path is different from write path, // verify that the target directory for the partition does not already exist - if (HiveWriteUtils.pathExists(hdfsEnvironment, target)) { + if (HiveWriteUtils.pathExists(session.getUser(), hdfsEnvironment, target)) { throw new PrestoException(HIVE_PATH_ALREADY_EXISTS, format("Target directory for new partition '%s' of table '%s.%s' already exists: %s", partitionName, schemaName, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java index 0f5d6a7454042..03ad8a0399aef 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java @@ -67,17 +67,17 @@ public HivePageSinkProvider( public ConnectorPageSink createPageSink(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorOutputTableHandle tableHandle) { HiveWritableTableHandle handle = checkType(tableHandle, HiveOutputTableHandle.class, "tableHandle"); - return createPageSink(handle, true); + return createPageSink(handle, true, session); } @Override public ConnectorPageSink createPageSink(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorInsertTableHandle tableHandle) { HiveInsertTableHandle handle = checkType(tableHandle, HiveInsertTableHandle.class, "tableHandle"); - return createPageSink(handle, false); + return createPageSink(handle, false, session); } - private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable) + private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable, ConnectorSession session) { return new HivePageSink( handle.getSchemaName(), @@ -97,6 +97,7 @@ private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean maxOpenPartitions, immutablePartitions, compressed, - partitionUpdateCodec); + partitionUpdateCodec, + session); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index df2eadc32a18b..927a397974845 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -30,7 +30,6 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.VarbinaryType; import com.facebook.presto.spi.type.VarcharType; -import com.google.common.base.StandardSystemProperty; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; import org.apache.hadoop.fs.Path; @@ -341,7 +340,7 @@ private static void checkWritable( } } - public static Path getTableDefaultLocation(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) + public static Path getTableDefaultLocation(String user, HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) { String location = getDatabase(metastore, schemaName).getLocationUri(); if (isNullOrEmpty(location)) { @@ -349,10 +348,10 @@ public static Path getTableDefaultLocation(HiveMetastore metastore, HdfsEnvironm } Path databasePath = new Path(location); - if (!pathExists(hdfsEnvironment, databasePath)) { + if (!pathExists(user, hdfsEnvironment, databasePath)) { throw new PrestoException(HIVE_DATABASE_LOCATION_ERROR, format("Database '%s' location does not exist: %s", schemaName, databasePath)); } - if (!isDirectory(hdfsEnvironment, databasePath)) { + if (!isDirectory(user, hdfsEnvironment, databasePath)) { throw new PrestoException(HIVE_DATABASE_LOCATION_ERROR, format("Database '%s' location is not a directory: %s", schemaName, databasePath)); } @@ -364,39 +363,39 @@ private static Database getDatabase(HiveMetastore metastore, String database) return metastore.getDatabase(database).orElseThrow(() -> new SchemaNotFoundException(database)); } - public static boolean pathExists(HdfsEnvironment hdfsEnvironment, Path path) + public static boolean pathExists(String user, HdfsEnvironment hdfsEnvironment, Path path) { try { - return hdfsEnvironment.getFileSystem(path).exists(path); + return hdfsEnvironment.getFileSystem(user, path).exists(path); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); } } - private static boolean isDirectory(HdfsEnvironment hdfsEnvironment, Path path) + private static boolean isDirectory(String user, HdfsEnvironment hdfsEnvironment, Path path) { try { - return hdfsEnvironment.getFileSystem(path).isDirectory(path); + return hdfsEnvironment.getFileSystem(user, path).isDirectory(path); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); } } - public static void renameDirectory(HdfsEnvironment hdfsEnvironment, String schemaName, String tableName, Path source, Path target) + public static void renameDirectory(String user, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName, Path source, Path target) { - if (pathExists(hdfsEnvironment, target)) { + if (pathExists(user, hdfsEnvironment, target)) { throw new PrestoException(HIVE_PATH_ALREADY_EXISTS, format("Unable to commit creation of table '%s.%s': target directory already exists: %s", schemaName, tableName, target)); } - if (!pathExists(hdfsEnvironment, target.getParent())) { - createDirectory(hdfsEnvironment, target.getParent()); + if (!pathExists(user, hdfsEnvironment, target.getParent())) { + createDirectory(user, hdfsEnvironment, target.getParent()); } try { - if (!hdfsEnvironment.getFileSystem(source).rename(source, target)) { + if (!hdfsEnvironment.getFileSystem(user, source).rename(source, target)) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, format("Failed to rename %s to %s: rename returned false", source, target)); } } @@ -405,25 +404,24 @@ public static void renameDirectory(HdfsEnvironment hdfsEnvironment, String schem } } - public static Path createTemporaryPath(HdfsEnvironment hdfsEnvironment, Path targetPath) + public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironment, Path targetPath) { // use a per-user temporary directory to avoid permission problems - // TODO: this should use Hadoop UserGroupInformation - String temporaryPrefix = "/tmp/presto-" + StandardSystemProperty.USER_NAME.value(); + String temporaryPrefix = "/tmp/presto-" + user; // create a temporary directory on the same filesystem Path temporaryRoot = new Path(targetPath, temporaryPrefix); Path temporaryPath = new Path(temporaryRoot, randomUUID().toString()); - createDirectory(hdfsEnvironment, temporaryPath); + createDirectory(user, hdfsEnvironment, temporaryPath); return temporaryPath; } - public static void createDirectory(HdfsEnvironment hdfsEnvironment, Path path) + public static void createDirectory(String user, HdfsEnvironment hdfsEnvironment, Path path) { try { - if (!hdfsEnvironment.getFileSystem(path).mkdirs(path, ALL_PERMISSIONS)) { + if (!hdfsEnvironment.getFileSystem(user, path).mkdirs(path, ALL_PERMISSIONS)) { throw new IOException("mkdirs returned false"); } } @@ -433,7 +431,7 @@ public static void createDirectory(HdfsEnvironment hdfsEnvironment, Path path) // explicitly set permission since the default umask overrides it on creation try { - hdfsEnvironment.getFileSystem(path).setPermission(path, ALL_PERMISSIONS); + hdfsEnvironment.getFileSystem(user, path).setPermission(path, ALL_PERMISSIONS); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed to set permission on directory: " + path, e); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/LocationService.java b/presto-hive/src/main/java/com/facebook/presto/hive/LocationService.java index 60d14615f56f9..256d670f5b379 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/LocationService.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/LocationService.java @@ -21,9 +21,9 @@ public interface LocationService { - LocationHandle forNewTable(String queryId, String schemaName, String tableName); + LocationHandle forNewTable(String user, String queryId, String schemaName, String tableName); - LocationHandle forExistingTable(String queryId, Table table); + LocationHandle forExistingTable(String user, String queryId, Table table); /** * Target path for the specified existing partition. diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index dab73794ec922..42870f46a3126 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -15,6 +15,7 @@ import com.facebook.presto.GroupByHashPageIndexerFactory; import com.facebook.presto.hadoop.HadoopFileStatus; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.metastore.CachingHiveMetastore; import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.hive.orc.OrcPageSource; @@ -437,7 +438,7 @@ protected final void setup(String host, int port, String databaseName, String ti HiveConnectorId connectorId = new HiveConnectorId(connectorName); HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationUpdater(hiveClientConfig)); - hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hiveClientConfig); + hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hiveClientConfig, new NoHdfsAuthentication()); locationService = new HiveLocationService(metastoreClient, hdfsEnvironment); TypeManager typeManager = new TypeRegistry(); JsonCodec partitionUpdateCodec = JsonCodec.jsonCodec(PartitionUpdate.class); @@ -1815,7 +1816,7 @@ protected Set listAllDataFiles(Path path) throws IOException { Set result = new HashSet<>(); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(path); + FileSystem fileSystem = hdfsEnvironment.getFileSystem("user", path); if (fileSystem.exists(path)) { for (FileStatus fileStatus : fileSystem.listStatus(path)) { if (HadoopFileStatus.isFile(fileStatus)) { @@ -2575,9 +2576,9 @@ protected void createEmptyTable(SchemaTableName schemaTableName, HiveStorageForm String tableName = schemaTableName.getTableName(); LocationService locationService = getLocationService(schemaName); - LocationHandle locationHandle = locationService.forNewTable(session.getQueryId(), schemaName, tableName); + LocationHandle locationHandle = locationService.forNewTable(session.getUser(), session.getQueryId(), schemaName, tableName); Path targetPath = locationService.targetPathRoot(locationHandle); - HiveWriteUtils.createDirectory(hdfsEnvironment, targetPath); + HiveWriteUtils.createDirectory(session.getUser(), hdfsEnvironment, targetPath); SerDeInfo serdeInfo = new SerDeInfo(); serdeInfo.setName(tableName); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java index b888aa278664c..f9b82a911c31f 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive; import com.facebook.presto.GroupByHashPageIndexerFactory; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.metastore.CachingHiveMetastore; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.facebook.presto.spi.ColumnHandle; @@ -148,7 +149,7 @@ protected void setup(String host, int port, String databaseName, String awsAcces HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationUpdater(hiveClientConfig)); HivePartitionManager hivePartitionManager = new HivePartitionManager(connectorId, TYPE_MANAGER, hiveClientConfig); - hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hiveClientConfig); + hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, hiveClientConfig, new NoHdfsAuthentication()); metastoreClient = new TestingHiveMetastore(hiveCluster, executor, hiveClientConfig, writableBucket, hdfsEnvironment); locationService = new HiveLocationService(metastoreClient, hdfsEnvironment); TypeRegistry typeManager = new TypeRegistry(); @@ -213,7 +214,7 @@ public void testGetFileStatus() Path basePath = new Path("s3://presto-test-hive/"); Path tablePath = new Path(basePath, "presto_test_s3"); Path filePath = new Path(tablePath, "test1.csv"); - FileSystem fs = hdfsEnvironment.getFileSystem(basePath); + FileSystem fs = hdfsEnvironment.getFileSystem("user", basePath); assertTrue(isDirectory(fs.getFileStatus(basePath))); assertTrue(isDirectory(fs.getFileStatus(tablePath))); @@ -226,7 +227,7 @@ public void testRename() throws Exception { Path basePath = new Path(format("s3://%s/rename/%s/", writableBucket, UUID.randomUUID())); - FileSystem fs = hdfsEnvironment.getFileSystem(basePath); + FileSystem fs = hdfsEnvironment.getFileSystem("user", basePath); assertFalse(fs.exists(basePath)); // create file foo.txt @@ -448,7 +449,7 @@ public void dropTable(String databaseName, String tableName) // drop data for (String location : locations) { Path path = new Path(location); - hdfsEnvironment.getFileSystem(path).delete(path, true); + hdfsEnvironment.getFileSystem("user", path).delete(path, true); } } catch (Exception e) { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java index 2c456b56dee8c..a6857643912c7 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive; import com.facebook.presto.GroupByHashPageIndexerFactory; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.hive.metastore.InMemoryHiveMetastore; import com.facebook.presto.spi.ConnectorPageSink; @@ -257,6 +258,6 @@ private static HiveType getHiveType(TpchColumnType type) private static HdfsEnvironment createHdfsEnvironment(HiveClientConfig config) { - return new HdfsEnvironment(new HiveHdfsConfiguration(new HdfsConfigurationUpdater(config)), config); + return new HdfsEnvironment(new HiveHdfsConfiguration(new HdfsConfigurationUpdater(config)), config, new NoHdfsAuthentication()); } } From ce90e472ea02ceaf9bd55de01b85fd1ca50c86b9 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 13:51:57 +0100 Subject: [PATCH 08/20] Wrap HivePageSink methods into HdfsAuthention --- .../facebook/presto/hive/HdfsEnvironment.java | 12 +++++++++++ .../facebook/presto/hive/HivePageSink.java | 21 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java index 2eda6669d8fee..424c4a063ff0f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java @@ -15,6 +15,7 @@ import com.facebook.presto.hadoop.HadoopFileSystemCache; import com.facebook.presto.hadoop.HadoopNative; +import com.facebook.presto.hive.authentication.GenericExceptionAction; import com.facebook.presto.hive.authentication.HdfsAuthentication; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -62,4 +63,15 @@ public FileSystem getFileSystem(String user, Path path) return fileSystem; }); } + + public R doAs(String user, GenericExceptionAction action) + throws E + { + return hdfsAuthentication.doAs(user, action); + } + + public void doAs(String user, Runnable action) + { + hdfsAuthentication.doAs(user, action); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java index 07684a46d2c50..bd53c91740322 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java @@ -283,6 +283,13 @@ public HivePageSink( @Override public Collection finish() + { + // Must be wrapped in doAs entirely + // Implicit FileSystem initializations are possible in HiveRecordWriter#commit -> RecordWriter#close + return hdfsEnvironment.doAs(session.getUser(), this::doFinish); + } + + private ImmutableList doFinish() { ImmutableList.Builder partitionUpdates = ImmutableList.builder(); if (!bucketCount.isPresent()) { @@ -308,6 +315,13 @@ public Collection finish() @Override public void abort() + { + // Must be wrapped in doAs entirely + // Implicit FileSystem initializations are possible in HiveRecordWriter#rollback -> RecordWriter#close + hdfsEnvironment.doAs(session.getUser(), this::doAbort); + } + + private void doAbort() { if (!bucketCount.isPresent()) { for (HiveRecordWriter writer : writers) { @@ -340,6 +354,13 @@ public CompletableFuture appendPage(Page page, Block sampleWeightBlock) throw new PrestoException(HIVE_TOO_MANY_OPEN_PARTITIONS, "Too many open partitions"); } + // Must be wrapped in doAs entirely + // Implicit FileSystem initializations are possible in HiveRecordWriter#addRow or #createWriter + return hdfsEnvironment.doAs(session.getUser(), () -> doAppend(page, dataBlocks, partitionBlocks, indexes)); + } + + private CompletableFuture doAppend(Page page, Block[] dataBlocks, Block[] partitionBlocks, int[] indexes) + { if (!bucketCount.isPresent()) { if (pageIndexer.getMaxIndex() >= writers.length) { writers = Arrays.copyOf(writers, pageIndexer.getMaxIndex() + 1); From 41df9be9f821a92ee948f16afea2979e6ff42129 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 15:18:01 +0100 Subject: [PATCH 09/20] Retrieve FileSystem from HdfsEnvironment Use HdfsEnvironment.getFileSystem in custom readers instead of plain FileSystem.get(). --- .../facebook/presto/hive/HdfsEnvironment.java | 8 ++++- .../hive/orc/DwrfPageSourceFactory.java | 7 +++- .../presto/hive/orc/OrcPageSourceFactory.java | 18 +++++++--- .../hive/parquet/HdfsParquetDataSource.java | 4 +-- .../hive/parquet/ParquetHiveRecordCursor.java | 23 +++++++++++-- .../parquet/ParquetPageSourceFactory.java | 27 +++++++++++---- .../parquet/ParquetRecordCursorProvider.java | 11 +++++-- .../parquet/reader/ParquetMetadataReader.java | 4 +-- .../hive/rcfile/RcFilePageSourceFactory.java | 7 ++-- .../facebook/presto/hive/HiveTestUtils.java | 18 +++++++--- .../presto/hive/TestHiveFileFormats.java | 33 ++++++++++--------- .../presto/hive/TestHivePageSink.java | 11 ++----- .../hive/TestOrcPageSourceMemoryTracking.java | 3 +- .../presto/hive/benchmark/FileFormat.java | 30 +++++++++-------- .../benchmark/HiveFileFormatBenchmark.java | 12 +++++-- .../presto/hive/parquet/ParquetTester.java | 4 +-- .../tests/hive/TestHiveStorageFormats.java | 6 ++-- 17 files changed, 148 insertions(+), 78 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java index 424c4a063ff0f..c95eec37c7f9f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HdfsEnvironment.java @@ -56,9 +56,15 @@ public Configuration getConfiguration(Path path) public FileSystem getFileSystem(String user, Path path) throws IOException + { + return getFileSystem(user, path, getConfiguration(path)); + } + + public FileSystem getFileSystem(String user, Path path, Configuration configuration) + throws IOException { return hdfsAuthentication.doAs(user, () -> { - FileSystem fileSystem = path.getFileSystem(getConfiguration(path)); + FileSystem fileSystem = path.getFileSystem(configuration); fileSystem.setVerifyChecksum(verifyChecksum); return fileSystem; }); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java index 585534d2e72d2..a1f37bc9e67d9 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/DwrfPageSourceFactory.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive.orc; import com.facebook.hive.orc.OrcSerde; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePageSourceFactory; import com.facebook.presto.hive.HivePartitionKey; @@ -43,11 +44,13 @@ public class DwrfPageSourceFactory implements HivePageSourceFactory { private final TypeManager typeManager; + private final HdfsEnvironment hdfsEnvironment; @Inject - public DwrfPageSourceFactory(TypeManager typeManager) + public DwrfPageSourceFactory(TypeManager typeManager, HdfsEnvironment hdfsEnvironment) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); } @Override @@ -68,6 +71,8 @@ public Optional createPageSource(Configuration co return Optional.of(createOrcPageSource( new DwrfMetadataReader(), + hdfsEnvironment, + session.getUser(), configuration, path, start, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java index 96b52aa3d19dd..48b5f66923101 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcPageSourceFactory.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.orc; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveClientConfig; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePageSourceFactory; @@ -69,17 +70,19 @@ public class OrcPageSourceFactory private static final Pattern DEFAULT_HIVE_COLUMN_NAME_PATTERN = Pattern.compile("_col\\d+"); private final TypeManager typeManager; private final boolean useOrcColumnNames; + private final HdfsEnvironment hdfsEnvironment; @Inject - public OrcPageSourceFactory(TypeManager typeManager, HiveClientConfig config) + public OrcPageSourceFactory(TypeManager typeManager, HiveClientConfig config, HdfsEnvironment hdfsEnvironment) { - this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseOrcColumnNames()); + this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseOrcColumnNames(), hdfsEnvironment); } - public OrcPageSourceFactory(TypeManager typeManager, boolean useOrcColumnNames) + public OrcPageSourceFactory(TypeManager typeManager, boolean useOrcColumnNames, HdfsEnvironment hdfsEnvironment) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.useOrcColumnNames = useOrcColumnNames; + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); } @Override @@ -101,6 +104,8 @@ public Optional createPageSource( return Optional.of(createOrcPageSource( new OrcMetadataReader(), + hdfsEnvironment, + session.getUser(), configuration, path, start, @@ -116,7 +121,10 @@ public Optional createPageSource( getOrcStreamBufferSize(session))); } - public static OrcPageSource createOrcPageSource(MetadataReader metadataReader, + public static OrcPageSource createOrcPageSource( + MetadataReader metadataReader, + HdfsEnvironment hdfsEnvironment, + String sessionUser, Configuration configuration, Path path, long start, @@ -133,7 +141,7 @@ public static OrcPageSource createOrcPageSource(MetadataReader metadataReader, { OrcDataSource orcDataSource; try { - FileSystem fileSystem = path.getFileSystem(configuration); + FileSystem fileSystem = hdfsEnvironment.getFileSystem(sessionUser, path, configuration); long size = fileSystem.getFileStatus(path).getLen(); FSDataInputStream inputStream = fileSystem.open(path); orcDataSource = new HdfsOrcDataSource(path.toString(), size, maxMergeDistance, maxBufferSize, streamBufferSize, inputStream); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java index 681dc385e6380..d01a4f25f1a82 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive.parquet; import com.facebook.presto.spi.PrestoException; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -91,10 +90,9 @@ private void readInternal(long position, byte[] buffer, int bufferOffset, int bu } } - public static HdfsParquetDataSource buildHdfsParquetDataSource(Path path, Configuration configuration, long start, long length) + public static HdfsParquetDataSource buildHdfsParquetDataSource(FileSystem fileSystem, Path path, long start, long length) { try { - FileSystem fileSystem = path.getFileSystem(configuration); long size = fileSystem.getFileStatus(path).getLen(); FSDataInputStream inputStream = fileSystem.open(path); return new HdfsParquetDataSource(path, size, inputStream); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java index d3b203ffef214..6a98c7c5e67f8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.parquet; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePartitionKey; import com.facebook.presto.hive.HiveRecordCursor; @@ -32,6 +33,7 @@ import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; @@ -134,6 +136,8 @@ public class ParquetHiveRecordCursor private boolean closed; public ParquetHiveRecordCursor( + HdfsEnvironment hdfsEnvironment, + String sessionUser, Configuration configuration, Path path, long start, @@ -230,6 +234,8 @@ else if (isLongDecimal(type)) { } this.recordReader = createParquetRecordReader( + hdfsEnvironment, + sessionUser, configuration, path, start, @@ -380,6 +386,8 @@ public void close() } private ParquetRecordReader createParquetRecordReader( + HdfsEnvironment hdfsEnvironment, + String sessionUser, Configuration configuration, Path path, long start, @@ -390,7 +398,10 @@ private ParquetRecordReader createParquetRecordReader( boolean predicatePushdownEnabled, TupleDomain effectivePredicate) { - try (ParquetDataSource dataSource = buildHdfsParquetDataSource(path, configuration, start, length)) { + ParquetDataSource dataSource = null; + try { + FileSystem fileSystem = hdfsEnvironment.getFileSystem(sessionUser, path, configuration); + dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length); ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(configuration, path, NO_FILTER); List blocks = parquetMetadata.getBlocks(); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); @@ -416,8 +427,9 @@ private ParquetRecordReader createParquetRecordReader( if (predicatePushdownEnabled) { ParquetPredicate parquetPredicate = buildParquetPredicate(columns, effectivePredicate, fileMetaData.getSchema(), typeManager); + ParquetDataSource finalDataSource = dataSource; splitGroup = splitGroup.stream() - .filter(block -> predicateMatches(parquetPredicate, block, dataSource, requestedSchema, effectivePredicate)) + .filter(block -> predicateMatches(parquetPredicate, block, finalDataSource, requestedSchema, effectivePredicate)) .collect(toList()); } @@ -435,6 +447,13 @@ private ParquetRecordReader createParquetRecordReader( return realReader; } catch (Exception e) { + if (dataSource != null) { + try { + dataSource.close(); + } + catch (IOException ignored) { + } + } if (e instanceof PrestoException) { throw (PrestoException) e; } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java index b35b4df2ff3af..720ceee1f0dda 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.parquet; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveClientConfig; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePageSourceFactory; @@ -28,6 +29,7 @@ import com.facebook.presto.spi.type.TypeSignature; import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.joda.time.DateTimeZone; import parquet.hadoop.metadata.BlockMetaData; @@ -78,17 +80,19 @@ public class ParquetPageSourceFactory private final TypeManager typeManager; private final boolean useParquetColumnNames; + private final HdfsEnvironment hdfsEnvironment; @Inject - public ParquetPageSourceFactory(TypeManager typeManager, HiveClientConfig config) + public ParquetPageSourceFactory(TypeManager typeManager, HiveClientConfig config, HdfsEnvironment hdfsEnvironment) { - this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseParquetColumnNames()); + this(typeManager, requireNonNull(config, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment); } - public ParquetPageSourceFactory(TypeManager typeManager, boolean useParquetColumnNames) + public ParquetPageSourceFactory(TypeManager typeManager, boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.useParquetColumnNames = useParquetColumnNames; + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); } @Override @@ -117,6 +121,8 @@ public Optional createPageSource( } return Optional.of(createParquetPageSource( + hdfsEnvironment, + session.getUser(), configuration, path, start, @@ -132,6 +138,8 @@ public Optional createPageSource( } public static ParquetPageSource createParquetPageSource( + HdfsEnvironment hdfsEnvironment, + String user, Configuration configuration, Path path, long start, @@ -145,9 +153,11 @@ public static ParquetPageSource createParquetPageSource( boolean predicatePushdownEnabled, TupleDomain effectivePredicate) { - ParquetDataSource dataSource = buildHdfsParquetDataSource(path, configuration, start, length); + ParquetDataSource dataSource = null; try { - ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(configuration, path); + FileSystem fileSystem = hdfsEnvironment.getFileSystem(user, path, configuration); + dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length); + ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); @@ -169,8 +179,9 @@ public static ParquetPageSource createParquetPageSource( if (predicatePushdownEnabled) { ParquetPredicate parquetPredicate = buildParquetPredicate(columns, effectivePredicate, fileMetaData.getSchema(), typeManager); + final ParquetDataSource finalDataSource = dataSource; blocks = blocks.stream() - .filter(block -> predicateMatches(parquetPredicate, block, dataSource, requestedSchema, effectivePredicate)) + .filter(block -> predicateMatches(parquetPredicate, block, finalDataSource, requestedSchema, effectivePredicate)) .collect(toList()); } @@ -198,7 +209,9 @@ public static ParquetPageSource createParquetPageSource( } catch (Exception e) { try { - dataSource.close(); + if (dataSource != null) { + dataSource.close(); + } } catch (IOException ignored) { } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java index ac6db05af74f7..c40399bac8926 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetRecordCursorProvider.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.parquet; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveClientConfig; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePartitionKey; @@ -46,16 +47,18 @@ public class ParquetRecordCursorProvider .build(); private final boolean useParquetColumnNames; + private final HdfsEnvironment hdfsEnvironment; @Inject - public ParquetRecordCursorProvider(HiveClientConfig hiveClientConfig) + public ParquetRecordCursorProvider(HiveClientConfig hiveClientConfig, HdfsEnvironment hdfsEnvironment) { - this(requireNonNull(hiveClientConfig, "hiveClientConfig is null").isUseParquetColumnNames()); + this(requireNonNull(hiveClientConfig, "hiveClientConfig is null").isUseParquetColumnNames(), hdfsEnvironment); } - public ParquetRecordCursorProvider(boolean useParquetColumnNames) + public ParquetRecordCursorProvider(boolean useParquetColumnNames, HdfsEnvironment hdfsEnvironment) { this.useParquetColumnNames = useParquetColumnNames; + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); } @Override @@ -78,6 +81,8 @@ public Optional createHiveRecordCursor( } return Optional.of(new ParquetHiveRecordCursor( + hdfsEnvironment, + session.getUser(), configuration, path, start, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java index 0ff85b243bbd1..1b9a249f0f9a1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetMetadataReader.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.hive.parquet.reader; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -63,10 +62,9 @@ public final class ParquetMetadataReader private ParquetMetadataReader() {} - public static ParquetMetadata readFooter(Configuration configuration, Path file) + public static ParquetMetadata readFooter(FileSystem fileSystem, Path file) throws IOException { - FileSystem fileSystem = file.getFileSystem(configuration); FileStatus fileStatus = fileSystem.getFileStatus(file); try (FSDataInputStream inputStream = fileSystem.open(file)) { // Parquet File Layout: diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java index 22d166108a8cc..45cf435d7bdfa 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive.rcfile; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HivePageSourceFactory; import com.facebook.presto.hive.HivePartitionKey; @@ -47,11 +48,13 @@ public class RcFilePageSourceFactory implements HivePageSourceFactory { private final TypeManager typeManager; + private final HdfsEnvironment hdfsEnvironment; @Inject - public RcFilePageSourceFactory(TypeManager typeManager) + public RcFilePageSourceFactory(TypeManager typeManager, HdfsEnvironment hdfsEnvironment) { this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); } @Override @@ -101,7 +104,7 @@ else if (deserializerClassName.equals(ColumnarSerDe.class.getName())) { RCFile.Reader recordReader; try { - FileSystem fileSystem = path.getFileSystem(configuration); + FileSystem fileSystem = hdfsEnvironment.getFileSystem(session.getUser(), path, configuration); recordReader = new RCFile.Reader(fileSystem, path, configuration); } catch (Exception e) { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java index a030844a6523f..2449bc2ef2274 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.orc.DwrfPageSourceFactory; import com.facebook.presto.hive.orc.OrcPageSourceFactory; import com.facebook.presto.hive.parquet.ParquetRecordCursorProvider; @@ -39,19 +40,23 @@ private HiveTestUtils() public static final TypeRegistry TYPE_MANAGER = new TypeRegistry(); + public static final HdfsEnvironment HDFS_ENVIRONMENT = createTestHdfsEnvironment(new HiveClientConfig()); + public static Set getDefaultHiveDataStreamFactories(HiveClientConfig hiveClientConfig) { + HdfsEnvironment testHdfsEnvironment = createTestHdfsEnvironment(hiveClientConfig); return ImmutableSet.builder() - .add(new RcFilePageSourceFactory(TYPE_MANAGER)) - .add(new OrcPageSourceFactory(TYPE_MANAGER, hiveClientConfig)) - .add(new DwrfPageSourceFactory(TYPE_MANAGER)) + .add(new RcFilePageSourceFactory(TYPE_MANAGER, testHdfsEnvironment)) + .add(new OrcPageSourceFactory(TYPE_MANAGER, hiveClientConfig, testHdfsEnvironment)) + .add(new DwrfPageSourceFactory(TYPE_MANAGER, testHdfsEnvironment)) .build(); } public static Set getDefaultHiveRecordCursorProvider(HiveClientConfig hiveClientConfig) { + HdfsEnvironment testHdfsEnvironment = createTestHdfsEnvironment(hiveClientConfig); return ImmutableSet.builder() - .add(new ParquetRecordCursorProvider(hiveClientConfig)) + .add(new ParquetRecordCursorProvider(hiveClientConfig, testHdfsEnvironment)) .add(new ColumnarTextHiveRecordCursorProvider()) .add(new ColumnarBinaryHiveRecordCursorProvider()) .add(new GenericHiveRecordCursorProvider()) @@ -66,4 +71,9 @@ public static List getTypes(List columnHandles) } return types.build(); } + + public static HdfsEnvironment createTestHdfsEnvironment(HiveClientConfig config) + { + return new HdfsEnvironment(new HiveHdfsConfiguration(new HdfsConfigurationUpdater(config)), config, new NoHdfsAuthentication()); + } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java index 2333152ab23a9..3905795251214 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java @@ -67,6 +67,7 @@ import static com.facebook.presto.hive.HiveStorageFormat.RCTEXT; import static com.facebook.presto.hive.HiveStorageFormat.SEQUENCEFILE; import static com.facebook.presto.hive.HiveStorageFormat.TEXTFILE; +import static com.facebook.presto.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.HiveTestUtils.TYPE_MANAGER; import static com.facebook.presto.hive.HiveTestUtils.getTypes; @@ -135,7 +136,7 @@ public void testRcTextPageSource(int rowCount) assertThatFileFormat(RCTEXT) .withColumns(TEST_COLUMNS) .withRowsCount(rowCount) - .isReadableByPageSource(new RcFilePageSourceFactory(TYPE_MANAGER)); + .isReadableByPageSource(new RcFilePageSourceFactory(TYPE_MANAGER, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -162,7 +163,7 @@ public void testRcBinaryPageSource(int rowCount) assertThatFileFormat(RCBINARY) .withColumns(TEST_COLUMNS) .withRowsCount(rowCount) - .isReadableByPageSource(new RcFilePageSourceFactory(TYPE_MANAGER)); + .isReadableByPageSource(new RcFilePageSourceFactory(TYPE_MANAGER, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -172,7 +173,7 @@ public void testOrc(int rowCount) assertThatFileFormat(ORC) .withColumns(TEST_COLUMNS) .withRowsCount(rowCount) - .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false)); + .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -186,7 +187,7 @@ public void testOrcUseColumnNames(int rowCount) .withRowsCount(rowCount) .withReadColumns(Lists.reverse(TEST_COLUMNS)) .withSession(session) - .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, true)); + .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, true, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -197,7 +198,7 @@ public void testParquet(int rowCount) assertThatFileFormat(PARQUET) .withColumns(testColumns) .withRowsCount(rowCount) - .isReadableByRecordCursor(new ParquetRecordCursorProvider(false)); + .isReadableByRecordCursor(new ParquetRecordCursorProvider(false, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -210,7 +211,7 @@ public void testParquetCaseInsensitiveColumnLookup(int rowCount) .withWriteColumns(writeColumns) .withReadColumns(readColumns) .withRowsCount(rowCount) - .isReadableByRecordCursor(new ParquetRecordCursorProvider(true)); + .isReadableByRecordCursor(new ParquetRecordCursorProvider(true, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -227,7 +228,7 @@ public void testParquetPageSource(int rowCount) .withColumns(testColumns) .withSession(session) .withRowsCount(rowCount) - .isReadableByPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false)); + .isReadableByPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT)); } @Test(dataProvider = "rowCount") @@ -240,7 +241,7 @@ public void testParquetUseColumnNames(int rowCount) .withWriteColumns(writeColumns) .withReadColumns(readColumns) .withRowsCount(rowCount) - .isReadableByRecordCursor(new ParquetRecordCursorProvider(true)); + .isReadableByRecordCursor(new ParquetRecordCursorProvider(true, HDFS_ENVIRONMENT)); } private static List getTestColumnsSupportedByParquet() @@ -300,7 +301,7 @@ public void testParquetThrift(int rowCount) SerDe serde = new ParquetHiveSerDe(); File file = new File(this.getClass().getClassLoader().getResource("addressbook.parquet").getPath()); FileSplit split = new FileSplit(new Path(file.getAbsolutePath()), 0, file.length(), new String[0]); - HiveRecordCursorProvider cursorProvider = new ParquetRecordCursorProvider(false); + HiveRecordCursorProvider cursorProvider = new ParquetRecordCursorProvider(false, HDFS_ENVIRONMENT); testCursorProvider(cursorProvider, split, inputFormat, serde, testColumns, 1); } @@ -314,7 +315,7 @@ public void testDwrf(int rowCount) assertThatFileFormat(DWRF) .withColumns(testColumns) .withRowsCount(rowCount) - .isReadableByPageSource(new DwrfPageSourceFactory(TYPE_MANAGER)); + .isReadableByPageSource(new DwrfPageSourceFactory(TYPE_MANAGER, HDFS_ENVIRONMENT)); } @Test @@ -339,12 +340,12 @@ public void testTruncateVarcharColumn() assertThatFileFormat(ORC) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false)); + .isReadableByPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT)); assertThatFileFormat(PARQUET) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByRecordCursor(new ParquetRecordCursorProvider(false)); + .isReadableByRecordCursor(new ParquetRecordCursorProvider(false, HDFS_ENVIRONMENT)); TestingConnectorSession session = new TestingConnectorSession( new HiveSessionProperties(new HiveClientConfig().setParquetOptimizedReaderEnabled(true)).getSessionProperties()); @@ -352,7 +353,7 @@ public void testTruncateVarcharColumn() .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) .withSession(session) - .isReadableByPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false)); + .isReadableByPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT)); assertThatFileFormat(SEQUENCEFILE) .withWriteColumns(ImmutableList.of(writeColumn)) @@ -389,18 +390,18 @@ public void testFailForLongVarcharPartitionColumn() assertThatFileFormat(ORC) .withColumns(columns) - .isFailingForPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false), expectedErrorCode, expectedMessage); + .isFailingForPageSource(new OrcPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); assertThatFileFormat(PARQUET) .withColumns(columns) - .isFailingForRecordCursor(new ParquetRecordCursorProvider(false), expectedErrorCode, expectedMessage); + .isFailingForRecordCursor(new ParquetRecordCursorProvider(false, HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); TestingConnectorSession session = new TestingConnectorSession( new HiveSessionProperties(new HiveClientConfig().setParquetOptimizedReaderEnabled(true)).getSessionProperties()); assertThatFileFormat(PARQUET) .withColumns(columns) .withSession(session) - .isFailingForPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false), expectedErrorCode, expectedMessage); + .isFailingForPageSource(new ParquetPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); assertThatFileFormat(SEQUENCEFILE) .withColumns(columns) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java index a6857643912c7..15575fdd62a12 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHivePageSink.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive; import com.facebook.presto.GroupByHashPageIndexerFactory; -import com.facebook.presto.hive.authentication.NoHdfsAuthentication; import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.hive.metastore.InMemoryHiveMetastore; import com.facebook.presto.spi.ConnectorPageSink; @@ -51,6 +50,7 @@ import static com.facebook.presto.hive.HiveCompressionCodec.NONE; import static com.facebook.presto.hive.HiveTestUtils.TYPE_MANAGER; +import static com.facebook.presto.hive.HiveTestUtils.createTestHdfsEnvironment; import static com.facebook.presto.hive.HiveTestUtils.getDefaultHiveDataStreamFactories; import static com.facebook.presto.hive.HiveTestUtils.getDefaultHiveRecordCursorProvider; import static com.facebook.presto.hive.HiveType.HIVE_DATE; @@ -199,7 +199,7 @@ private static ConnectorPageSource createPageSource(HiveTransactionHandle transa splitProperties.setProperty("columns", Joiner.on(',').join(getColumnHandles().stream().map(HiveColumnHandle::getName).collect(toList()))); splitProperties.setProperty("columns.types", Joiner.on(',').join(getColumnHandles().stream().map(HiveColumnHandle::getHiveType).map(HiveType::getHiveTypeName).collect(toList()))); HiveSplit split = new HiveSplit(CLIENT_ID, SCHEMA_NAME, TABLE_NAME, "", "file:///" + outputFile.getAbsolutePath(), 0, outputFile.length(), splitProperties, ImmutableList.of(), ImmutableList.of(), OptionalInt.empty(), false, TupleDomain.all()); - HivePageSourceProvider provider = new HivePageSourceProvider(config, createHdfsEnvironment(config), getDefaultHiveRecordCursorProvider(config), getDefaultHiveDataStreamFactories(config), TYPE_MANAGER); + HivePageSourceProvider provider = new HivePageSourceProvider(config, createTestHdfsEnvironment(config), getDefaultHiveRecordCursorProvider(config), getDefaultHiveDataStreamFactories(config), TYPE_MANAGER); return provider.createPageSource(transaction, getSession(config), split, ImmutableList.copyOf(getColumnHandles())); } @@ -208,7 +208,7 @@ private static ConnectorPageSink createPageSink(HiveTransactionHandle transactio LocationHandle locationHandle = new LocationHandle(outputPath, Optional.of(outputPath), false); HiveOutputTableHandle handle = new HiveOutputTableHandle(CLIENT_ID, SCHEMA_NAME, TABLE_NAME, getColumnHandles(), "test", locationHandle, config.getHiveStorageFormat(), config.getHiveStorageFormat(), ImmutableList.of(), Optional.empty(), "test", ImmutableMap.of()); JsonCodec partitionUpdateCodec = JsonCodec.jsonCodec(PartitionUpdate.class); - HdfsEnvironment hdfsEnvironment = createHdfsEnvironment(config); + HdfsEnvironment hdfsEnvironment = createTestHdfsEnvironment(config); HivePageSinkProvider provider = new HivePageSinkProvider(hdfsEnvironment, metastore, new GroupByHashPageIndexerFactory(), TYPE_MANAGER, config, new HiveLocationService(metastore, hdfsEnvironment), partitionUpdateCodec); return provider.createPageSink(transaction, getSession(config), handle); } @@ -255,9 +255,4 @@ private static HiveType getHiveType(TpchColumnType type) throw new UnsupportedOperationException(); } } - - private static HdfsEnvironment createHdfsEnvironment(HiveClientConfig config) - { - return new HdfsEnvironment(new HiveHdfsConfiguration(new HdfsConfigurationUpdater(config)), config, new NoHdfsAuthentication()); - } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java index 56599edc1705b..8cb6350584748 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestOrcPageSourceMemoryTracking.java @@ -82,6 +82,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static com.facebook.presto.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.operator.ProjectionFunctions.singleColumn; import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; @@ -377,7 +378,7 @@ public TestPreparer(String tempFilePath) public ConnectorPageSource newPageSource() { - OrcPageSourceFactory orcPageSourceFactory = new OrcPageSourceFactory(TYPE_MANAGER, false); + OrcPageSourceFactory orcPageSourceFactory = new OrcPageSourceFactory(TYPE_MANAGER, false, HDFS_ENVIRONMENT); return orcPageSourceFactory.createPageSource( new Configuration(), SESSION, diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java index 72513395bad6a..9048c9ed7d276 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java @@ -16,6 +16,7 @@ import com.facebook.presto.hive.ColumnarBinaryHiveRecordCursorProvider; import com.facebook.presto.hive.ColumnarTextHiveRecordCursorProvider; import com.facebook.presto.hive.GenericHiveRecordCursorProvider; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveColumnHandle; import com.facebook.presto.hive.HiveCompressionCodec; import com.facebook.presto.hive.HivePageSink.DataColumn; @@ -63,7 +64,7 @@ public enum FileFormat { PRESTO_RCBINARY { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCBINARY); @@ -84,7 +85,7 @@ public FormatWriter createFileFormatWriter( PRESTO_RCTEXT { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCTEXT); @@ -105,9 +106,9 @@ public FormatWriter createFileFormatWriter( PRESTO_ORC { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HivePageSourceFactory pageSourceFactory = new OrcPageSourceFactory(TYPE_MANAGER, false); + HivePageSourceFactory pageSourceFactory = new OrcPageSourceFactory(TYPE_MANAGER, false, hdfsEnvironment); return createPageSource(pageSourceFactory, session, targetFile, columnNames, columnTypes, HiveStorageFormat.ORC); } @@ -126,9 +127,9 @@ public FormatWriter createFileFormatWriter( PRESTO_DWRF { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HivePageSourceFactory pageSourceFactory = new DwrfPageSourceFactory(TYPE_MANAGER); + HivePageSourceFactory pageSourceFactory = new DwrfPageSourceFactory(TYPE_MANAGER, hdfsEnvironment); return createPageSource(pageSourceFactory, session, targetFile, columnNames, columnTypes, HiveStorageFormat.DWRF); } @@ -153,9 +154,9 @@ public boolean supportsDate() PRESTO_PARQUET { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HivePageSourceFactory pageSourceFactory = new ParquetPageSourceFactory(TYPE_MANAGER, false); + HivePageSourceFactory pageSourceFactory = new ParquetPageSourceFactory(TYPE_MANAGER, false, hdfsEnvironment); return createPageSource(pageSourceFactory, session, targetFile, columnNames, columnTypes, HiveStorageFormat.PARQUET); } @@ -174,7 +175,7 @@ public FormatWriter createFileFormatWriter( HIVE_RCBINARY { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCBINARY); @@ -195,7 +196,7 @@ public FormatWriter createFileFormatWriter( HIVE_RCTEXT { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCTEXT); @@ -216,7 +217,7 @@ public FormatWriter createFileFormatWriter( HIVE_ORC { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.ORC); @@ -237,7 +238,7 @@ public FormatWriter createFileFormatWriter( HIVE_DWRF { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.DWRF); @@ -264,9 +265,9 @@ public boolean supportsDate() HIVE_PARQUET { @Override - public ConnectorPageSource createFileFormatReader(ConnectorSession session, File targetFile, List columnNames, List columnTypes) + public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new ParquetRecordCursorProvider(false); + HiveRecordCursorProvider cursorProvider = new ParquetRecordCursorProvider(false, hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.PARQUET); } @@ -290,6 +291,7 @@ public boolean supportsDate() public abstract ConnectorPageSource createFileFormatReader( ConnectorSession session, + HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java index c91ee318e75ef..02d683078d062 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/HiveFileFormatBenchmark.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive.benchmark; import com.facebook.presto.hadoop.HadoopNative; +import com.facebook.presto.hive.HdfsEnvironment; import com.facebook.presto.hive.HiveClientConfig; import com.facebook.presto.hive.HiveCompressionCodec; import com.facebook.presto.hive.HiveSessionProperties; @@ -58,6 +59,7 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; +import static com.facebook.presto.hive.HiveTestUtils.createTestHdfsEnvironment; import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; import static io.airlift.testing.FileUtils.createTempDir; import static io.airlift.testing.FileUtils.deleteRecursively; @@ -77,11 +79,14 @@ public class HiveFileFormatBenchmark } @SuppressWarnings("deprecation") - public static final ConnectorSession SESSION = new TestingConnectorSession(new HiveSessionProperties( - new HiveClientConfig() - .setParquetOptimizedReaderEnabled(true)) + public static final HiveClientConfig CONFIG = new HiveClientConfig() + .setParquetOptimizedReaderEnabled(true); + + public static final ConnectorSession SESSION = new TestingConnectorSession(new HiveSessionProperties(CONFIG) .getSessionProperties()); + public static final HdfsEnvironment HDFS_ENVIRONMENT = createTestHdfsEnvironment(CONFIG); + @Param({ "NONE", "SNAPPY", @@ -184,6 +189,7 @@ public List read(CompressionCounter counter) List pages = new ArrayList<>(100); try (ConnectorPageSource pageSource = fileFormat.createFileFormatReader( SESSION, + HDFS_ENVIRONMENT, dataFile, columnNames, fileFormat.supportsDate() ? columnTypes : noDateColumnTypes)) { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java index 0d0e436a5dd71..9562264a804cc 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/ParquetTester.java @@ -162,11 +162,11 @@ private static void assertFileContents(JobConf jobConf, throws IOException, InterruptedException { Path path = new Path(tempFile.getFile().toURI()); - ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(jobConf, path); + FileSystem fileSystem = path.getFileSystem(jobConf); + ParquetMetadata parquetMetadata = ParquetMetadataReader.readFooter(fileSystem, path); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); - FileSystem fileSystem = path.getFileSystem(jobConf); long size = fileSystem.getFileStatus(path).getLen(); FSDataInputStream inputStream = fileSystem.open(path); ParquetDataSource dataSource = new HdfsParquetDataSource(path, size, inputStream); diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index 12a59d715c850..7337db772524b 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -46,10 +46,10 @@ public class TestHiveStorageFormats public static Object[][] storageFormats() { return new StorageFormat[][] { - //{storageFormat("ORC")}, - //{storageFormat("DWRF")}, + {storageFormat("ORC")}, + {storageFormat("DWRF")}, //{storageFormat("PARQUET")}, - //{storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, + {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, //{storageFormat("RCBINARY")}, //{storageFormat("RCTEXT")}, //{storageFormat("SEQUENCEFILE")}, From 329cf92cf40add7b7214bda9d6577b590f07d069 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 15:40:48 +0100 Subject: [PATCH 10/20] Make PARQUET reader compatible with Keberos --- .../presto/hive/parquet/ParquetHiveRecordCursor.java | 11 +++++++---- .../presto/tests/hive/TestHiveStorageFormats.java | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java index 6a98c7c5e67f8..e32ad5f13d02a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetHiveRecordCursor.java @@ -402,7 +402,7 @@ private ParquetRecordReader createParquetRecordReader( try { FileSystem fileSystem = hdfsEnvironment.getFileSystem(sessionUser, path, configuration); dataSource = buildHdfsParquetDataSource(fileSystem, path, start, length); - ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(configuration, path, NO_FILTER); + ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(sessionUser, () -> ParquetFileReader.readFooter(configuration, path, NO_FILTER)); List blocks = parquetMetadata.getBlocks(); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); @@ -442,9 +442,12 @@ private ParquetRecordReader createParquetRecordReader( ParquetInputSplit split = new ParquetInputSplit(path, start, start + length, length, null, offsets); TaskAttemptContext taskContext = ContextUtil.newTaskAttemptContext(configuration, new TaskAttemptID()); - ParquetRecordReader realReader = new PrestoParquetRecordReader(readSupport); - realReader.initialize(split, taskContext); - return realReader; + + return hdfsEnvironment.doAs(sessionUser, () -> { + ParquetRecordReader realReader = new PrestoParquetRecordReader(readSupport); + realReader.initialize(split, taskContext); + return realReader; + }); } catch (Exception e) { if (dataSource != null) { diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index 7337db772524b..a25bbcdbf7eeb 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -48,7 +48,7 @@ public static Object[][] storageFormats() return new StorageFormat[][] { {storageFormat("ORC")}, {storageFormat("DWRF")}, - //{storageFormat("PARQUET")}, + {storageFormat("PARQUET")}, {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, //{storageFormat("RCBINARY")}, //{storageFormat("RCTEXT")}, From bc42b8f6d00efea322faf887190e6c49d023df59 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 15:57:38 +0100 Subject: [PATCH 11/20] Make RCFILE readers compatible with Keberos --- .../ColumnarBinaryHiveRecordCursorProvider.java | 13 ++++++++++++- .../hive/ColumnarTextHiveRecordCursorProvider.java | 13 ++++++++++++- .../presto/hive/AbstractTestHiveClient.java | 4 ++-- .../com/facebook/presto/hive/HiveTestUtils.java | 4 ++-- .../facebook/presto/hive/TestHiveFileFormats.java | 12 ++++++------ .../facebook/presto/hive/benchmark/FileFormat.java | 8 ++++---- .../presto/tests/hive/TestHiveStorageFormats.java | 4 ++-- 7 files changed, 40 insertions(+), 18 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java index 58fdc260bc400..8a881114024a8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursorProvider.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.TypeManager; +import com.google.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; @@ -28,10 +29,19 @@ import java.util.Properties; import static com.facebook.presto.hive.HiveUtil.isDeserializerClass; +import static java.util.Objects.requireNonNull; public class ColumnarBinaryHiveRecordCursorProvider implements HiveRecordCursorProvider { + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public ColumnarBinaryHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + @Override public Optional createHiveRecordCursor( String clientId, @@ -51,7 +61,8 @@ public Optional createHiveRecordCursor( return Optional.empty(); } - RecordReader recordReader = HiveUtil.createRecordReader(configuration, path, start, length, schema, columns); + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), + () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns)); return Optional.of(new ColumnarBinaryHiveRecordCursor<>( bytesRecordReader(recordReader), diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java index 99c1247a98e49..9665426bc9a0b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarTextHiveRecordCursorProvider.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.TypeManager; +import com.google.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; @@ -28,10 +29,19 @@ import java.util.Properties; import static com.facebook.presto.hive.HiveUtil.isDeserializerClass; +import static java.util.Objects.requireNonNull; public class ColumnarTextHiveRecordCursorProvider implements HiveRecordCursorProvider { + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public ColumnarTextHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + @Override public Optional createHiveRecordCursor( String clientId, @@ -51,7 +61,8 @@ public Optional createHiveRecordCursor( return Optional.empty(); } - RecordReader recordReader = HiveUtil.createRecordReader(configuration, path, start, length, schema, columns); + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), + () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns)); return Optional.of(new ColumnarTextHiveRecordCursor<>( columnarTextRecordReader(recordReader), diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java index 42870f46a3126..718654387d683 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClient.java @@ -1169,7 +1169,7 @@ public void testTypesRcTextRecordCursor() ConnectorPageSourceProvider pageSourceProvider = new HivePageSourceProvider( new HiveClientConfig().setTimeZone(timeZone.getID()), hdfsEnvironment, - ImmutableSet.of(new ColumnarTextHiveRecordCursorProvider()), + ImmutableSet.of(new ColumnarTextHiveRecordCursorProvider(hdfsEnvironment)), ImmutableSet.of(), TYPE_MANAGER); @@ -1203,7 +1203,7 @@ public void testTypesRcBinaryRecordCursor() ConnectorPageSourceProvider pageSourceProvider = new HivePageSourceProvider( new HiveClientConfig().setTimeZone(timeZone.getID()), hdfsEnvironment, - ImmutableSet.of(new ColumnarBinaryHiveRecordCursorProvider()), + ImmutableSet.of(new ColumnarBinaryHiveRecordCursorProvider(hdfsEnvironment)), ImmutableSet.of(), TYPE_MANAGER); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java index 2449bc2ef2274..35556551d9b58 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java @@ -57,8 +57,8 @@ public static Set getDefaultHiveRecordCursorProvider(H HdfsEnvironment testHdfsEnvironment = createTestHdfsEnvironment(hiveClientConfig); return ImmutableSet.builder() .add(new ParquetRecordCursorProvider(hiveClientConfig, testHdfsEnvironment)) - .add(new ColumnarTextHiveRecordCursorProvider()) - .add(new ColumnarBinaryHiveRecordCursorProvider()) + .add(new ColumnarTextHiveRecordCursorProvider(testHdfsEnvironment)) + .add(new ColumnarBinaryHiveRecordCursorProvider(testHdfsEnvironment)) .add(new GenericHiveRecordCursorProvider()) .build(); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java index 3905795251214..203fb12bea7b3 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java @@ -125,7 +125,7 @@ public void testRCText(int rowCount) assertThatFileFormat(RCTEXT) .withColumns(testColumns) .withRowsCount(rowCount) - .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider()) + .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT)) .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); } @@ -152,7 +152,7 @@ public void testRCBinary(int rowCount) assertThatFileFormat(RCBINARY) .withColumns(testColumns) .withRowsCount(rowCount) - .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider()) + .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT)) .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); } @@ -328,13 +328,13 @@ public void testTruncateVarcharColumn() assertThatFileFormat(RCTEXT) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider()) + .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT)) .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); assertThatFileFormat(RCBINARY) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider()) + .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT)) .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); assertThatFileFormat(ORC) @@ -380,12 +380,12 @@ public void testFailForLongVarcharPartitionColumn() assertThatFileFormat(RCTEXT) .withColumns(columns) - .isFailingForRecordCursor(new ColumnarTextHiveRecordCursorProvider(), expectedErrorCode, expectedMessage) + .isFailingForRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage) .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); assertThatFileFormat(RCBINARY) .withColumns(columns) - .isFailingForRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(), expectedErrorCode, expectedMessage) + .isFailingForRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage) .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); assertThatFileFormat(ORC) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java index 9048c9ed7d276..d3ebdb191d36e 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java @@ -66,7 +66,7 @@ public enum FileFormat @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCBINARY); } @@ -87,7 +87,7 @@ public FormatWriter createFileFormatWriter( @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCTEXT); } @@ -177,7 +177,7 @@ public FormatWriter createFileFormatWriter( @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new ColumnarBinaryHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCBINARY); } @@ -198,7 +198,7 @@ public FormatWriter createFileFormatWriter( @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new ColumnarTextHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.RCTEXT); } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index a25bbcdbf7eeb..c084cbcaf9721 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -50,8 +50,8 @@ public static Object[][] storageFormats() {storageFormat("DWRF")}, {storageFormat("PARQUET")}, {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, - //{storageFormat("RCBINARY")}, - //{storageFormat("RCTEXT")}, + {storageFormat("RCBINARY")}, + {storageFormat("RCTEXT")}, //{storageFormat("SEQUENCEFILE")}, //{storageFormat("TEXTFILE")} }; From aadff5a7077f7b11ffca7021ee149f3468acba59 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 23 Mar 2016 16:04:57 +0100 Subject: [PATCH 12/20] Make Generic reader compatible with Keberos --- .../hive/GenericHiveRecordCursorProvider.java | 14 ++++++++++++- .../facebook/presto/hive/HiveTestUtils.java | 2 +- .../presto/hive/TestHiveFileFormats.java | 20 +++++++++---------- .../presto/hive/benchmark/FileFormat.java | 4 ++-- .../tests/hive/TestHiveStorageFormats.java | 4 ++-- 5 files changed, 28 insertions(+), 16 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java index aee6306540915..f79020cb89212 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/GenericHiveRecordCursorProvider.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.TypeManager; +import com.google.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Writable; @@ -26,9 +27,19 @@ import java.util.Optional; import java.util.Properties; +import static java.util.Objects.requireNonNull; + public class GenericHiveRecordCursorProvider implements HiveRecordCursorProvider { + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public GenericHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + @Override public Optional createHiveRecordCursor( String clientId, @@ -44,7 +55,8 @@ public Optional createHiveRecordCursor( DateTimeZone hiveStorageTimeZone, TypeManager typeManager) { - RecordReader recordReader = HiveUtil.createRecordReader(configuration, path, start, length, schema, columns); + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), + () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns)); return Optional.of(new GenericHiveRecordCursor<>( genericRecordReader(recordReader), diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java index 35556551d9b58..e0b7168c28487 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveTestUtils.java @@ -59,7 +59,7 @@ public static Set getDefaultHiveRecordCursorProvider(H .add(new ParquetRecordCursorProvider(hiveClientConfig, testHdfsEnvironment)) .add(new ColumnarTextHiveRecordCursorProvider(testHdfsEnvironment)) .add(new ColumnarBinaryHiveRecordCursorProvider(testHdfsEnvironment)) - .add(new GenericHiveRecordCursorProvider()) + .add(new GenericHiveRecordCursorProvider(testHdfsEnvironment)) .build(); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java index 203fb12bea7b3..8377f3ada1240 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveFileFormats.java @@ -126,7 +126,7 @@ public void testRCText(int rowCount) .withColumns(testColumns) .withRowsCount(rowCount) .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); } @Test(enabled = false, dataProvider = "rowCount") @@ -153,7 +153,7 @@ public void testRCBinary(int rowCount) .withColumns(testColumns) .withRowsCount(rowCount) .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); } @Test(enabled = false, dataProvider = "rowCount") @@ -329,13 +329,13 @@ public void testTruncateVarcharColumn() .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) .isReadableByRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); assertThatFileFormat(RCBINARY) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) .isReadableByRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); assertThatFileFormat(ORC) .withWriteColumns(ImmutableList.of(writeColumn)) @@ -358,12 +358,12 @@ public void testTruncateVarcharColumn() assertThatFileFormat(SEQUENCEFILE) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); assertThatFileFormat(TEXTFILE) .withWriteColumns(ImmutableList.of(writeColumn)) .withReadColumns(ImmutableList.of(readColumn)) - .isReadableByRecordCursor(new GenericHiveRecordCursorProvider()); + .isReadableByRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT)); } @Test @@ -381,12 +381,12 @@ public void testFailForLongVarcharPartitionColumn() assertThatFileFormat(RCTEXT) .withColumns(columns) .isFailingForRecordCursor(new ColumnarTextHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage) - .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); + .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); assertThatFileFormat(RCBINARY) .withColumns(columns) .isFailingForRecordCursor(new ColumnarBinaryHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage) - .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); + .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); assertThatFileFormat(ORC) .withColumns(columns) @@ -405,11 +405,11 @@ public void testFailForLongVarcharPartitionColumn() assertThatFileFormat(SEQUENCEFILE) .withColumns(columns) - .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); + .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); assertThatFileFormat(TEXTFILE) .withColumns(columns) - .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(), expectedErrorCode, expectedMessage); + .isFailingForRecordCursor(new GenericHiveRecordCursorProvider(HDFS_ENVIRONMENT), expectedErrorCode, expectedMessage); } private void testCursorProvider(HiveRecordCursorProvider cursorProvider, diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java index d3ebdb191d36e..f1ad2b88ee0ba 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/benchmark/FileFormat.java @@ -219,7 +219,7 @@ public FormatWriter createFileFormatWriter( @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.ORC); } @@ -240,7 +240,7 @@ public FormatWriter createFileFormatWriter( @Override public ConnectorPageSource createFileFormatReader(ConnectorSession session, HdfsEnvironment hdfsEnvironment, File targetFile, List columnNames, List columnTypes) { - HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(); + HiveRecordCursorProvider cursorProvider = new GenericHiveRecordCursorProvider(hdfsEnvironment); return createPageSource(cursorProvider, session, targetFile, columnNames, columnTypes, HiveStorageFormat.DWRF); } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index c084cbcaf9721..731a1d5e67ef8 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -52,8 +52,8 @@ public static Object[][] storageFormats() {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, {storageFormat("RCBINARY")}, {storageFormat("RCTEXT")}, - //{storageFormat("SEQUENCEFILE")}, - //{storageFormat("TEXTFILE")} + {storageFormat("SEQUENCEFILE")}, + {storageFormat("TEXTFILE")} }; } From 5e5c1fc25f7b3d1cc711e9cb6a5a2d862f8a8509 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Thu, 24 Mar 2016 10:04:24 +0100 Subject: [PATCH 13/20] Document kerberos authentication for Hive connector --- .../src/main/sphinx/connector/hive.rst | 88 +++++++++++++++++++ 1 file changed, 88 insertions(+) diff --git a/presto-docs/src/main/sphinx/connector/hive.rst b/presto-docs/src/main/sphinx/connector/hive.rst index ff9d79d7601cc..f8c91573d56b6 100644 --- a/presto-docs/src/main/sphinx/connector/hive.rst +++ b/presto-docs/src/main/sphinx/connector/hive.rst @@ -67,6 +67,24 @@ The configuration files must exist on all Presto nodes. If you are referencing existing Hadoop config files, make sure to copy them to any Presto nodes that are not running Hadoop. +Accessing Hadoop clusters protected with Kerberos authentication +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Kerberos authentication is currently supported for both HDFS and Hive metastore. + +However there are still few limitations: + +* Kerberos authentication is supported only for ``hive-hadoop2`` and `hive-cdh5` connectors. +* Kerberos authentication by ticket cache is not yet supported. + +Please refer to `Configuration Properties`_ section for configuration details. + +.. note:: + + If your ``krb5.conf`` location is different than ``/etc/krb5.conf`` you must set it + explicitly using the ``java.security.krb5.conf`` JVM property in ``jvm.config`` file. + Example: ``-Djava.security.krb5.conf=/example/path/krb5.conf``. + Configuration Properties ------------------------ @@ -107,6 +125,76 @@ Property Name Description ``hive.max-partitions-per-writers`` Maximum number of partitions per writer. 100 ``hive.s3.sse.enabled`` Enable S3 server-side encryption. ``false`` + +``hive.metastore.authentication.type`` Hive metastore authentication type. ``NONE`` + Possible values are ``NONE`` or ``KERBEROS``. + +``hive.metastore.service.principal`` Hive metastore service principal. + The ``_HOST`` placeholder is allowed here and it is + substituted with the actual metastore host. Use ``_HOST`` + placeholder for configurations with more that + one Hive metastore server. + Example: ``hive/hive-server-host@EXAMPLE.COM`` or + ``hive/_HOST@EXAMPLE.COM``. + +``hive.metastore.client.principal`` Hive metastore client principal. + The ``_HOST`` placeholder is allowed here and it is + substituted with the actual Presto server host. Use + ``_HOST`` placeholder for the principal per server + configurations. + Example: ``presto/presto-server-node@EXAMPLE.COM`` or + ``presto/_HOST@EXAMPLE.COM``. + + .. warning:: + + The principal specified by + ``hive.metastore.client.principal`` + must have sufficient privileges to remove files + and directories within the ``hive/warehouse`` + directory. If the principal does not, only the + metadata will be removed, and the data will + continue to consume disk space. + + This occurs because the Hive metastore is + responsible for deleting the internal table data. + When the metastore is configured to use Kerberos + authentication, all of the HDFS operations performed + by the metastore are impersonated. Errors + deleting data are silently ignored. + +``hive.metastore.client.keytab`` Hive metastore client keytab location. Must be accessible + for the user running Presto and must contain the + credentials for the ``hive.metastore.client.principal``. + +``hive.hdfs.authentication.type`` HDFS authentication type. ``NONE`` + Possible values are ``NONE`` or ``KERBEROS``. + +``hive.hdfs.impersonation.enabled`` Enable HDFS calls impersonation. ``false`` + + When set to the default of ``false``, Presto accesses + HDFS as the Unix user the presto process is running as, + or as the Kerberos principal specified in + ``hive.hdfs.presto.principal`` + + When set to ``true``, Presto accesses HDFS as the Presto + user or Kerberos principal specified by ``--user`` or + ``--krb5-principal`` passed to the CLI, or as the user + in the JDBC credentials. + +``hive.hdfs.presto.principal`` HDFS client principal. The ``_HOST`` placeholder + is allowed here and it is substituted with the actual + Presto server host. Use ``_HOST`` placeholder for the + principal per server configurations. + When impersonation is enabled make sure that provided + user is configured to be a super user and has the + impersonation allowed. + Example: + ``presto-hdfs-superuser/presto-server-node@EXAMPLE.COM`` or + ``presto-hdfs-superuser/_HOST@EXAMPLE.COM``. + +``hive.hdfs.presto.keytab`` HDFS client keytab location. Must be accessible + for the user running Presto and must contain the + credentials for the ``hive.hdfs.presto.principal``. ================================================== ============================================================ ========== Querying Hive Tables From e291ee8e5c4e6494274b055d968e4204bd619a77 Mon Sep 17 00:00:00 2001 From: arhimondr Date: Wed, 6 Apr 2016 01:11:18 +0200 Subject: [PATCH 14/20] Update Tempto to 1.8 --- pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index ba9abba26a5d7..e715c3e60ef03 100644 --- a/pom.xml +++ b/pom.xml @@ -49,6 +49,7 @@ ${dep.airlift.version} 0.19 1.9.40 + 1.8