Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -40,15 +40,22 @@
import com.facebook.presto.spi.type.TypeSignature;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
import com.facebook.presto.sql.analyzer.TypeSignatureProvider;
import com.facebook.presto.sql.gen.CacheStatsMBean;
import com.facebook.presto.sql.tree.QualifiedName;
import com.facebook.presto.transaction.TransactionId;
import com.facebook.presto.transaction.TransactionManager;
import com.facebook.presto.type.TypeRegistry;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Ordering;
import com.google.common.util.concurrent.UncheckedExecutionException;
import org.weakref.jmx.Managed;
import org.weakref.jmx.Nested;

import javax.annotation.concurrent.ThreadSafe;
import javax.inject.Inject;
Expand All @@ -58,6 +65,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -101,6 +109,8 @@ public class FunctionManager
private final HandleResolver handleResolver;
private final Map<CatalogSchemaPrefix, String> functionNamespaces = new ConcurrentHashMap<>();
private final Map<String, FunctionNamespaceManager<?>> functionNamespaceManagers = new ConcurrentHashMap<>();
private final LoadingCache<FunctionResolutionCacheKey, FunctionHandle> functionCache;
private final CacheStatsMBean cacheStatsMBean;

@Inject
public FunctionManager(
Expand All @@ -122,6 +132,11 @@ public FunctionManager(
}
// TODO: Provide a more encapsulated way for TransactionManager to register FunctionNamespaceManager
transactionManager.registerFunctionNamespaceManager(BuiltInFunctionNamespaceManager.ID, builtInFunctionNamespaceManager);
this.functionCache = CacheBuilder.newBuilder()
.recordStats()
.maximumSize(1000)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We have 500+ functions, with high chances of different parameterTypes resolving to the same function, the cache could easily exceeds 1k entries after a reasonable amount of queries are executable. What about giving it a more generous size, like 10k? Or even better, make else it configurable?

Copy link
Contributor Author

@rongrong rongrong Jan 21, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The goal is not to cache everything. Caching the top use cases is enough. The major target is the operators that's used over and over again in query planning (e.g., operator$equal(varchar, varchar) is used for every partition in PickTableLayout. Shadow testing showed that depending on the workload, No.1 hit can be 1 to 3 orders of magnitude higher than No.5. The overhead of resolving one function that's actually used in query is not that much. It's not clear to me that the processing time saved would worth the memory spent for the majority of the functions.

.build(CacheLoader.from(key -> resolveBuiltInFunction(key.functionName, fromTypeSignatures(key.parameterTypes))));
this.cacheStatsMBean = new CacheStatsMBean(functionCache);
}

@VisibleForTesting
Expand All @@ -131,6 +146,13 @@ public FunctionManager(TypeManager typeManager, BlockEncodingSerde blockEncoding
this(typeManager, createTestTransactionManager(), blockEncodingSerde, featuresConfig, new HandleResolver());
}

@Managed
@Nested
public CacheStatsMBean getFunctionResolutionCacheStats()
{
return cacheStatsMBean;
}

public void loadFunctionNamespaceManager(
String functionNamespaceManagerName,
String functionNamespaceManagerId,
Expand Down Expand Up @@ -237,6 +259,14 @@ public FunctionHandle resolveFunction(Optional<TransactionId> transactionId, Qua
}

public FunctionHandle resolveFunction(Optional<TransactionId> transactionId, QualifiedFunctionName functionName, List<TypeSignatureProvider> parameterTypes)
{
if (functionName.getFunctionNamespace().equals(DEFAULT_NAMESPACE) && parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency)) {
return lookupCachedFunction(functionName, parameterTypes);
}
return resolveFunctionInternal(transactionId, functionName, parameterTypes);
}

