diff --git a/presto-cache/src/main/java/com/facebook/presto/cache/alluxio/AlluxioCachingFileSystem.java b/presto-cache/src/main/java/com/facebook/presto/cache/alluxio/AlluxioCachingFileSystem.java index 54c4627187f7..87656ec782df 100644 --- a/presto-cache/src/main/java/com/facebook/presto/cache/alluxio/AlluxioCachingFileSystem.java +++ b/presto-cache/src/main/java/com/facebook/presto/cache/alluxio/AlluxioCachingFileSystem.java @@ -29,7 +29,7 @@ import static alluxio.conf.PropertyKey.USER_CLIENT_CACHE_QUOTA_ENABLED; import static com.google.common.base.Preconditions.checkState; -import static com.google.common.hash.Hashing.md5; +import static com.google.common.hash.Hashing.sha256; import static java.nio.charset.StandardCharsets.UTF_8; public class AlluxioCachingFileSystem @@ -81,16 +81,16 @@ public FSDataInputStream openFile(Path path, HiveFileContext hiveFileContext) // Using Alluxio caching requires knowing file size for now if (hiveFileContext.isCacheable() && hiveFileContext.getFileSize().isPresent()) { // FilePath is a unique identifier for a file, however it can be a long string - // hence using md5 hash of the file path as the identifier in the cache. + // hence using sha256 hash of the file path as the identifier in the cache. // We don't set fileId because fileId is Alluxio specific FileInfo info = new FileInfo() .setLastModificationTimeMs(hiveFileContext.getModificationTime()) .setPath(path.toString()) .setFolder(false) .setLength(hiveFileContext.getFileSize().getAsLong()); - String cacheIdentifier = md5().hashString(path.toString(), UTF_8).toString(); + String cacheIdentifier = sha256().hashString(path.toString(), UTF_8).toString(); if (lastModifiedTimeCheckEnabled) { - cacheIdentifier = md5().hashString(cacheIdentifier + hiveFileContext.getModificationTime(), UTF_8).toString(); + cacheIdentifier = sha256().hashString(cacheIdentifier + hiveFileContext.getModificationTime(), UTF_8).toString(); } // CacheContext is the mechanism to pass the cache related context to the source filesystem CacheContext cacheContext = PrestoCacheContext.build(cacheIdentifier, hiveFileContext, cacheQuotaEnabled); diff --git a/presto-hdfs-core/src/main/java/com/facebook/presto/hive/CacheQuota.java b/presto-hdfs-core/src/main/java/com/facebook/presto/hive/CacheQuota.java index 45338a390029..f7d26cdba182 100644 --- a/presto-hdfs-core/src/main/java/com/facebook/presto/hive/CacheQuota.java +++ b/presto-hdfs-core/src/main/java/com/facebook/presto/hive/CacheQuota.java @@ -18,7 +18,7 @@ import java.util.Objects; import java.util.Optional; -import static com.google.common.hash.Hashing.md5; +import static com.google.common.hash.Hashing.sha256; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; @@ -33,7 +33,7 @@ public class CacheQuota public CacheQuota(String identity, Optional quota) { this.identity = requireNonNull(identity, "identity is null"); - this.identifier = md5().hashString(identity, UTF_8).asLong(); + this.identifier = sha256().hashString(identity, UTF_8).asLong(); this.quota = requireNonNull(quota, "quota is null"); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java index dd5f3d581134..4dcb404f5342 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowToRowCast.java @@ -100,13 +100,13 @@ private static Class generateRowCast(Type fromType, Type toType, FunctionAndT CallSiteBinder binder = new CallSiteBinder(); - // Embed the MD5 hash code of input and output types into the generated class name instead of the raw type names, + // Embed the SHA256 hash code of input and output types into the generated class name instead of the raw type names, // which could prevent the class name from hitting the length limitation and invalid characters. - byte[] md5Suffix = Hashing.md5().hashBytes((fromType + "$" + toType).getBytes()).asBytes(); + byte[] sha256Suffix = Hashing.sha256().hashBytes((fromType + "$" + toType).getBytes()).asBytes(); ClassDefinition definition = new ClassDefinition( a(PUBLIC, FINAL), - makeClassName(Joiner.on("$").join("RowCast", BaseEncoding.base16().encode(md5Suffix))), + makeClassName(Joiner.on("$").join("RowCast", BaseEncoding.base16().encode(sha256Suffix))), type(Object.class)); Parameter properties = arg("properties", SqlFunctionProperties.class);