From bbc22dbe8a210e03a6dc063277935d74bb250214 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Wed, 13 Nov 2024 12:43:15 -0800 Subject: [PATCH 1/4] dependency-scope for presto-jdbc --- presto-jdbc/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index afcdabb118294..744cc2922ce71 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -212,6 +212,15 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + com.facebook.airlift:security + + + org.basepom.maven duplicate-finder-maven-plugin From 458e1172b085a4563dbdabc276fe5867456fb529 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Wed, 13 Nov 2024 12:43:59 -0800 Subject: [PATCH 2/4] modernizer for presto-jdbc --- .../com/facebook/presto/jdbc/PrestoConnection.java | 4 ++-- .../java/com/facebook/presto/jdbc/PrestoDriverUri.java | 9 +++++---- .../java/com/facebook/presto/jdbc/PrestoResultSet.java | 6 +++--- .../java/com/facebook/presto/jdbc/PrestoStatement.java | 2 +- .../com/facebook/presto/jdbc/TestJdbcConnection.java | 10 +++++----- .../presto/jdbc/TestPrestoDatabaseMetaData.java | 4 ++-- .../com/facebook/presto/jdbc/TestPrestoDriver.java | 2 +- .../com/facebook/presto/jdbc/TestPrestoDriverUri.java | 6 +++--- 8 files changed, 22 insertions(+), 21 deletions(-) diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java index 1f3f760631850..ca982a1cb0fa8 100644 --- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java +++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoConnection.java @@ -755,7 +755,7 @@ StatementClient startQuery(String sql, Map sessionPropertiesOver String source = "presto-jdbc"; String applicationName = clientInfo.get("ApplicationName"); if (applicationNamePrefix.isPresent()) { - source = applicationNamePrefix.get(); + source = applicationNamePrefix.orElseThrow(); if (applicationName != null) { source += applicationName; } @@ -845,7 +845,7 @@ PrestoResultSet invokeQueryInterceptorsPost(String sql, Statement interceptedSta for (QueryInterceptor interceptor : this.queryInterceptorInstances) { Optional newResultSet = interceptor.postProcess(sql, interceptedStatement, interceptedResultSet); if (newResultSet.isPresent()) { - interceptedResultSet = newResultSet.get(); + interceptedResultSet = newResultSet.orElseThrow(); } } return interceptedResultSet; diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoDriverUri.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoDriverUri.java index e355b58f32331..17dff9299dae3 100644 --- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoDriverUri.java +++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoDriverUri.java @@ -164,10 +164,11 @@ public String getTimeZoneId() if (timezone.isPresent()) { List timeZoneIds = Arrays.asList(TimeZone.getAvailableIDs()); - if (!timeZoneIds.contains(timezone.get())) { - throw new SQLException("Specified timeZoneId is not supported: " + timezone.get()); + String tz = timezone.orElseThrow(); + if (!timeZoneIds.contains(tz)) { + throw new SQLException("Specified timeZoneId is not supported: " + tz); } - return timezone.get(); + return tz; } return TimeZone.getDefault().getID(); } @@ -280,7 +281,7 @@ public void setupClient(OkHttpClient.Builder builder) if (!useSecureConnection) { throw new SQLException("Authentication using an access token requires SSL to be enabled"); } - builder.addInterceptor(tokenAuth(ACCESS_TOKEN.getValue(properties).get())); + builder.addInterceptor(tokenAuth(ACCESS_TOKEN.getValue(properties).orElseThrow())); } } catch (ClientException e) { diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoResultSet.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoResultSet.java index b0763caadcfe8..2d588f950fc77 100644 --- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoResultSet.java +++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoResultSet.java @@ -63,9 +63,9 @@ import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.jdbc.ColumnInfo.setTypeInfo; import static com.google.common.base.Verify.verify; -import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.collect.Iterators.concat; import static com.google.common.collect.Iterators.transform; +import static com.google.common.collect.MoreCollectors.onlyElement; import static java.lang.String.format; import static java.math.RoundingMode.HALF_UP; import static java.util.Locale.ENGLISH; @@ -1126,8 +1126,8 @@ public Array getArray(int columnIndex) } ColumnInfo columnInfo = columnInfo(columnIndex); - String elementTypeName = getOnlyElement(columnInfo.getColumnTypeSignature().getParameters()).toString(); - int elementType = getOnlyElement(columnInfo.getColumnParameterTypes()); + String elementTypeName = columnInfo.getColumnTypeSignature().getParameters().stream().collect(onlyElement()).toString(); + int elementType = columnInfo.getColumnParameterTypes().stream().collect(onlyElement()); return new PrestoArray(elementTypeName, elementType, (List) value); } diff --git a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoStatement.java b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoStatement.java index 8141f073208fe..79f8162f3b8a0 100644 --- a/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoStatement.java +++ b/presto-jdbc/src/main/java/com/facebook/presto/jdbc/PrestoStatement.java @@ -250,7 +250,7 @@ final boolean internalExecute(String sql) if (shouldIntercept) { Optional newResultSet = connection().invokeQueryInterceptorsPre(sql, this); if (newResultSet.isPresent()) { - resultSet = newResultSet.get(); + resultSet = newResultSet.orElseThrow(); } } diff --git a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java index 7e80977af6349..a838778d58729 100644 --- a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java +++ b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestJdbcConnection.java @@ -37,7 +37,6 @@ import java.io.UnsupportedEncodingException; import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; @@ -55,6 +54,7 @@ import static com.facebook.presto.spi.SystemTable.Distribution.ALL_NODES; import static com.google.inject.multibindings.Multibinder.newSetBinder; import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -347,7 +347,7 @@ public void testCustomHeaders() { Map customHeadersMap = ImmutableMap.of("testHeaderKey", "testHeaderValue"); String customHeaders = "testHeaderKey:testHeaderValue"; - String encodedCustomHeaders = URLEncoder.encode(customHeaders, StandardCharsets.UTF_8.toString()); + String encodedCustomHeaders = URLEncoder.encode(customHeaders, UTF_8); Connection connection = createConnection("customHeaders=" + encodedCustomHeaders); assertTrue(connection instanceof PrestoConnection); PrestoConnection prestoConnection = connection.unwrap(PrestoConnection.class); @@ -358,9 +358,9 @@ public void testCustomHeaders() public void testCustomHeadersWithSpecialCharacters(String testHeaderValue) throws SQLException, UnsupportedEncodingException { - Map customHeadersMap = ImmutableMap.of("testHeaderKey", URLEncoder.encode(testHeaderValue, StandardCharsets.UTF_8.toString())); - String customHeaders = "testHeaderKey:" + URLEncoder.encode(testHeaderValue, StandardCharsets.UTF_8.toString()) + ""; - String encodedCustomHeaders = URLEncoder.encode(customHeaders, StandardCharsets.UTF_8.toString()); + Map customHeadersMap = ImmutableMap.of("testHeaderKey", URLEncoder.encode(testHeaderValue, UTF_8)); + String customHeaders = "testHeaderKey:" + URLEncoder.encode(testHeaderValue, UTF_8); + String encodedCustomHeaders = URLEncoder.encode(customHeaders, UTF_8); Connection connection = createConnection("customHeaders=" + encodedCustomHeaders); assertTrue(connection instanceof PrestoConnection); PrestoConnection prestoConnection = connection.unwrap(PrestoConnection.class); diff --git a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDatabaseMetaData.java b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDatabaseMetaData.java index 17e64f99121ab..929ae57e3efef 100644 --- a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDatabaseMetaData.java +++ b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDatabaseMetaData.java @@ -37,7 +37,7 @@ import static com.facebook.presto.common.type.VarcharType.MAX_LENGTH; import static com.facebook.presto.jdbc.TestPrestoDriver.closeQuietly; import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.MoreCollectors.onlyElement; import static java.lang.String.format; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -95,7 +95,7 @@ public void testPassEscapeInMetaDataQuery() }); assertEquals(queries.size(), 1, "Expected exactly one query, got " + queries.size()); - String query = getOnlyElement(queries); + String query = queries.stream().collect(onlyElement()); assertContains(query, "_t' ESCAPE '", "Metadata query does not contain ESCAPE"); } diff --git a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriver.java b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriver.java index 1d91cc3bf05d7..f5a6ca004e31c 100644 --- a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriver.java +++ b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriver.java @@ -1549,7 +1549,7 @@ public void testUpdateCancelExplicit() while (true) { Optional state = findQueryState(queryUuid); if (state.isPresent()) { - assertFalse(state.get().isDone()); + assertFalse(state.orElseThrow().isDone()); break; } MILLISECONDS.sleep(50); diff --git a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriverUri.java b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriverUri.java index 282556b0252ad..5bc15c1910976 100644 --- a/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriverUri.java +++ b/presto-jdbc/src/test/java/com/facebook/presto/jdbc/TestPrestoDriverUri.java @@ -18,7 +18,6 @@ import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.sql.SQLException; import java.util.Properties; @@ -36,6 +35,7 @@ import static com.facebook.presto.jdbc.ConnectionProperties.SSL_TRUST_STORE_PATH; import static com.facebook.presto.jdbc.ConnectionProperties.VALIDATE_NEXTURI_SOURCE; import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -266,7 +266,7 @@ public void testUriWithExtraCredentials() throws SQLException, UnsupportedEncodingException { String extraCredentials = "test.token.foo:bar;test.token.abc:xyz;test.scopes:read_only|read_write"; - String encodedExtraCredentials = URLEncoder.encode(extraCredentials, StandardCharsets.UTF_8.toString()); + String encodedExtraCredentials = URLEncoder.encode(extraCredentials, UTF_8); PrestoDriverUri parameters = createDriverUri("presto://localhost:8080?extraCredentials=" + encodedExtraCredentials); Properties properties = parameters.getProperties(); assertEquals(properties.getProperty(EXTRA_CREDENTIALS.getKey()), extraCredentials); @@ -277,7 +277,7 @@ public void testUriWithCustomHeaders() throws SQLException, UnsupportedEncodingException { String customHeaders = "testHeaderKey:testHeaderValue"; - String encodedCustomHeaders = URLEncoder.encode(customHeaders, StandardCharsets.UTF_8.toString()); + String encodedCustomHeaders = URLEncoder.encode(customHeaders, UTF_8); PrestoDriverUri parameters = createDriverUri("presto://localhost:8080?customHeaders=" + encodedCustomHeaders); Properties properties = parameters.getProperties(); assertEquals(properties.getProperty(CUSTOM_HEADERS.getKey()), customHeaders); From a5265813e1e228d22a3adab4e1918a413b03dce9 Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Wed, 13 Nov 2024 12:52:25 -0800 Subject: [PATCH 3/4] dependency-scope and modernizer for presto-proxy --- presto-proxy/pom.xml | 6 ++++++ .../java/com/facebook/presto/proxy/JsonWebTokenHandler.java | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 878bcdfedae9a..bcb7e0a4c39c7 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -171,6 +171,12 @@ test + + com.facebook.presto + presto-spi + test + + org.testng testng diff --git a/presto-proxy/src/main/java/com/facebook/presto/proxy/JsonWebTokenHandler.java b/presto-proxy/src/main/java/com/facebook/presto/proxy/JsonWebTokenHandler.java index dd1b7a4333cfe..1b7601dd8dd01 100644 --- a/presto-proxy/src/main/java/com/facebook/presto/proxy/JsonWebTokenHandler.java +++ b/presto-proxy/src/main/java/com/facebook/presto/proxy/JsonWebTokenHandler.java @@ -64,7 +64,7 @@ public String getBearerToken(String subject) .setSubject(subject) .setExpiration(Date.from(ZonedDateTime.now().plusMinutes(5).toInstant())); - jwtSigner.get().accept(jwt); + jwtSigner.orElseThrow().accept(jwt); jwtKeyId.ifPresent(keyId -> jwt.setHeaderParam(KEY_ID, keyId)); jwtIssuer.ifPresent(jwt::setIssuer); jwtAudience.ifPresent(jwt::setAudience); From 7ab7b0e0952341fda8b43f2e63f7b8815e0b46ff Mon Sep 17 00:00:00 2001 From: Zac Blanco Date: Wed, 13 Nov 2024 13:56:28 -0800 Subject: [PATCH 4/4] modernizer for presto-hive-metastore --- .../presto/hive/HiveBucketProperty.java | 2 +- .../presto/hive/HiveTypeTranslator.java | 2 +- ...TableConstraintAlreadyExistsException.java | 2 +- .../hive/metastore/HiveMetastoreModule.java | 2 +- .../hive/metastore/HivePageSinkMetadata.java | 2 +- .../HivePageSinkMetadataProvider.java | 2 +- .../InMemoryCachingHiveMetastore.java | 33 +++++----- .../presto/hive/metastore/MetastoreUtil.java | 16 ++--- .../presto/hive/metastore/Partition.java | 2 +- .../SemiTransactionalHiveMetastore.java | 64 +++++++++---------- .../presto/hive/metastore/Statistics.java | 24 +++---- .../metastore/file/FileHiveMetastore.java | 16 ++--- .../metastore/file/PartitionMetadata.java | 4 +- .../hive/metastore/file/TableMetadata.java | 4 +- .../metastore/glue/GlueExpressionUtil.java | 4 +- .../metastore/glue/GlueHiveMetastore.java | 14 ++-- .../glue/converter/GlueInputConverter.java | 8 +-- .../thrift/BridgingHiveMetastore.java | 10 +-- .../hive/metastore/thrift/HiveMetastore.java | 6 +- .../thrift/StaticMetastoreConfig.java | 6 +- .../metastore/thrift/ThriftHiveMetastore.java | 31 +++++---- .../metastore/thrift/ThriftMetastoreUtil.java | 20 +++--- .../hive/metastore/thrift/Transport.java | 2 +- .../glue/TestGlueExpressionUtil.java | 2 +- .../thrift/InMemoryHiveMetastore.java | 4 +- .../thrift/TestThriftHiveMetastoreUtil.java | 2 +- 26 files changed, 143 insertions(+), 141 deletions(-) diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveBucketProperty.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveBucketProperty.java index 7060511756606..4a1579cf2e516 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveBucketProperty.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveBucketProperty.java @@ -56,7 +56,7 @@ public HiveBucketProperty( this.types = requireNonNull(types, "type is null"); if (bucketFunctionType.equals(PRESTO_NATIVE)) { checkArgument(types.isPresent(), "Types must be present for bucket function type " + bucketFunctionType); - checkArgument(types.get().size() == bucketedBy.size(), "The sizes of bucketedBy and types should match"); + checkArgument(types.orElseThrow().size() == bucketedBy.size(), "The sizes of bucketedBy and types should match"); } else { checkArgument(!types.isPresent(), "Types not needed for bucket function type " + bucketFunctionType); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveTypeTranslator.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveTypeTranslator.java index c973f12330f4b..3d8aa50d6c366 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveTypeTranslator.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/HiveTypeTranslator.java @@ -160,7 +160,7 @@ else if (varcharLength == VarcharType.UNBOUNDED_LENGTH) { if (!namedTypeSignature.getName().isPresent()) { throw new PrestoException(NOT_SUPPORTED, format("Anonymous row type is not supported in Hive. Please give each field a name: %s", type)); } - fieldNames.add(namedTypeSignature.getName().get()); + fieldNames.add(namedTypeSignature.getName().orElseThrow()); } return getStructTypeInfo( fieldNames.build(), diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/TableConstraintAlreadyExistsException.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/TableConstraintAlreadyExistsException.java index 1258e4e0d1ca5..901feea202948 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/TableConstraintAlreadyExistsException.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/TableConstraintAlreadyExistsException.java @@ -27,7 +27,7 @@ public class TableConstraintAlreadyExistsException public TableConstraintAlreadyExistsException(Optional constraintName) { - this(constraintName, format("Constraint already exists: '%s'", constraintName.get())); + this(constraintName, format("Constraint already exists: '%s'", constraintName.orElseThrow())); } public TableConstraintAlreadyExistsException(Optional constraintName, String message) diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HiveMetastoreModule.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HiveMetastoreModule.java index 972fdc150e739..baf11a23cfee1 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HiveMetastoreModule.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HiveMetastoreModule.java @@ -40,7 +40,7 @@ public HiveMetastoreModule(String connectorId, Optional m protected void setup(Binder binder) { if (metastore.isPresent()) { - binder.bind(ExtendedHiveMetastore.class).toInstance(metastore.get()); + binder.bind(ExtendedHiveMetastore.class).toInstance(metastore.orElseThrow()); } else { bindMetastoreModule("thrift", new ThriftMetastoreModule(connectorId)); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadata.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadata.java index c6655c0cd278d..b5c50b0448e53 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadata.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadata.java @@ -39,7 +39,7 @@ public HivePageSinkMetadata( this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); this.table = requireNonNull(table, "table is null"); this.modifiedPartitions = requireNonNull(modifiedPartitions, "modifiedPartitions is null"); - checkArgument(table.isPresent() && !table.get().getPartitionColumns().isEmpty() || modifiedPartitions.isEmpty()); + checkArgument(table.isPresent() && !table.orElseThrow().getPartitionColumns().isEmpty() || modifiedPartitions.isEmpty()); } @JsonCreator diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadataProvider.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadataProvider.java index 1a33bebf68742..ce5e2f3180d9c 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadataProvider.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/HivePageSinkMetadataProvider.java @@ -47,7 +47,7 @@ public Optional getTable() public Optional getPartition(List partitionValues) { - if (!table.isPresent() || table.get().getPartitionColumns().isEmpty()) { + if (!table.isPresent() || table.orElseThrow().getPartitionColumns().isEmpty()) { throw new IllegalArgumentException( format("Unexpected call to getPartition. Table name: %s", schemaTableName)); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/InMemoryCachingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/InMemoryCachingHiveMetastore.java index 0fb6a11616484..954d83053cf56 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/InMemoryCachingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/InMemoryCachingHiveMetastore.java @@ -27,8 +27,8 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.SetMultimap; +import com.google.common.collect.Streams; import com.google.common.util.concurrent.UncheckedExecutionException; import io.airlift.units.Duration; import org.weakref.jmx.Managed; @@ -49,6 +49,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadLocalRandom; import java.util.function.Function; +import java.util.stream.Stream; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CORRUPTED_PARTITION_CACHE; import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_DROPPED_DURING_QUERY; @@ -66,7 +67,6 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.ImmutableSetMultimap.toImmutableSetMultimap; -import static com.google.common.collect.Iterables.transform; import static com.google.common.collect.Streams.stream; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static java.lang.String.format; @@ -412,12 +412,12 @@ public Map getPartitionStatistics(MetastoreContext Map, PartitionStatistics> statistics = getAll(partitionStatisticsCache, partitions); return statistics.entrySet() .stream() - .collect(toImmutableMap(entry -> entry.getKey().getKey().getPartitionNameWithVersion().get().getPartitionName(), Entry::getValue)); + .collect(toImmutableMap(entry -> entry.getKey().getKey().getPartitionNameWithVersion().orElseThrow().getPartitionName(), Entry::getValue)); } private PartitionStatistics loadPartitionColumnStatistics(KeyAndContext partition) { - String partitionName = partition.getKey().getPartitionNameWithVersion().get().getPartitionName(); + String partitionName = partition.getKey().getPartitionNameWithVersion().orElseThrow().getPartitionName(); Map partitionStatistics = delegate.getPartitionStatistics( partition.getContext(), partition.getKey().getHiveTableName().getDatabaseName(), @@ -436,7 +436,7 @@ private Map, PartitionStatistics> loadPartition ImmutableMap.Builder, PartitionStatistics> result = ImmutableMap.builder(); tablePartitions.keySet().forEach(table -> { Set partitionNames = tablePartitions.get(table).stream() - .map(partitionName -> partitionName.getKey().getPartitionNameWithVersion().get().getPartitionName()) + .map(partitionName -> partitionName.getKey().getPartitionNameWithVersion().orElseThrow().getPartitionName()) .collect(toImmutableSet()); Map partitionStatistics = delegate.getPartitionStatistics(table.getContext(), table.getKey().getDatabaseName(), table.getKey().getTableName(), partitionNames); for (String partitionName : partitionNames) { @@ -612,7 +612,7 @@ private void invalidateStalePartitions( partitionStatisticsCache.invalidate(partitionNameKey); } else { - Optional partitionVersion = partition.get().getPartitionVersion(); + Optional partitionVersion = partition.orElseThrow().getPartitionVersion(); if (!partitionVersion.isPresent() || !partitionVersion.equals(partitionNameWithVersion.getPartitionVersion())) { partitionCache.invalidate(partitionNameKey); partitionStatisticsCache.invalidate(partitionNameKey); @@ -624,7 +624,7 @@ private void invalidateStalePartitions( private void invalidatePartitionsWithHighColumnCount(Optional partition, KeyAndContext partitionCacheKey) { // Do NOT cache partitions with # of columns > partitionCacheColumnLimit - if (partition.isPresent() && partition.get().getColumns().size() > partitionCacheColumnCountLimit) { + if (partition.isPresent() && partition.orElseThrow().getColumns().size() > partitionCacheColumnCountLimit) { partitionCache.invalidate(partitionCacheKey); metastoreCacheStats.incrementPartitionsWithColumnCountGreaterThanThreshold(); } @@ -679,9 +679,10 @@ private List loadPartitionNamesByFilter(KeyAndContext< @Override public Map> getPartitionsByNames(MetastoreContext metastoreContext, String databaseName, String tableName, List partitionNames) { - Iterable> names = transform(partitionNames, name -> getCachingKey(metastoreContext, HivePartitionName.hivePartitionName(databaseName, tableName, name))); + Stream> names = partitionNames.stream() + .map(name -> getCachingKey(metastoreContext, HivePartitionName.hivePartitionName(databaseName, tableName, name))); - Map, Optional> all = getAll(partitionCache, names); + Map, Optional> all = getAll(partitionCache, names.toList()); if (isPartitionCacheValidationEnabled()) { validatePartitionCache(all); } @@ -689,7 +690,7 @@ public Map> getPartitionsByNames(MetastoreContext me for (Entry, Optional> entry : all.entrySet()) { Optional value = entry.getValue(); invalidatePartitionsWithHighColumnCount(value, entry.getKey()); - partitionsByName.put(entry.getKey().getKey().getPartitionNameWithVersion().get().getPartitionName(), value); + partitionsByName.put(entry.getKey().getKey().getPartitionNameWithVersion().orElseThrow().getPartitionName(), value); } return partitionsByName.build(); } @@ -709,12 +710,14 @@ private Optional loadPartitionByName(KeyAndContext private Map, Optional> loadPartitionsByNames(Iterable> partitionNamesKey) { requireNonNull(partitionNamesKey, "partitionNames is null"); - checkArgument(!Iterables.isEmpty(partitionNamesKey), "partitionNames is empty"); + checkArgument(Streams.stream(partitionNamesKey).findAny().isPresent(), "partitionNames is empty"); //Invalidate Partition Statistics Cache on a partition cache miss. - partitionStatisticsCache.invalidateAll(transform(partitionNamesKey, partitionNameKey -> getCachingKey(partitionNameKey.getContext(), partitionNameKey.getKey()))); + Streams.stream(partitionNamesKey) + .map(partitionNameKey -> getCachingKey(partitionNameKey.getContext(), partitionNameKey.getKey())) + .forEach(partitionStatisticsCache::invalidate); - KeyAndContext firstPartitionKey = Iterables.get(partitionNamesKey, 0); + KeyAndContext firstPartitionKey = Streams.stream(partitionNamesKey).findFirst().orElseThrow(); HiveTableName hiveTableName = firstPartitionKey.getKey().getHiveTableName(); String databaseName = hiveTableName.getDatabaseName(); @@ -725,8 +728,8 @@ private Map, Optional> loadPartition for (KeyAndContext partitionNameKey : partitionNamesKey) { checkArgument(partitionNameKey.getKey().getHiveTableName().equals(hiveTableName), "Expected table name %s but got %s", hiveTableName, partitionNameKey.getKey().getHiveTableName()); checkArgument(partitionNameKey.getContext().equals(firstPartitionKey.getContext()), "Expected context %s but got %s", firstPartitionKey.getContext(), partitionNameKey.getContext()); - partitionsToFetch.add(partitionNameKey.getKey().getPartitionNameWithVersion().get()); - partitionNameToVersionMap.put(partitionNameKey.getKey().getPartitionNameWithVersion().get().getPartitionName(), partitionNameKey.getKey().getPartitionNameWithVersion().get()); + partitionsToFetch.add(partitionNameKey.getKey().getPartitionNameWithVersion().orElseThrow()); + partitionNameToVersionMap.put(partitionNameKey.getKey().getPartitionNameWithVersion().orElseThrow().getPartitionName(), partitionNameKey.getKey().getPartitionNameWithVersion().orElseThrow()); } ImmutableMap.Builder, Optional> partitions = ImmutableMap.builder(); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java index 28cb2b84017d5..5e11e5cca6b00 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/MetastoreUtil.java @@ -273,11 +273,11 @@ public static Properties getHiveSchema( schema.setProperty(META_TABLE_LOCATION, storage.getLocation()); if (storage.getBucketProperty().isPresent()) { - List bucketedBy = storage.getBucketProperty().get().getBucketedBy(); + List bucketedBy = storage.getBucketProperty().orElseThrow().getBucketedBy(); if (!bucketedBy.isEmpty()) { schema.setProperty(BUCKET_FIELD_NAME, Joiner.on(",").join(bucketedBy)); } - schema.setProperty(BUCKET_COUNT, Integer.toString(storage.getBucketProperty().get().getBucketCount())); + schema.setProperty(BUCKET_COUNT, Integer.toString(storage.getBucketProperty().orElseThrow().getBucketCount())); } else { schema.setProperty(BUCKET_COUNT, "0"); @@ -377,7 +377,7 @@ public static List reconstructPartitionSchema(List tableSchema, ImmutableList.Builder columns = ImmutableList.builder(); if (tableToPartitionColumns.isPresent()) { - Map partitionToTableColumns = tableToPartitionColumns.get() + Map partitionToTableColumns = tableToPartitionColumns.orElseThrow() .entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); @@ -432,7 +432,7 @@ public static String getPartitionLocation(Table table, Optional parti if (!partition.isPresent()) { return table.getStorage().getLocation(); } - return partition.get().getStorage().getLocation(); + return partition.orElseThrow().getStorage().getLocation(); } private static String toThriftDdl(String structName, List columns) @@ -473,7 +473,7 @@ public static void verifyOnline(SchemaTableName tableName, Optional part { if (protectMode.offline) { if (partitionName.isPresent()) { - throw new PartitionOfflineException(tableName, partitionName.get(), false, null); + throw new PartitionOfflineException(tableName, partitionName.orElseThrow(), false, null); } throw new TableOfflineException(tableName, false, null); } @@ -481,7 +481,7 @@ public static void verifyOnline(SchemaTableName tableName, Optional part String prestoOffline = parameters.get(PRESTO_OFFLINE); if (!isNullOrEmpty(prestoOffline)) { if (partitionName.isPresent()) { - throw new PartitionOfflineException(tableName, partitionName.get(), true, prestoOffline); + throw new PartitionOfflineException(tableName, partitionName.orElseThrow(), true, prestoOffline); } throw new TableOfflineException(tableName, true, prestoOffline); } @@ -610,11 +610,11 @@ else if (current == '/') { values.add(unescapePathName(partitionName.substring(valueStart, partitionName.length()))); keys.add(unescapePathName(partitionName.substring(keyStart, keyEnd))); - if (!partitionColumnNames.isPresent() || partitionColumnNames.get().size() == 1) { + if (!partitionColumnNames.isPresent() || partitionColumnNames.orElseThrow().size() == 1) { return values.build(); } ImmutableList.Builder orderedValues = ImmutableList.builder(); - partitionColumnNames.get() + partitionColumnNames.orElseThrow() .forEach(columnName -> orderedValues.add(values.build().get(keys.build().indexOf(columnName)))); return orderedValues.build(); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Partition.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Partition.java index fdf8e87a3153e..b3327a31859ec 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Partition.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Partition.java @@ -157,7 +157,7 @@ public long getLastDataCommitTime() public Optional getRowIdPartitionComponent() { if (rowIdPartitionComponent.isPresent()) { - byte[] copy = Arrays.copyOf(rowIdPartitionComponent.get(), rowIdPartitionComponent.get().length); + byte[] copy = Arrays.copyOf(rowIdPartitionComponent.orElseThrow(), rowIdPartitionComponent.orElseThrow().length); return Optional.of(copy); } return Optional.empty(); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java index 3d73bdfcaa641..fd338e893ae8a 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/SemiTransactionalHiveMetastore.java @@ -258,7 +258,7 @@ public synchronized Map getPartitionStatistics(Meta return ImmutableMap.of(); } TableSource tableSource = getTableSource(databaseName, tableName); - Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(table.get().getSchemaTableName(), k -> new HashMap<>()); + Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(table.orElseThrow().getSchemaTableName(), k -> new HashMap<>()); ImmutableSet.Builder partitionNamesToQuery = ImmutableSet.builder(); ImmutableMap.Builder resultBuilder = ImmutableMap.builder(); for (String partitionName : partitionNames) { @@ -586,15 +586,15 @@ public synchronized void truncateUnpartitionedTable(ConnectorSession session, St if (!table.isPresent()) { throw new TableNotFoundException(schemaTableName); } - if (!table.get().getTableType().equals(MANAGED_TABLE) && !table.get().getTableType().equals(MATERIALIZED_VIEW)) { + if (!table.orElseThrow().getTableType().equals(MANAGED_TABLE) && !table.orElseThrow().getTableType().equals(MATERIALIZED_VIEW)) { throw new PrestoException(NOT_SUPPORTED, "Cannot delete from non-managed Hive table"); } - if (!table.get().getPartitionColumns().isEmpty()) { + if (!table.orElseThrow().getPartitionColumns().isEmpty()) { throw new IllegalArgumentException("Table is partitioned"); } - Path path = new Path(table.get().getStorage().getLocation()); - HdfsContext context = new HdfsContext(session, databaseName, tableName, table.get().getStorage().getLocation(), false); + Path path = new Path(table.orElseThrow().getStorage().getLocation()); + HdfsContext context = new HdfsContext(session, databaseName, tableName, table.orElseThrow().getStorage().getLocation(), false); setExclusive((delegate, hdfsEnvironment) -> { RecursiveDeleteResult recursiveDeleteResult = recursiveDeleteFiles(hdfsEnvironment, context, path, ImmutableSet.of(""), false); if (!recursiveDeleteResult.getNotDeletedEligibleItems().isEmpty()) { @@ -653,13 +653,13 @@ private Optional> doGetPartitionNames( if (!partitionNameResult.isPresent()) { throw new PrestoException(TRANSACTION_CONFLICT, format("Table %s.%s was dropped by another transaction", hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName())); } - partitionNames = partitionNameResult.get(); + partitionNames = partitionNameResult.orElseThrow(); break; } default: throw new UnsupportedOperationException("Unknown table source"); } - Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(table.get().getSchemaTableName(), k -> new HashMap<>()); + Map, Action> partitionActionsOfTable = partitionActions.computeIfAbsent(table.orElseThrow().getSchemaTableName(), k -> new HashMap<>()); ImmutableList.Builder resultBuilder = ImmutableList.builder(); // alter/remove newly-altered/dropped partitions from the results from underlying metastore for (PartitionNameWithVersion partitionNameWithVersion : partitionNames) { @@ -686,7 +686,7 @@ private Optional> doGetPartitionNames( } // add newly-added partitions to the results from underlying metastore if (!partitionActionsOfTable.isEmpty()) { - List partitionColumns = table.get().getPartitionColumns(); + List partitionColumns = table.orElseThrow().getPartitionColumns(); List partitionColumnNames = partitionColumns.stream().map(Column::getName).collect(toList()); List parts = convertPredicateToParts(partitionPredicates); for (Action partitionAction : partitionActionsOfTable.values()) { @@ -774,7 +774,7 @@ private synchronized void cacheLastDataCommitTimes(Map lastDataCommitTimes = existingPartitions.values().stream() .filter(Optional::isPresent) - .map(partition -> partition.get().getLastDataCommitTime()) + .map(partition -> partition.orElseThrow().getLastDataCommitTime()) .limit(MAX_LAST_DATA_COMMIT_TIME_ENTRY_PER_TABLE) .collect(toImmutableList()); @@ -1109,7 +1109,7 @@ public synchronized void declareIntentionToWrite( throw new PrestoException(NOT_SUPPORTED, "Can not insert into a table with a partition that has been modified in the same transaction when Presto is configured to skip temporary directories."); } } - declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(writeMode, context, metastoreContext, stagingPathRoot, tempPathRoot, context.getSession().get().getQueryId(), schemaTableName, temporaryTable)); + declaredIntentionsToWrite.add(new DeclaredIntentionToWrite(writeMode, context, metastoreContext, stagingPathRoot, tempPathRoot, context.getSession().orElseThrow().getQueryId(), schemaTableName, temporaryTable)); } public synchronized ConnectorCommitHandle commit() @@ -1371,12 +1371,12 @@ private void prepareAddTable(MetastoreContext metastoreContext, HdfsContext cont checkArgument(!targetLocation.isEmpty(), "target location is empty"); Optional currentPath = tableAndMore.getCurrentLocation(); Path targetPath = new Path(targetLocation); - if (table.getPartitionColumns().isEmpty() && currentPath.isPresent() && !targetPath.equals(currentPath.get())) { + if (table.getPartitionColumns().isEmpty() && currentPath.isPresent() && !targetPath.equals(currentPath.orElseThrow())) { // CREATE TABLE AS SELECT unpartitioned table with staging directory renameDirectory( context, hdfsEnvironment, - currentPath.get(), + currentPath.orElseThrow(), targetPath, () -> cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, true))); } @@ -1385,7 +1385,7 @@ private void prepareAddTable(MetastoreContext metastoreContext, HdfsContext cont // CREATE TABLE AS SELECT unpartitioned table without temporary staging directory // CREATE TABLE partitioned/unpartitioned table (without data) if (pathExists(context, hdfsEnvironment, targetPath)) { - if (currentPath.isPresent() && currentPath.get().equals(targetPath)) { + if (currentPath.isPresent() && currentPath.orElseThrow().equals(targetPath)) { // It is okay to skip directory creation when currentPath is equal to targetPath // because the directory may have been created when creating partition directories. // However, it is important to note that the two being equal does not guarantee @@ -1425,10 +1425,10 @@ private void prepareInsertExistingTable(MetastoreContext metastoreContext, HdfsC } Path targetPath = new Path(table.getStorage().getLocation()); - Path currentPath = tableAndMore.getCurrentLocation().get(); + Path currentPath = tableAndMore.getCurrentLocation().orElseThrow(); cleanUpTasksForAbort.add(new DirectoryCleanUpTask(context, targetPath, false)); if (!targetPath.equals(currentPath)) { - asyncRename(hdfsEnvironment, renameExecutor, fileRenameCancelled, fileRenameFutures, context, currentPath, targetPath, tableAndMore.getFileNames().get()); + asyncRename(hdfsEnvironment, renameExecutor, fileRenameCancelled, fileRenameFutures, context, currentPath, targetPath, tableAndMore.getFileNames().orElseThrow()); } updateStatisticsOperations.add(new UpdateStatisticsOperation(metastoreContext, table.getSchemaTableName(), @@ -1457,12 +1457,12 @@ private void prepareAlterPartition(MetastoreContext metastoreContext, HdfsContex format("The partition that this transaction modified was deleted in another transaction. %s %s", partition.getTableName(), partition.getValues())); } partition = Partition.builder(partition) - .setCreateTime(oldPartition.get().getCreateTime()) - .setLastDataCommitTime(oldPartition.get().getLastDataCommitTime()) + .setCreateTime(oldPartition.orElseThrow().getCreateTime()) + .setLastDataCommitTime(oldPartition.orElseThrow().getLastDataCommitTime()) .build(); String partitionName = getPartitionName(metastoreContext, partition.getDatabaseName(), partition.getTableName(), partition.getValues()); PartitionStatistics oldPartitionStatistics = getExistingPartitionStatistics(metastoreContext, partition, partitionName); - String oldPartitionLocation = oldPartition.get().getStorage().getLocation(); + String oldPartitionLocation = oldPartition.orElseThrow().getStorage().getLocation(); Path oldPartitionPath = new Path(oldPartitionLocation); // Location of the old partition and the new partition can be different because we allow arbitrary directories through LocationService. @@ -1504,7 +1504,7 @@ private void prepareAlterPartition(MetastoreContext metastoreContext, HdfsContex // because metadata might change: e.g. storage format, column types, etc alterPartitionOperations.add(new AlterPartitionOperation(metastoreContext, new PartitionWithStatistics(partition, partitionName, partitionAndMore.getStatisticsUpdate()), - new PartitionWithStatistics(oldPartition.get(), partitionName, oldPartitionStatistics))); + new PartitionWithStatistics(oldPartition.orElseThrow(), partitionName, oldPartitionStatistics))); } private PartitionStatistics getExistingPartitionStatistics(MetastoreContext metastoreContext, Partition partition, String partitionName) @@ -1815,7 +1815,7 @@ private ImmutableMap> buildTableCreationResults() } builder.put( operation.getTable(), - operation.getOperationResult().get().getLastDataCommitTimes()); + operation.getOperationResult().orElseThrow().getLastDataCommitTimes()); } return builder.build(); } @@ -1848,7 +1848,7 @@ private ImmutableMap> buildPartitionAlterationResult continue; } table = operation.getTable(); - lastCommitTimeBuilder.addAll(operation.getOperationResult().get().getLastDataCommitTimes()); + lastCommitTimeBuilder.addAll(operation.getOperationResult().orElseThrow().getLastDataCommitTimes()); } if (table != null) { @@ -1904,7 +1904,7 @@ private void rollbackShared() Optional
table = delegate.getTable(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName()); if (table.isPresent()) { // check every existing partition that is outside for the base directory - if (!table.get().getPartitionColumns().isEmpty()) { + if (!table.orElseThrow().getPartitionColumns().isEmpty()) { List partitionNamesWithVersion = delegate.getPartitionNames(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName()) .orElse(ImmutableList.of()); for (List partitionNameBatch : Iterables.partition(partitionNamesWithVersion, 10)) { @@ -1959,7 +1959,7 @@ private static void deleteTempPathRootDirectory(List d { for (DeclaredIntentionToWrite declaredIntentionToWrite : declaredIntentionsToWrite) { if (declaredIntentionToWrite.getTempPathRoot().isPresent()) { - deleteRecursivelyIfExists(declaredIntentionToWrite.getContext(), hdfsEnvironment, declaredIntentionToWrite.getTempPathRoot().get()); + deleteRecursivelyIfExists(declaredIntentionToWrite.getContext(), hdfsEnvironment, declaredIntentionToWrite.getTempPathRoot().orElseThrow()); } } } @@ -2403,7 +2403,7 @@ public Table getTable() public PrincipalPrivileges getPrincipalPrivileges() { checkState(principalPrivileges.isPresent()); - return principalPrivileges.get(); + return principalPrivileges.orElseThrow(); } public Optional getCurrentLocation() @@ -2448,7 +2448,7 @@ public Table getAugmentedTableForInTransactionRead() // to the staging location. // This way, if the table is accessed in an ongoing transaction, staged data // can be found and accessed. - String currentLocation = this.currentLocation.get().toString(); + String currentLocation = this.currentLocation.orElseThrow().toString(); if (!currentLocation.equals(table.getStorage().getLocation())) { return Table.builder(table) .withStorage(storage -> storage.setLocation(currentLocation)) @@ -2503,7 +2503,7 @@ public Path getCurrentLocation() public List getFileNames() { checkState(fileNames.isPresent()); - return fileNames.get(); + return fileNames.orElseThrow(); } public PartitionStatistics getStatistics() @@ -2808,9 +2808,9 @@ public void run(ExtendedHiveMetastore metastore) try { Optional
existingTable = metastore.getTable(metastoreContext, newTable.getDatabaseName(), newTable.getTableName()); if (existingTable.isPresent()) { - Table table = existingTable.get(); + Table table = existingTable.orElseThrow(); Optional existingTableQueryId = getPrestoQueryId(table); - if (existingTableQueryId.isPresent() && existingTableQueryId.get().equals(queryId)) { + if (existingTableQueryId.isPresent() && existingTableQueryId.orElseThrow().equals(queryId)) { // ignore table if it was already created by the same query during retries done = true; } @@ -2952,7 +2952,7 @@ public UpdateStatisticsOperation(MetastoreContext metastoreContext, SchemaTableN public void run(ExtendedHiveMetastore metastore) { if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); + metastore.updatePartitionStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), partitionName.orElseThrow(), this::updateStatistics); } else { metastore.updateTableStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), this::updateStatistics); @@ -2966,7 +2966,7 @@ public void undo(ExtendedHiveMetastore metastore) return; } if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); + metastore.updatePartitionStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), partitionName.orElseThrow(), this::resetStatistics); } else { metastore.updateTableStatistics(metastoreContext, tableName.getSchemaName(), tableName.getTableName(), this::resetStatistics); @@ -2976,7 +2976,7 @@ public void undo(ExtendedHiveMetastore metastore) public String getDescription() { if (partitionName.isPresent()) { - return format("replace partition parameters %s %s", tableName, partitionName.get()); + return format("replace partition parameters %s %s", tableName, partitionName.orElseThrow()); } return format("replace table parameters %s", tableName); } @@ -3053,7 +3053,7 @@ public void execute() try { Optional remotePartition = metastore.getPartition(metastoreContext, schemaName, tableName, partition.getPartition().getValues()); // getPrestoQueryId(partition) is guaranteed to be non-empty. It is asserted in PartitionAdder.addPartition. - if (remotePartition.isPresent() && getPrestoQueryId(remotePartition.get()).equals(getPrestoQueryId(partition.getPartition()))) { + if (remotePartition.isPresent() && getPrestoQueryId(remotePartition.orElseThrow()).equals(getPrestoQueryId(partition.getPartition()))) { createdPartitionValues.add(partition.getPartition().getValues()); } else { diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Statistics.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Statistics.java index edd52ff2f7fc9..6a1c236e30fb4 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Statistics.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/Statistics.java @@ -118,8 +118,8 @@ private static Optional mergeIntegerStatistics(Optional mergeDoubleStatistics(Optional mergeDecimalStatistics(Optional mergeDateStatistics(Optional mergeBooleanStatistics(Optional> Optional reduce(Optional if (first.isPresent() && second.isPresent()) { switch (operator) { case MAX: - return Optional.of(max(first.get(), second.get())); + return Optional.of(max(first.orElseThrow(), second.orElseThrow())); case MIN: - return Optional.of(min(first.get(), second.get())); + return Optional.of(min(first.orElseThrow(), second.orElseThrow())); default: throw new IllegalArgumentException("Unexpected operator: " + operator); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java index 5ed409e353669..8ec3f39a23d45 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/FileHiveMetastore.java @@ -428,7 +428,7 @@ public synchronized Optional> getAllViews(MetastoreContext metastor return Optional.empty(); } - List views = tables.get().stream() + List views = tables.orElseThrow().stream() .map(tableName -> getTable(metastoreContext, databaseName, tableName)) .filter(Optional::isPresent) .map(Optional::get) @@ -717,7 +717,7 @@ public synchronized void dropPartition(MetastoreContext metastoreContext, String if (!tableReference.isPresent()) { return; } - Table table = tableReference.get(); + Table table = tableReference.orElseThrow(); Path partitionMetadataDirectory = getPartitionMetadataDirectory(table, partitionValues); if (deleteData) { @@ -895,7 +895,7 @@ public synchronized Optional> getPartitionNames(M if (!tableReference.isPresent()) { return Optional.empty(); } - Table table = tableReference.get(); + Table table = tableReference.orElseThrow(); Path tableMetadataDirectory = getTableMetadataDirectory(table); @@ -958,7 +958,7 @@ public synchronized Optional getPartition(MetastoreContext metastoreC if (!tableReference.isPresent()) { return Optional.empty(); } - Table table = tableReference.get(); + Table table = tableReference.orElseThrow(); Path partitionDirectory = getPartitionMetadataDirectory(table, partitionValues); return readSchemaFile("partition", partitionDirectory, partitionCodec) @@ -1058,11 +1058,11 @@ public synchronized void setPartitionLeases(MetastoreContext metastoreContext, S public MetastoreOperationResult dropConstraint(MetastoreContext metastoreContext, String databaseName, String tableName, String constraintName) { Set constraints = readConstraintsFile(databaseName, tableName); - if (constraints.stream().noneMatch(c -> c.getName().get().equals(constraintName))) { + if (constraints.stream().noneMatch(c -> c.getName().orElseThrow().equals(constraintName))) { throw new TableConstraintNotFoundException(Optional.of(constraintName)); } Set updatedConstraints = constraints.stream() - .filter(constraint -> constraint.getName().isPresent() && !constraint.getName().get().equals(constraintName)) + .filter(constraint -> constraint.getName().isPresent() && !constraint.getName().orElseThrow().equals(constraintName)) .collect(toSet()); writeConstraintsFile(updatedConstraints, databaseName, tableName); return EMPTY_RESULT; @@ -1077,7 +1077,7 @@ public MetastoreOperationResult addConstraint(MetastoreContext metastoreContext, } if (tableConstraint.getName().isPresent()) { final TableConstraint finalTableConstraint = tableConstraint; - if (constraints.stream().anyMatch(constraint -> (finalTableConstraint.getName().get().equalsIgnoreCase(((String) constraint.getName().get()))))) { + if (constraints.stream().anyMatch(constraint -> (finalTableConstraint.getName().orElseThrow().equalsIgnoreCase(((String) constraint.getName().orElseThrow()))))) { throw new TableConstraintAlreadyExistsException(tableConstraint.getName()); } } @@ -1122,7 +1122,7 @@ public List> getTableConstraints(MetastoreContext metast constraints.addAll(rawConstraints.stream() .map(constraint -> (TableConstraint) constraint) .filter(constraint -> (constraint instanceof UniqueConstraint) && !(constraint instanceof PrimaryKeyConstraint)) - .sorted(Comparator.comparing(constraint -> constraint.getName().get())) + .sorted(Comparator.comparing(constraint -> constraint.getName().orElseThrow())) .collect(toList())); constraints.addAll(rawConstraints.stream() diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/PartitionMetadata.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/PartitionMetadata.java index f56a90a84e915..cd6b28b05d114 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/PartitionMetadata.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/PartitionMetadata.java @@ -38,8 +38,8 @@ import static com.facebook.presto.hive.metastore.MetastoreUtil.updateStatisticsParameters; import static com.facebook.presto.hive.metastore.PrestoTableType.EXTERNAL_TABLE; import static com.facebook.presto.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT; -import static com.google.common.base.MoreObjects.firstNonNull; import static java.util.Objects.requireNonNull; +import static java.util.Objects.requireNonNullElse; public class PartitionMetadata { @@ -78,7 +78,7 @@ public PartitionMetadata( this.storageFormat = storageFormat == null ? VIEW_STORAGE_FORMAT : storageFormat; this.bucketProperty = requireNonNull(bucketProperty, "bucketProperty is null"); - this.storageParameters = ImmutableMap.copyOf(firstNonNull(storageParameters, ImmutableMap.of())); + this.storageParameters = ImmutableMap.copyOf(requireNonNullElse(storageParameters, ImmutableMap.of())); this.serdeParameters = requireNonNull(serdeParameters, "serdeParameters is null"); this.externalLocation = requireNonNull(externalLocation, "externalLocation is null"); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/TableMetadata.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/TableMetadata.java index f9520ea40d33e..187bd63b638bd 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/TableMetadata.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/file/TableMetadata.java @@ -35,9 +35,9 @@ import static com.facebook.presto.hive.HiveStorageFormat.getHiveStorageFormat; import static com.facebook.presto.hive.metastore.PrestoTableType.EXTERNAL_TABLE; import static com.facebook.presto.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT; -import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +import static java.util.Objects.requireNonNullElse; public class TableMetadata { @@ -81,7 +81,7 @@ public TableMetadata( this.dataColumns = ImmutableList.copyOf(requireNonNull(dataColumns, "dataColumns is null")); this.partitionColumns = ImmutableList.copyOf(requireNonNull(partitionColumns, "partitionColumns is null")); this.parameters = ImmutableMap.copyOf(requireNonNull(parameters, "parameters is null")); - this.storageParameters = ImmutableMap.copyOf(firstNonNull(storageParameters, ImmutableMap.of())); + this.storageParameters = ImmutableMap.copyOf(requireNonNullElse(storageParameters, ImmutableMap.of())); this.storageFormat = storageFormat == null ? VIEW_STORAGE_FORMAT : storageFormat; this.bucketProperty = requireNonNull(bucketProperty, "bucketProperty is null"); this.serdeParameters = requireNonNull(serdeParameters, "serdeParameters is null"); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueExpressionUtil.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueExpressionUtil.java index 3d9c3f3c29580..96d8e5dbfea8f 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueExpressionUtil.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueExpressionUtil.java @@ -79,7 +79,7 @@ public static String buildGlueExpression(Map partitionPredicates if (domain != null && !domain.isAll()) { Optional columnExpression = buildGlueExpressionForSingleDomain(columnName, domain); if (columnExpression.isPresent()) { - int newExpressionLength = expressionLength + columnExpression.get().length(); + int newExpressionLength = expressionLength + columnExpression.orElseThrow().length(); if (expressionLength > 0) { newExpressionLength += CONJUNCT_SEPARATOR.length(); } @@ -88,7 +88,7 @@ public static String buildGlueExpression(Map partitionPredicates continue; } - perColumnExpressions.add((columnExpression.get())); + perColumnExpressions.add((columnExpression.orElseThrow())); expressionLength = newExpressionLength; } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueHiveMetastore.java index c9c2f502c3592..740c8d0bea171 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/GlueHiveMetastore.java @@ -210,11 +210,11 @@ private static AWSGlueAsync createAsyncGlueClient(GlueHiveMetastoreConfig config if (config.getGlueEndpointUrl().isPresent()) { checkArgument(config.getGlueRegion().isPresent(), "Glue region must be set when Glue endpoint URL is set"); asyncGlueClientBuilder.setEndpointConfiguration(new EndpointConfiguration( - config.getGlueEndpointUrl().get(), - config.getGlueRegion().get())); + config.getGlueEndpointUrl().orElseThrow(), + config.getGlueRegion().orElseThrow())); } else if (config.getGlueRegion().isPresent()) { - asyncGlueClientBuilder.setRegion(config.getGlueRegion().get()); + asyncGlueClientBuilder.setRegion(config.getGlueRegion().orElseThrow()); } else if (config.getPinGlueClientToCurrentRegion()) { Region currentRegion = Regions.getCurrentRegion(); @@ -225,12 +225,12 @@ else if (config.getPinGlueClientToCurrentRegion()) { if (config.getAwsAccessKey().isPresent() && config.getAwsSecretKey().isPresent()) { AWSCredentialsProvider credentialsProvider = new AWSStaticCredentialsProvider( - new BasicAWSCredentials(config.getAwsAccessKey().get(), config.getAwsSecretKey().get())); + new BasicAWSCredentials(config.getAwsAccessKey().orElseThrow(), config.getAwsSecretKey().orElseThrow())); asyncGlueClientBuilder.setCredentials(credentialsProvider); } else if (config.getIamRole().isPresent()) { AWSCredentialsProvider credentialsProvider = new STSAssumeRoleSessionCredentialsProvider - .Builder(config.getIamRole().get(), "roleSessionName") + .Builder(config.getIamRole().orElseThrow(), "roleSessionName") .build(); asyncGlueClientBuilder.setCredentials(credentialsProvider); } @@ -470,7 +470,7 @@ public Optional> getAllViews(MetastoreContext metastoreContext, Str public void createDatabase(MetastoreContext metastoreContext, Database database) { if (!database.getLocation().isPresent() && defaultDir.isPresent()) { - String databaseLocation = new Path(defaultDir.get(), database.getDatabaseName()).toString(); + String databaseLocation = new Path(defaultDir.orElseThrow(), database.getDatabaseName()).toString(); database = Database.builder(database) .setLocation(Optional.of(databaseLocation)) .build(); @@ -488,7 +488,7 @@ public void createDatabase(MetastoreContext metastoreContext, Database database) } if (database.getLocation().isPresent()) { - createDirectory(hdfsContext, hdfsEnvironment, new Path(database.getLocation().get())); + createDirectory(hdfsContext, hdfsEnvironment, new Path(database.getLocation().orElseThrow())); } } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/converter/GlueInputConverter.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/converter/GlueInputConverter.java index edf2b7d17fdf5..7b55a385842da 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/converter/GlueInputConverter.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/glue/converter/GlueInputConverter.java @@ -125,10 +125,10 @@ private static StorageDescriptor convertStorage(Storage storage, List co Optional bucketProperty = storage.getBucketProperty(); if (bucketProperty.isPresent()) { - sd.setNumberOfBuckets(bucketProperty.get().getBucketCount()); - sd.setBucketColumns(bucketProperty.get().getBucketedBy()); - if (!bucketProperty.get().getSortedBy().isEmpty()) { - sd.setSortColumns(bucketProperty.get().getSortedBy().stream() + sd.setNumberOfBuckets(bucketProperty.orElseThrow().getBucketCount()); + sd.setBucketColumns(bucketProperty.orElseThrow().getBucketedBy()); + if (!bucketProperty.orElseThrow().getSortedBy().isEmpty()) { + sd.setSortColumns(bucketProperty.orElseThrow().getSortedBy().stream() .map(column -> new Order().withColumn(column.getColumnName()).withSortOrder(column.getOrder().getHiveOrder())) .collect(toImmutableList())); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java index 61c76f9e4cace..1ac8426ed2891 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/BridgingHiveMetastore.java @@ -107,7 +107,7 @@ public Optional
getTable(MetastoreContext metastoreContext, HiveTableHand { return delegate.getTable(metastoreContext, hiveTableHandle).map(table -> { if (isAvroTableWithSchemaSet(table) || isCsvTable(table)) { - return fromMetastoreApiTable(table, delegate.getFields(metastoreContext, hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()).get(), metastoreContext.getColumnConverter()); + return fromMetastoreApiTable(table, delegate.getFields(metastoreContext, hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()).orElseThrow(), metastoreContext.getColumnConverter()); } return fromMetastoreApiTable(table, metastoreContext.getColumnConverter()); }); @@ -119,7 +119,7 @@ public List> getTableConstraints(MetastoreContext metast ImmutableList.Builder> constraints = ImmutableList.builder(); Optional> primaryKey = delegate.getPrimaryKey(metastoreContext, databaseName, tableName); if (primaryKey.isPresent()) { - constraints.add(primaryKey.get()); + constraints.add(primaryKey.orElseThrow()); } constraints.addAll(delegate.getUniqueConstraints(metastoreContext, databaseName, tableName)); constraints.addAll(delegate.getNotNullConstraints(metastoreContext, databaseName, tableName)); @@ -222,7 +222,7 @@ public MetastoreOperationResult renameTable(MetastoreContext metastoreContext, S if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - org.apache.hadoop.hive.metastore.api.Table table = source.get(); + org.apache.hadoop.hive.metastore.api.Table table = source.orElseThrow(); table.setDbName(newDatabaseName); table.setTableName(newTableName); return alterTable(metastoreContext, databaseName, tableName, table); @@ -235,7 +235,7 @@ public MetastoreOperationResult addColumn(MetastoreContext metastoreContext, Str if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - org.apache.hadoop.hive.metastore.api.Table table = source.get(); + org.apache.hadoop.hive.metastore.api.Table table = source.orElseThrow(); Column column = new Column(columnName, columnType, Optional.ofNullable(columnComment), Optional.empty()); table.getSd().getCols().add(metastoreContext.getColumnConverter().fromColumn(column)); return alterTable(metastoreContext, databaseName, tableName, table); @@ -248,7 +248,7 @@ public MetastoreOperationResult renameColumn(MetastoreContext metastoreContext, if (!source.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - org.apache.hadoop.hive.metastore.api.Table table = source.get(); + org.apache.hadoop.hive.metastore.api.Table table = source.orElseThrow(); for (FieldSchema fieldSchema : table.getPartitionKeys()) { if (fieldSchema.getName().equals(oldColumnName)) { throw new PrestoException(NOT_SUPPORTED, "Renaming partition columns is not supported"); diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java index d5172c78ceefc..fd46c41f12359 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/HiveMetastore.java @@ -161,7 +161,7 @@ default boolean isTableOwner(MetastoreContext metastoreContext, String user, Str { // a table can only be owned by a user Optional
table = getTable(metastoreContext, databaseName, tableName); - return table.isPresent() && user.equals(table.get().getOwner()); + return table.isPresent() && user.equals(table.orElseThrow().getOwner()); } default Optional> getFields(MetastoreContext metastoreContext, String databaseName, String tableName) @@ -171,11 +171,11 @@ default Optional> getFields(MetastoreContext metastoreContext, throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - if (table.get().getSd() == null) { + if (table.orElseThrow().getSd() == null) { throw new PrestoException(HIVE_INVALID_METADATA, "Table is missing storage descriptor"); } - return Optional.of(table.get().getSd().getCols()); + return Optional.of(table.orElseThrow().getSd().getCols()); } default Optional> getPrimaryKey(MetastoreContext metastoreContext, String databaseName, String tableName) diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/StaticMetastoreConfig.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/StaticMetastoreConfig.java index f1f2f6d5d4394..9a3fbf63373d8 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/StaticMetastoreConfig.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/StaticMetastoreConfig.java @@ -16,14 +16,14 @@ import com.facebook.airlift.configuration.Config; import com.facebook.airlift.configuration.ConfigDescription; import com.google.common.base.Splitter; -import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; import javax.validation.constraints.NotNull; import java.net.URI; import java.util.List; -import static com.google.common.collect.Iterables.transform; +import static com.google.common.collect.ImmutableList.toImmutableList; public class StaticMetastoreConfig { @@ -48,7 +48,7 @@ public StaticMetastoreConfig setMetastoreUris(String uris) return this; } - this.metastoreUris = ImmutableList.copyOf(transform(SPLITTER.split(uris), URI::create)); + this.metastoreUris = Streams.stream(SPLITTER.split(uris)).map(URI::create).collect(toImmutableList()); return this; } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java index db795ed0fd6b2..ee9c8f7a2060a 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftHiveMetastore.java @@ -54,7 +54,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.TableType; @@ -155,7 +154,7 @@ import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.MoreCollectors.onlyElement; import static com.google.common.collect.Sets.difference; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -234,11 +233,11 @@ public Optional> getPrimaryKey(MetastoreContext met .run("getPrimaryKey", stats.getGetPrimaryKey().wrap(() -> getMetastoreClientThenCall(metastoreContext, client -> client.getPrimaryKey(dbName, tableName)))); - if (!pkResponse.isPresent() || pkResponse.get().getPrimaryKeys().size() == 0) { + if (!pkResponse.isPresent() || pkResponse.orElseThrow().getPrimaryKeys().size() == 0) { return Optional.empty(); } - List pkCols = pkResponse.get().getPrimaryKeys(); + List pkCols = pkResponse.orElseThrow().getPrimaryKeys(); boolean isEnabled = pkCols.get(0).isEnable_cstr(); boolean isRely = pkCols.get(0).isRely_cstr(); boolean isEnforced = pkCols.get(0).isValidate_cstr(); @@ -263,11 +262,11 @@ public List> getUniqueConstraints(MetastoreContext meta .run("getUniqueConstraints", stats.getGetUniqueConstraints().wrap(() -> getMetastoreClientThenCall(metastoreContext, client -> client.getUniqueConstraints("hive", dbName, tableName)))); - if (!uniqueConstraintsResponse.isPresent() || uniqueConstraintsResponse.get().getUniqueConstraints().size() == 0) { + if (!uniqueConstraintsResponse.isPresent() || uniqueConstraintsResponse.orElseThrow().getUniqueConstraints().size() == 0) { return ImmutableList.of(); } - List uniqueConstraints = uniqueConstraintsResponse.get().getUniqueConstraints(); + List uniqueConstraints = uniqueConstraintsResponse.orElseThrow().getUniqueConstraints(); //bucket the unique constraint columns by constraint name Map> bucketedConstraints = uniqueConstraints.stream().collect(Collectors.groupingBy(SQLUniqueConstraint::getUk_name)); @@ -300,11 +299,11 @@ public List> getNotNullConstraints(MetastoreContext me .run("getNotNullConstraints", stats.getGetNotNullConstraints().wrap(() -> getMetastoreClientThenCall(metastoreContext, client -> client.getNotNullConstraints("hive", dbName, tableName)))); - if (!notNullConstraintsResponse.isPresent() || notNullConstraintsResponse.get().getNotNullConstraints().size() == 0) { + if (!notNullConstraintsResponse.isPresent() || notNullConstraintsResponse.orElseThrow().getNotNullConstraints().size() == 0) { return ImmutableList.of(); } - ImmutableList> result = notNullConstraintsResponse.get().getNotNullConstraints().stream() + ImmutableList> result = notNullConstraintsResponse.orElseThrow().getNotNullConstraints().stream() .map(constraint -> new NotNullConstraint<>(constraint.getColumn_name())) .collect(toImmutableList()); @@ -596,7 +595,7 @@ public synchronized void updateTableStatistics(MetastoreContext metastoreContext com.facebook.presto.hive.metastore.Table table = fromMetastoreApiTable(modifiedTable, metastoreContext.getColumnConverter()); OptionalLong rowCount = basicStatistics.getRowCount(); List metastoreColumnStatistics = updatedStatistics.getColumnStatistics().entrySet().stream() - .map(entry -> createMetastoreColumnStatistics(entry.getKey(), table.getColumn(entry.getKey()).get().getType(), entry.getValue(), rowCount)) + .map(entry -> createMetastoreColumnStatistics(entry.getKey(), table.getColumn(entry.getKey()).orElseThrow().getType(), entry.getValue(), rowCount)) .collect(toImmutableList()); if (!metastoreColumnStatistics.isEmpty()) { setTableColumnStatistics(metastoreContext, databaseName, tableName, metastoreColumnStatistics); @@ -663,7 +662,7 @@ public synchronized void updatePartitionStatistics(MetastoreContext metastoreCon throw new PrestoException(HIVE_METASTORE_ERROR, "Metastore returned multiple partitions for name: " + partitionName); } - Partition originalPartition = getOnlyElement(partitions); + Partition originalPartition = partitions.stream().collect(onlyElement()); Partition modifiedPartition = originalPartition.deepCopy(); HiveBasicStatistics basicStatistics = updatedStatistics.getBasicStatistics(); modifiedPartition.setParameters(updateStatisticsParameters(modifiedPartition.getParameters(), basicStatistics)); @@ -1038,7 +1037,7 @@ else if (constraint instanceof NotNullConstraint) { table.getCatName(), table.getDbName(), table.getTableName(), - constraint.getColumns().stream().findFirst().get(), + constraint.getColumns().stream().findFirst().orElseThrow(), constraint.getName().orElse(null), constraint.isEnabled(), constraint.isEnforced(), @@ -1409,7 +1408,7 @@ public Optional getPartition(MetastoreContext metastoreContext, Strin public List getPartitionsByNames(MetastoreContext metastoreContext, String databaseName, String tableName, List partitionNames) { requireNonNull(partitionNames, "partitionNames is null"); - checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); + checkArgument(partitionNames.stream().findAny().isPresent(), "partitionNames is empty"); try { return retry() @@ -1447,7 +1446,7 @@ public void grantTablePrivileges(MetastoreContext metastoreContext, String datab Set privilegesToGrant = new HashSet<>(requestedPrivileges); Iterator iterator = privilegesToGrant.iterator(); while (iterator.hasNext()) { - HivePrivilegeInfo requestedPrivilege = getOnlyElement(parsePrivilege(iterator.next(), Optional.empty())); + HivePrivilegeInfo requestedPrivilege = parsePrivilege(iterator.next(), Optional.empty()).stream().collect(onlyElement()); for (HivePrivilegeInfo existingPrivilege : existingPrivileges) { if ((requestedPrivilege.isContainedIn(existingPrivilege))) { @@ -1496,7 +1495,7 @@ public void revokeTablePrivileges(MetastoreContext metastoreContext, String data .collect(toSet()); Set privilegesToRevoke = requestedPrivileges.stream() - .filter(privilegeGrantInfo -> existingHivePrivileges.contains(getOnlyElement(parsePrivilege(privilegeGrantInfo, Optional.empty())).getHivePrivilege())) + .filter(privilegeGrantInfo -> existingHivePrivileges.contains(parsePrivilege(privilegeGrantInfo, Optional.empty()).stream().collect(onlyElement()).getHivePrivilege())) .collect(toSet()); if (privilegesToRevoke.isEmpty()) { @@ -1593,7 +1592,7 @@ public MetastoreOperationResult addConstraint(MetastoreContext metastoreContext, throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } - org.apache.hadoop.hive.metastore.api.Table table = source.get(); + org.apache.hadoop.hive.metastore.api.Table table = source.orElseThrow(); Set constraintColumns = tableConstraint.getColumns(); int keySequence = 1; List primaryKeyConstraint = new ArrayList<>(); @@ -1649,7 +1648,7 @@ else if (tableConstraint instanceof NotNullConstraint) { new SQLNotNullConstraint(table.getCatName(), table.getDbName(), table.getTableName(), - tableConstraint.getColumns().stream().findFirst().get(), + tableConstraint.getColumns().stream().findFirst().orElseThrow(), tableConstraint.getName().orElse(null), true, true, diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreUtil.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreUtil.java index 40a1afcc6398c..43a6303825e61 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreUtil.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreUtil.java @@ -307,16 +307,16 @@ public static boolean isRoleEnabled(ConnectorIdentity identity, Function listEnabledRoles(ConnectorIdentity identity, Function> listRoleGrants) { Optional role = identity.getRole(); - if (role.isPresent() && role.get().getType() == SelectedRole.Type.NONE) { + if (role.isPresent() && role.orElseThrow().getType() == SelectedRole.Type.NONE) { return Stream.of(PUBLIC_ROLE_NAME); } PrestoPrincipal principal; - if (!role.isPresent() || role.get().getType() == SelectedRole.Type.ALL) { + if (!role.isPresent() || role.orElseThrow().getType() == SelectedRole.Type.ALL) { principal = new PrestoPrincipal(USER, identity.getUser()); } else { - principal = new PrestoPrincipal(ROLE, role.get().getRole().get()); + principal = new PrestoPrincipal(ROLE, role.orElseThrow().getRole().orElseThrow()); } Stream roles = Stream.of(PUBLIC_ROLE_NAME); @@ -714,10 +714,10 @@ private static StorageDescriptor makeStorageDescriptor(String tableName, List bucketProperty = storage.getBucketProperty(); if (bucketProperty.isPresent()) { - sd.setNumBuckets(bucketProperty.get().getBucketCount()); - sd.setBucketCols(bucketProperty.get().getBucketedBy()); - if (!bucketProperty.get().getSortedBy().isEmpty()) { - sd.setSortCols(bucketProperty.get().getSortedBy().stream() + sd.setNumBuckets(bucketProperty.orElseThrow().getBucketCount()); + sd.setBucketCols(bucketProperty.orElseThrow().getBucketedBy()); + if (!bucketProperty.orElseThrow().getSortedBy().isEmpty()) { + sd.setSortCols(bucketProperty.orElseThrow().getSortedBy().stream() .map(column -> new Order(column.getColumnName(), column.getOrder().getHiveOrder())) .collect(toList())); } diff --git a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/Transport.java b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/Transport.java index 3ab0dafd5480e..f6c54d6f46575 100644 --- a/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/Transport.java +++ b/presto-hive-metastore/src/main/java/com/facebook/presto/hive/metastore/thrift/Transport.java @@ -71,7 +71,7 @@ private static TTransport createRaw(HostAndPort address, Optional ss // SSL will connect to the SOCKS address when present HostAndPort sslConnectAddress = socksProxy.orElse(address); - socket = sslContext.get().getSocketFactory().createSocket(socket, sslConnectAddress.getHost(), sslConnectAddress.getPort(), true); + socket = sslContext.orElseThrow().getSocketFactory().createSocket(socket, sslConnectAddress.getHost(), sslConnectAddress.getPort(), true); } return new TSocket(socket); } diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/glue/TestGlueExpressionUtil.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/glue/TestGlueExpressionUtil.java index a75dcabc9da98..763804a77d655 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/glue/TestGlueExpressionUtil.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/glue/TestGlueExpressionUtil.java @@ -43,7 +43,7 @@ public void testBuildGlueExpressionDomainEqualsSingleValue() { Domain domain = Domain.singleValue(VarcharType.VARCHAR, utf8Slice("2020-01-01")); Optional foo = buildGlueExpressionForSingleDomain("foo", domain); - assertEquals(foo.get(), "((foo = '2020-01-01'))"); + assertEquals(foo.orElseThrow(), "((foo = '2020-01-01'))"); } @Test diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/InMemoryHiveMetastore.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/InMemoryHiveMetastore.java index 81bcf00059b8f..ddf21106b3a5c 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/InMemoryHiveMetastore.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/InMemoryHiveMetastore.java @@ -237,7 +237,7 @@ public synchronized void dropTable(MetastoreContext metastoreContext, String dat private static List listAllDataPaths(MetastoreContext metastoreContext, HiveMetastore metastore, String schemaName, String tableName) { ImmutableList.Builder locations = ImmutableList.builder(); - Table table = metastore.getTable(metastoreContext, schemaName, tableName).get(); + Table table = metastore.getTable(metastoreContext, schemaName, tableName).orElseThrow(); if (table.getSd().getLocation() != null) { // For unpartitioned table, there should be nothing directly under this directory. // But including this location in the set makes the directory content assert more @@ -247,7 +247,7 @@ private static List listAllDataPaths(MetastoreContext metastoreContext, Optional> partitionNames = metastore.getPartitionNames(metastoreContext, schemaName, tableName); if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(metastoreContext, schemaName, tableName, partitionNames.get()).stream() + metastore.getPartitionsByNames(metastoreContext, schemaName, tableName, partitionNames.orElseThrow()).stream() .map(partition -> partition.getSd().getLocation()) .filter(location -> !location.startsWith(table.getSd().getLocation())) .forEach(locations::add); diff --git a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/TestThriftHiveMetastoreUtil.java b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/TestThriftHiveMetastoreUtil.java index 1dae98e7d3217..2e574dc5adf80 100644 --- a/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/TestThriftHiveMetastoreUtil.java +++ b/presto-hive-metastore/src/test/java/com/facebook/presto/hive/metastore/thrift/TestThriftHiveMetastoreUtil.java @@ -398,6 +398,6 @@ public void testGetRowIDPartitionComponent() partition.setParameters(ImmutableMap.of("rowIDPartitionComponent", "\u0000\u0001\u00FF")); byte[] expected = {0, 1, (byte) 255}; - assertEquals(ThriftMetastoreUtil.getRowIDPartitionComponent(partition).get(), expected); + assertEquals(ThriftMetastoreUtil.getRowIDPartitionComponent(partition).orElseThrow(), expected); } }