private FunctionHandle resolveFunctionInternal(Optional<TransactionId> transactionId, QualifiedFunctionName functionName, List<TypeSignatureProvider> parameterTypes)
{
Optional<String> functionNamespaceManagerId = getServingFunctionNamespaceManagerId(functionName.getFunctionNamespace());
if (!functionNamespaceManagerId.isPresent()) {
Expand Down Expand Up @@ -278,6 +308,13 @@ public FunctionHandle resolveFunction(Optional<TransactionId> transactionId, Qua
throw new PrestoException(FUNCTION_NOT_FOUND, constructFunctionNotFoundErrorMessage(functionName, parameterTypes, candidates));
}

private FunctionHandle resolveBuiltInFunction(QualifiedFunctionName functionName, List<TypeSignatureProvider> parameterTypes)
{
checkArgument(functionName.getFunctionNamespace().equals(DEFAULT_NAMESPACE), "Expect built-in functions");
checkArgument(parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency), "Expect parameter types not to have dependency");
return resolveFunctionInternal(Optional.empty(), functionName, parameterTypes);
}

@Override
public FunctionMetadata getFunctionMetadata(FunctionHandle functionHandle)
{
Expand Down Expand Up @@ -348,6 +385,9 @@ public FunctionHandle resolveOperator(OperatorType operatorType, List<TypeSignat
public FunctionHandle lookupFunction(String name, List<TypeSignatureProvider> parameterTypes)
{
QualifiedFunctionName functionName = QualifiedFunctionName.of(DEFAULT_NAMESPACE, name);
if (parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency)) {
return lookupCachedFunction(functionName, parameterTypes);
}
Collection<? extends SqlFunction> candidates = builtInFunctionNamespaceManager.getFunctions(Optional.empty(), functionName);
return lookupFunction(builtInFunctionNamespaceManager, Optional.empty(), functionName, parameterTypes, candidates);
}
Expand All @@ -368,6 +408,19 @@ public FunctionHandle lookupCast(CastType castType, TypeSignature fromType, Type
return builtInFunctionNamespaceManager.getFunctionHandle(Optional.empty(), signature);
}

private FunctionHandle lookupCachedFunction(QualifiedFunctionName functionName, List<TypeSignatureProvider> parameterTypes)
{
try {
return functionCache.getUnchecked(new FunctionResolutionCacheKey(functionName, parameterTypes));
}
catch (UncheckedExecutionException e) {
if (e.getCause() instanceof PrestoException) {
throw (PrestoException) e.getCause();
}
throw e;
}
}

private FunctionHandle lookupFunction(
FunctionNamespaceManager<?> functionNamespaceManager,
Optional<? extends FunctionNamespaceTransactionHandle> transactionHandle,
Expand Down Expand Up @@ -688,4 +741,48 @@ public String toString()
.toString();
}
}

private static class FunctionResolutionCacheKey
{
private final QualifiedFunctionName functionName;
private final List<TypeSignature> parameterTypes;

private FunctionResolutionCacheKey(QualifiedFunctionName functionName, List<TypeSignatureProvider> parameterTypes)
{
checkArgument(parameterTypes.stream().noneMatch(TypeSignatureProvider::hasDependency), "Only type signatures without dependency can be cached");
this.functionName = requireNonNull(functionName, "functionName is null");
this.parameterTypes = requireNonNull(parameterTypes, "parameterTypes is null").stream()
.map(TypeSignatureProvider::getTypeSignature)
.collect(toImmutableList());
}

@Override
public int hashCode()
{
return Objects.hash(functionName, parameterTypes);
}

@Override
public boolean equals(Object obj)
{
if (this == obj) {
return true;
}
if (obj == null || getClass() != obj.getClass()) {
return false;
}
FunctionResolutionCacheKey other = (FunctionResolutionCacheKey) obj;
return Objects.equals(this.functionName, other.functionName) &&
Objects.equals(this.parameterTypes, other.parameterTypes);
}

@Override
public String toString()
{
return toStringHelper(this)
.add("functionName", functionName)
.add("parameterTypes", parameterTypes)
.toString();
}
}
}