diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index fab2388ba222e..46ea0d95cce40 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -219,6 +219,7 @@ public final class SystemSessionProperties public static final String VERBOSE_RUNTIME_STATS_ENABLED = "verbose_runtime_stats_enabled"; public static final String STREAMING_FOR_PARTIAL_AGGREGATION_ENABLED = "streaming_for_partial_aggregation_enabled"; public static final String MAX_STAGE_COUNT_FOR_EAGER_SCHEDULING = "max_stage_count_for_eager_scheduling"; + public static final String HYPERLOGLOG_STANDARD_ERROR_WARNING_THRESHOLD = "hyperloglog_standard_error_warning_threshold"; //TODO: Prestissimo related session properties that are temporarily put here. They will be relocated in the future public static final String PRESTISSIMO_SIMPLIFIED_EXPRESSION_EVALUATION_ENABLED = "simplified_expression_evaluation_enabled"; @@ -1235,6 +1236,11 @@ public SystemSessionProperties( MAX_STAGE_COUNT_FOR_EAGER_SCHEDULING, "Maximum stage count to use eager scheduling when using the adaptive scheduling policy", featuresConfig.getMaxStageCountForEagerScheduling(), + false), + doubleProperty( + HYPERLOGLOG_STANDARD_ERROR_WARNING_THRESHOLD, + "Threshold for obtaining precise results from aggregation functions", + featuresConfig.getHyperloglogStandardErrorWarningThreshold(), false)); } @@ -2075,4 +2081,9 @@ public static int getMaxStageCountForEagerScheduling(Session session) { return session.getSystemProperty(MAX_STAGE_COUNT_FOR_EAGER_SCHEDULING, Integer.class); } + + public static double getHyperloglogStandardErrorWarningThreshold(Session session) + { + return session.getSystemProperty(HYPERLOGLOG_STANDARD_ERROR_WARNING_THRESHOLD, Double.class); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/warnings/DefaultWarningCollector.java b/presto-main/src/main/java/com/facebook/presto/execution/warnings/DefaultWarningCollector.java index de4ce3d247223..ccbdf7fd480a6 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/warnings/DefaultWarningCollector.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/warnings/DefaultWarningCollector.java @@ -18,13 +18,13 @@ import com.facebook.presto.spi.WarningCode; import com.facebook.presto.spi.WarningCollector; import com.google.common.collect.ImmutableList; +import com.google.common.collect.LinkedHashMultimap; +import com.google.common.collect.Multimap; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import static com.facebook.presto.spi.StandardErrorCode.WARNING_AS_ERROR; import static com.facebook.presto.spi.StandardWarningCode.PARSER_WARNING; @@ -35,7 +35,7 @@ public class DefaultWarningCollector implements WarningCollector { @GuardedBy("this") - private final Map warnings = new LinkedHashMap<>(); + private final Multimap warnings = LinkedHashMultimap.create(); private final WarningCollectorConfig config; private final WarningHandlingLevel warningHandlingLevel; @@ -77,7 +77,7 @@ public synchronized List getWarnings() private synchronized void addWarningIfNumWarningsLessThanConfig(PrestoWarning warning) { if (warnings.size() < config.getMaxWarnings()) { - warnings.putIfAbsent(warning.getWarningCode(), warning); + warnings.put(warning.getWarningCode(), warning); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DefaultApproximateCountDistinctAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DefaultApproximateCountDistinctAggregation.java index 069b719ec6bdf..0f053366797bf 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DefaultApproximateCountDistinctAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DefaultApproximateCountDistinctAggregation.java @@ -37,7 +37,7 @@ @AggregationFunction("approx_distinct") public final class DefaultApproximateCountDistinctAggregation { - private static final double DEFAULT_STANDARD_ERROR = 0.023; + public static final double DEFAULT_STANDARD_ERROR = 0.023; private DefaultApproximateCountDistinctAggregation() {} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java index 80141c3f11ef8..4f023024fc6a8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java @@ -13,7 +13,11 @@ */ package com.facebook.presto.sql.analyzer; +import com.facebook.presto.Session; +import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.operator.aggregation.ApproximateSetAggregation; +import com.facebook.presto.operator.aggregation.DefaultApproximateCountDistinctAggregation; import com.facebook.presto.spi.PrestoWarning; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.sql.planner.ParameterRewriter; @@ -30,6 +34,7 @@ import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.CurrentTime; import com.facebook.presto.sql.tree.DereferenceExpression; +import com.facebook.presto.sql.tree.DoubleLiteral; import com.facebook.presto.sql.tree.ExistsPredicate; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; @@ -114,6 +119,7 @@ class AggregationAnalyzer private final Scope sourceScope; private final Optional orderByScope; private final WarningCollector warningCollector; + private final Session session; private final FunctionResolution functionResolution; public static void verifySourceAggregations( @@ -122,9 +128,10 @@ public static void verifySourceAggregations( Expression expression, Metadata metadata, Analysis analysis, - WarningCollector warningCollector) + WarningCollector warningCollector, + Session session) { - AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, sourceScope, Optional.empty(), metadata, analysis, warningCollector); + AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, sourceScope, Optional.empty(), metadata, analysis, warningCollector, session); analyzer.analyze(expression); } @@ -135,13 +142,14 @@ public static void verifyOrderByAggregations( Expression expression, Metadata metadata, Analysis analysis, - WarningCollector warningCollector) + WarningCollector warningCollector, + Session session) { - AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, sourceScope, Optional.of(orderByScope), metadata, analysis, warningCollector); + AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, sourceScope, Optional.of(orderByScope), metadata, analysis, warningCollector, session); analyzer.analyze(expression); } - private AggregationAnalyzer(List groupByExpressions, Scope sourceScope, Optional orderByScope, Metadata metadata, Analysis analysis, WarningCollector warningCollector) + private AggregationAnalyzer(List groupByExpressions, Scope sourceScope, Optional orderByScope, Metadata metadata, Analysis analysis, WarningCollector warningCollector, Session session) { requireNonNull(groupByExpressions, "groupByExpressions is null"); requireNonNull(sourceScope, "sourceScope is null"); @@ -149,12 +157,14 @@ private AggregationAnalyzer(List groupByExpressions, Scope sourceSco requireNonNull(metadata, "metadata is null"); requireNonNull(analysis, "analysis is null"); requireNonNull(warningCollector, "warningCollector is null"); + requireNonNull(session, "session is null"); this.sourceScope = sourceScope; - this.warningCollector = warningCollector; this.orderByScope = orderByScope; this.metadata = metadata; this.analysis = analysis; + this.warningCollector = warningCollector; + this.session = session; this.functionResolution = new FunctionResolution(metadata.getFunctionAndTypeManager()); this.expressions = groupByExpressions.stream() .map(e -> ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters()), e)) @@ -335,6 +345,34 @@ protected Boolean visitFunctionCall(FunctionCall node, Void context) PERFORMANCE_WARNING, "COUNT(DISTINCT xxx) can be a very expensive operation when the cardinality is high for xxx. In most scenarios, using approx_distinct instead would be enough")); } + if (functionResolution.isApproximateCountDistinctFunction(analysis.getFunctionHandle(node))) { + double maxStandardError = DefaultApproximateCountDistinctAggregation.DEFAULT_STANDARD_ERROR; + double lowestMaxStandardError = SystemSessionProperties.getHyperloglogStandardErrorWarningThreshold(session); + // maxStandardError is supplied + if (node.getArguments().size() > 1) { + Expression maxStandardErrorExpr = node.getArguments().get(1); + if (maxStandardErrorExpr instanceof DoubleLiteral) { + maxStandardError = ((DoubleLiteral) maxStandardErrorExpr).getValue(); + } + } + if (maxStandardError <= lowestMaxStandardError) { + warningCollector.add(new PrestoWarning(PERFORMANCE_WARNING, String.format("approx_distinct can produce low-precision results with the current standard error: %.4f (<=%.4f)", maxStandardError, lowestMaxStandardError))); + } + } + if (functionResolution.isApproximateSetFunction(analysis.getFunctionHandle(node))) { + double maxStandardError = ApproximateSetAggregation.DEFAULT_STANDARD_ERROR; + double lowestMaxStandardError = SystemSessionProperties.getHyperloglogStandardErrorWarningThreshold(session); + // maxStandardError is supplied + if (node.getArguments().size() > 1) { + Expression maxStandardErrorExpr = node.getArguments().get(1); + if (maxStandardErrorExpr instanceof DoubleLiteral) { + maxStandardError = ((DoubleLiteral) maxStandardErrorExpr).getValue(); + } + } + if (maxStandardError <= lowestMaxStandardError) { + warningCollector.add(new PrestoWarning(PERFORMANCE_WARNING, String.format("approx_set can produce low-precision results with the current standard error: %.4f (<=%.4f)", maxStandardError, lowestMaxStandardError))); + } + } if (!node.getWindow().isPresent()) { List aggregateFunctions = extractAggregateFunctions(analysis.getFunctionHandles(), node.getArguments(), metadata.getFunctionAndTypeManager()); List windowFunctions = extractWindowFunctions(node.getArguments()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index ff22b1f59c8a5..1c2c4e4aabb95 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -218,6 +218,8 @@ public class FeaturesConfig private int maxStageCountForEagerScheduling = 25; + private double hyperloglogStandardErrorWarningThreshold = 0.004; + public enum PartitioningPrecisionStrategy { // Let Presto decide when to repartition @@ -1985,4 +1987,17 @@ public FeaturesConfig setMaxStageCountForEagerScheduling(int maxStageCountForEag this.maxStageCountForEagerScheduling = maxStageCountForEagerScheduling; return this; } + + public double getHyperloglogStandardErrorWarningThreshold() + { + return hyperloglogStandardErrorWarningThreshold; + } + + @Config("hyperloglog-standard-error-warning-threshold") + @ConfigDescription("aggregation functions can produce low-precision results when the max standard error lower than this value.") + public FeaturesConfig setHyperloglogStandardErrorWarningThreshold(double hyperloglogStandardErrorWarningThreshold) + { + this.hyperloglogStandardErrorWarningThreshold = hyperloglogStandardErrorWarningThreshold; + return this; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index ffa8e7df41f73..e1bfba6b4c2f6 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -2554,11 +2554,11 @@ private void verifyAggregations( .collect(toImmutableList()); for (Expression expression : outputExpressions) { - verifySourceAggregations(distinctGroupingColumns, sourceScope, expression, metadata, analysis, warningCollector); + verifySourceAggregations(distinctGroupingColumns, sourceScope, expression, metadata, analysis, warningCollector, session); } for (Expression expression : orderByExpressions) { - verifyOrderByAggregations(distinctGroupingColumns, sourceScope, orderByScope.get(), expression, metadata, analysis, warningCollector); + verifyOrderByAggregations(distinctGroupingColumns, sourceScope, orderByScope.get(), expression, metadata, analysis, warningCollector, session); } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java index 30ef42396c9bd..7c668fcc7afca 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/FunctionResolution.java @@ -303,4 +303,28 @@ public FunctionHandle minFunction(Type valueType) { return functionAndTypeManager.lookupFunction("min", fromTypes(valueType)); } + + @Override + public boolean isApproximateCountDistinctFunction(FunctionHandle functionHandle) + { + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "approx_distinct")); + } + + @Override + public FunctionHandle approximateCountDistinctFunction(Type valueType) + { + return functionAndTypeManager.lookupFunction("approx_distinct", fromTypes(valueType)); + } + + @Override + public boolean isApproximateSetFunction(FunctionHandle functionHandle) + { + return functionAndTypeManager.getFunctionMetadata(functionHandle).getName().equals(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "approx_set")); + } + + @Override + public FunctionHandle approximateSetFunction(Type valueType) + { + return functionAndTypeManager.lookupFunction("approx_set", fromTypes(valueType)); + } } diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingWarningCollector.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingWarningCollector.java index 2992c0b1a7cc8..7e8352d1c92dd 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingWarningCollector.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingWarningCollector.java @@ -19,13 +19,13 @@ import com.facebook.presto.spi.WarningCollector; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.common.collect.LinkedHashMultimap; +import com.google.common.collect.Multimap; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import static java.lang.String.format; @@ -36,7 +36,7 @@ public class TestingWarningCollector implements WarningCollector { @GuardedBy("this") - private final Map warnings = new LinkedHashMap<>(); + private final Multimap warnings = LinkedHashMultimap.create(); private final WarningCollectorConfig config; private final boolean addWarnings; @@ -59,7 +59,7 @@ public synchronized void add(PrestoWarning warning) { requireNonNull(warning, "warning is null"); if (warnings.size() < config.getMaxWarnings()) { - warnings.putIfAbsent(warning.getWarningCode(), warning); + warnings.put(warning.getWarningCode(), warning); } } diff --git a/presto-main/src/test/java/com/facebook/presto/execution/warnings/TestDefaultWarningCollector.java b/presto-main/src/test/java/com/facebook/presto/execution/warnings/TestDefaultWarningCollector.java index bc3d0e4442682..c4fcd58ba6e23 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/warnings/TestDefaultWarningCollector.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/warnings/TestDefaultWarningCollector.java @@ -42,6 +42,17 @@ public void testMaxWarnings() assertEquals(warningCollector.getWarnings().size(), 2); } + @Test + public void testAddingSameTypeWarnings() + { + WarningCollector warningCollector = new DefaultWarningCollector(new WarningCollectorConfig(), WarningHandlingLevel.NORMAL); + warningCollector.add(new PrestoWarning(new WarningCode(1, "1"), "warning 1-1")); + warningCollector.add(new PrestoWarning(new WarningCode(1, "1"), "warning 1-2")); + warningCollector.add(new PrestoWarning(new WarningCode(2, "2"), "warning 2")); + warningCollector.add(new PrestoWarning(new WarningCode(3, "3"), "warning 3")); + assertEquals(warningCollector.getWarnings().size(), 4); + } + @Test public void testWarningSuppress() { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java index 85002f83c561c..e481e5a1dffaf 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java @@ -91,7 +91,7 @@ public class TestAnalyzer private static void assertHasWarning(WarningCollector warningCollector, StandardWarningCode code, String match) { List warnings = warningCollector.getWarnings(); - assertEquals(warnings.size(), 1); + assertTrue(warnings.size() > 0); PrestoWarning warning = warnings.get(0); assertEquals(warning.getWarningCode(), code.toWarningCode()); assertTrue(warning.getMessage().startsWith(match)); @@ -1115,6 +1115,66 @@ public void testCountDistinctPerformanceWarning() assertTrue(warning.getMessage().contains("COUNT(DISTINCT xxx)")); } + @Test + public void testApproxDistinctPerformanceWarning() + { + WarningCollector warningCollector = analyzeWithWarnings("SELECT approx_distinct(a) FROM t1 GROUP BY b"); + List warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 0); + + warningCollector = analyzeWithWarnings("SELECT approx_distinct(a, 0.0025E0) FROM t1 GROUP BY b"); + warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 1); + + // Ensure warning is the performance warning we expect + PrestoWarning warning = warnings.get(0); + assertEquals(warning.getWarningCode(), PERFORMANCE_WARNING.toWarningCode()); + assertTrue(warning.getMessage().startsWith("approx_distinct")); + + // Ensure warning is only issued for values lower than threshold + warningCollector = analyzeWithWarnings("SELECT approx_distinct(a, 0.0055E0) FROM t1 GROUP BY b"); + warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 0); + } + + @Test + public void testApproxSetPerformanceWarning() + { + WarningCollector warningCollector = analyzeWithWarnings("SELECT approx_set(a) FROM t1 GROUP BY b"); + List warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 0); + + warningCollector = analyzeWithWarnings("SELECT approx_set(a, 0.0015E0) FROM t1 GROUP BY b"); + warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 1); + + // Ensure warning is the performance warning we expect + PrestoWarning warning = warnings.get(0); + assertEquals(warning.getWarningCode(), PERFORMANCE_WARNING.toWarningCode()); + assertTrue(warning.getMessage().startsWith("approx_set")); + + // Ensure warning is only issued for values lower than threshold + warningCollector = analyzeWithWarnings("SELECT approx_set(a, 0.0055E0) FROM t1 GROUP BY b"); + warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 0); + } + + @Test + public void testApproxDistinctAndApproxSetPerformanceWarning() + { + WarningCollector warningCollector = analyzeWithWarnings("SELECT approx_distinct(a, 0.0025E0), approx_set(a, 0.0013E0) FROM t1 GROUP BY b"); + List warnings = warningCollector.getWarnings(); + assertEquals(warnings.size(), 2); + + // Ensure warnings are the performance warnings we expect + PrestoWarning approxDistinctWarning = warnings.get(0); + assertEquals(approxDistinctWarning.getWarningCode(), PERFORMANCE_WARNING.toWarningCode()); + assertTrue(approxDistinctWarning.getMessage().startsWith("approx_distinct")); + PrestoWarning approxSetWarning = warnings.get(1); + assertEquals(approxSetWarning.getWarningCode(), PERFORMANCE_WARNING.toWarningCode()); + assertTrue(approxSetWarning.getMessage().startsWith("approx_set")); + } + @Test public void testUnionNoPerformanceWarning() { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index e097ff8e04377..1f929a422227e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -189,7 +189,8 @@ public void testDefaults() .setHashBasedDistinctLimitEnabled(false) .setHashBasedDistinctLimitThreshold(10000) .setStreamingForPartialAggregationEnabled(false) - .setMaxStageCountForEagerScheduling(25)); + .setMaxStageCountForEagerScheduling(25) + .setHyperloglogStandardErrorWarningThreshold(0.004)); } @Test @@ -330,6 +331,7 @@ public void testExplicitPropertyMappings() .put("hash-based-distinct-limit-threshold", "500") .put("streaming-for-partial-aggregation-enabled", "true") .put("execution-policy.max-stage-count-for-eager-scheduling", "123") + .put("hyperloglog-standard-error-warning-threshold", "0.02") .build(); FeaturesConfig expected = new FeaturesConfig() @@ -467,7 +469,8 @@ public void testExplicitPropertyMappings() .setHashBasedDistinctLimitEnabled(true) .setHashBasedDistinctLimitThreshold(500) .setStreamingForPartialAggregationEnabled(true) - .setMaxStageCountForEagerScheduling(123); + .setMaxStageCountForEagerScheduling(123) + .setHyperloglogStandardErrorWarningThreshold(0.02); assertFullMapping(properties, expected); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/function/StandardFunctionResolution.java b/presto-spi/src/main/java/com/facebook/presto/spi/function/StandardFunctionResolution.java index 27c50edce7b40..5cffddd8337d5 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/function/StandardFunctionResolution.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/function/StandardFunctionResolution.java @@ -71,4 +71,12 @@ public interface StandardFunctionResolution boolean isMinFunction(FunctionHandle functionHandle); FunctionHandle minFunction(Type valueType); + + boolean isApproximateCountDistinctFunction(FunctionHandle functionHandle); + + FunctionHandle approximateCountDistinctFunction(Type valueType); + + boolean isApproximateSetFunction(FunctionHandle functionHandle); + + FunctionHandle approximateSetFunction(Type valueType); }