Skip to content

Commit

Permalink
[Iceberg] Add histogram statistic support
Browse files Browse the repository at this point in the history
Utilizes the sketch_kll function to generate histograms and store them
into the Iceberg table's puffin files for table-level statistic storage.

Histograms are always collected by ANALYZE, but they are not used by the
cost calculator unless enabled via optimizer.use-histograms
  • Loading branch information
ZacBlanco authored and tdcmeehan committed Dec 16, 2024
1 parent 22a70a7 commit c4afd0b
Show file tree
Hide file tree
Showing 39 changed files with 1,441 additions and 329 deletions.
70 changes: 67 additions & 3 deletions presto-common/src/main/java/com/facebook/presto/common/Utils.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,14 @@
import com.facebook.presto.common.predicate.Primitives;
import com.facebook.presto.common.type.Type;

import javax.annotation.Nullable;

import java.util.function.Supplier;

import static com.facebook.presto.common.type.TypeUtils.readNativeValue;
import static com.facebook.presto.common.type.TypeUtils.writeNativeValue;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public final class Utils
{
Expand All @@ -30,7 +36,7 @@ private Utils()
public static Block nativeValueToBlock(Type type, Object object)
{
if (object != null && !Primitives.wrap(type.getJavaType()).isInstance(object)) {
throw new IllegalArgumentException(String.format("Object '%s' does not match type %s", object, type.getJavaType()));
throw new IllegalArgumentException(format("Object '%s' does not match type %s", object, type.getJavaType()));
}
BlockBuilder blockBuilder = type.createBlockBuilder(null, 1);
writeNativeValue(type, blockBuilder, object);
Expand All @@ -49,10 +55,68 @@ public static void checkArgument(boolean expression)
}
}

public static void checkArgument(boolean expression, String errorMessage)
public static void checkArgument(boolean expression, String message, Object... args)
{
if (!expression) {
throw new IllegalArgumentException(errorMessage);
throw new IllegalArgumentException(format(message, args));
}
}

/**
* Returns a supplier which caches the instance retrieved during the first call to {@code get()}
* and returns that value on subsequent calls to {@code get()}.
*/
public static <T> Supplier<T> memoizedSupplier(Supplier<T> delegate)
{
if (delegate instanceof MemoizingSupplier) {
return delegate;
}
return new MemoizingSupplier<>(delegate);
}

/**
* Vendored from Guava
*/
static class MemoizingSupplier<T>
implements Supplier<T>
{
volatile Supplier<T> delegate;
volatile boolean initialized;
// "value" does not need to be volatile; visibility piggy-backs
// on volatile read of "initialized".
@Nullable T value;

MemoizingSupplier(Supplier<T> delegate)
{
this.delegate = requireNonNull(delegate);
}

@Override
public T get()
{
// A 2-field variant of Double Checked Locking.
if (!initialized) {
synchronized (this) {
if (!initialized) {
T t = delegate.get();
value = t;
initialized = true;
// Release the delegate to GC.
delegate = null;
return t;
}
}
}
return value;
}

@Override
public String toString()
{
Supplier<T> delegate = this.delegate;
return "Suppliers.memoize("
+ (delegate == null ? "<supplier that returned " + value + ">" : delegate)
+ ")";
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,11 @@ public Object getValue()
return Utils.blockToNativeValue(type, valueBlock.get());
}

public Optional<Object> getObjectValue()
{
return valueBlock.map(block -> Utils.blockToNativeValue(type, block));
}

public Object getPrintableValue(SqlFunctionProperties properties)
{
if (!valueBlock.isPresent()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,16 @@ public boolean equals(Object obj)
Objects.equals(this.high, other.high);
}

@Override
public String toString()
{
return (low.getBound() == Marker.Bound.EXACTLY ? "[" : "(") +
low.getObjectValue().orElse(Double.NEGATIVE_INFINITY) +
".." +
high.getObjectValue().orElse(Double.POSITIVE_INFINITY) +
(high.getBound() == Marker.Bound.EXACTLY ? "]" : ")");
}

private void appendQuotedValue(StringBuilder buffer, Marker marker, SqlFunctionProperties properties)
{
buffer.append('"');
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,28 @@ public Object getSingleValue()
return lowIndexedRanges.values().iterator().next().getSingleValue();
}

/**
* Build a new {@link SortedRangeSet} that contains ranges which lie within the argument range
*
* @param span the range which the new set should span
* @return a new range set
*/
public SortedRangeSet subRangeSet(Range span)
{
Builder builder = new Builder(type);

for (Range range : getOrderedRanges()) {
if (span.contains(range)) {
builder.add(range);
}
else if (span.overlaps(range)) {
builder.add(range.intersect(span));
}
}

return builder.build();
}

@Override
public boolean containsValue(Object value)
{
Expand Down
148 changes: 148 additions & 0 deletions presto-common/src/test/java/com/facebook/presto/common/TestUtils.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,148 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.common;

import org.testng.annotations.Test;

import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Supplier;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertSame;
import static org.testng.Assert.fail;

public class TestUtils
{
@Test
public void testCheckArgumentFailWithMessage()
{
try {
Utils.checkArgument(false, "test %s", "test");
fail();
}
catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "test test");
}
}

@Test
public void testCheckArgumentPassWithMessage()
{
try {
Utils.checkArgument(true, "test %s", "test");
}
catch (IllegalArgumentException e) {
fail();
}
}

@Test
public void testMemoizedSupplierThreadSafe()
throws Throwable
{
Function<Supplier<Boolean>, Supplier<Boolean>> memoizer =
supplier -> Utils.memoizedSupplier(supplier);
testSupplierThreadSafe(memoizer);
}

/**
* Vendored from Guava
*/
private void testSupplierThreadSafe(Function<Supplier<Boolean>, Supplier<Boolean>> memoizer)
throws Throwable
{
final AtomicInteger count = new AtomicInteger(0);
final AtomicReference<Throwable> thrown = new AtomicReference<>(null);
final int numThreads = 3;
final Thread[] threads = new Thread[numThreads];
final long timeout = TimeUnit.SECONDS.toNanos(60);

final Supplier<Boolean> supplier =
new Supplier<Boolean>()
{
boolean isWaiting(Thread thread)
{
switch (thread.getState()) {
case BLOCKED:
case WAITING:
case TIMED_WAITING:
return true;
default:
return false;
}
}

int waitingThreads()
{
int waitingThreads = 0;
for (Thread thread : threads) {
if (isWaiting(thread)) {
waitingThreads++;
}
}
return waitingThreads;
}

@Override
@SuppressWarnings("ThreadPriorityCheck") // doing our best to test for races
public Boolean get()
{
// Check that this method is called exactly once, by the first
// thread to synchronize.
long t0 = System.nanoTime();
while (waitingThreads() != numThreads - 1) {
if (System.nanoTime() - t0 > timeout) {
thrown.set(
new TimeoutException(
"timed out waiting for other threads to block"
+ " synchronizing on supplier"));
break;
}
Thread.yield();
}
count.getAndIncrement();
return Boolean.TRUE;
}
};

final Supplier<Boolean> memoizedSupplier = memoizer.apply(supplier);

for (int i = 0; i < numThreads; i++) {
threads[i] =
new Thread()
{
@Override
public void run()
{
assertSame(Boolean.TRUE, memoizedSupplier.get());
}
};
}
for (Thread t : threads) {
t.start();
}
for (Thread t : threads) {
t.join();
}

if (thrown.get() != null) {
throw thrown.get();
}
assertEquals(1, count.get());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,16 @@ public void testCanonicalize()
assertDifferentMarker(Marker.upperUnbounded(BIGINT), Marker.lowerUnbounded(BIGINT), true);
}

@Test
public void testGetValue()
{
assertTrue(Marker.exactly(BIGINT, 1L).getObjectValue().isPresent());
assertTrue(Marker.above(BIGINT, 1L).getObjectValue().isPresent());
assertTrue(Marker.below(BIGINT, 1L).getObjectValue().isPresent());
assertFalse(Marker.upperUnbounded(BIGINT).getObjectValue().isPresent());
assertFalse(Marker.lowerUnbounded(BIGINT).getObjectValue().isPresent());
}

private void assertSameMarker(Marker marker1, Marker marker2, boolean removeConstants)
throws Exception
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@
import com.google.common.collect.Iterables;
import org.testng.annotations.Test;

import java.util.Arrays;
import java.util.stream.Collectors;

import static com.facebook.presto.common.type.BigintType.BIGINT;
import static com.facebook.presto.common.type.BooleanType.BOOLEAN;
import static com.facebook.presto.common.type.DoubleType.DOUBLE;
Expand Down Expand Up @@ -500,6 +503,65 @@ public void testCanonicalize()
assertDifferentSet(SortedRangeSet.all(BIGINT), SortedRangeSet.all(BOOLEAN), true);
}

@Test
public void testSubRangeSet()
{
// test subrange no overlap below and above
assertEquals(SortedRangeSet.of(Range.lessThan(BIGINT, 10L))
.subRangeSet(Range.greaterThan(BIGINT, 10L))
.getOrderedRanges()
.size(),
0);
assertEquals(SortedRangeSet.of(Range.greaterThan(BIGINT, 10L))
.subRangeSet(Range.lessThan(BIGINT, 10L))
.getOrderedRanges()
.size(),
0);
assertEquals(SortedRangeSet.of(Range.greaterThanOrEqual(BIGINT, 10L))
.subRangeSet(Range.lessThan(BIGINT, 10L))
.getOrderedRanges()
.size(),
0);
assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L))
.subRangeSet(Range.greaterThan(BIGINT, 10L))
.getOrderedRanges()
.size(),
0);

// test with equal bounds
assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L))
.subRangeSet(Range.greaterThanOrEqual(BIGINT, 10L))
.getOrderedRanges()
.size(),
1);
assertEquals(SortedRangeSet.of(Range.greaterThanOrEqual(BIGINT, 10L))
.subRangeSet(Range.lessThanOrEqual(BIGINT, 10L))
.getOrderedRanges()
.size(),
1);
assertEquals(SortedRangeSet.of(Range.lessThanOrEqual(BIGINT, 10L))
.subRangeSet(Range.greaterThanOrEqual(BIGINT, 10L))
.getOrderedRanges().get(0), Range.range(BIGINT, 10L, true, 10L, true));
// two ranges
assertEquals(SortedRangeSet.of(Range.lessThan(BIGINT, -10L), Range.greaterThan(BIGINT, 10L))
.subRangeSet(Range.range(BIGINT, -20L, true, 20L, true)).getOrderedRanges(),
Arrays.stream(new Range[] {
Range.range(BIGINT, -20L, true, -10L, false),
Range.range(BIGINT, 10L, false, 20L, true)})
.collect(Collectors.toList()));
// range entirely contained
assertEquals(SortedRangeSet.of(
Range.lessThan(BIGINT, -10L),
Range.greaterThan(BIGINT, 10L),
Range.range(BIGINT, -5L, true, 5L, true))
.subRangeSet(Range.range(BIGINT, -20L, true, 20L, true)).getOrderedRanges(),
Arrays.stream(new Range[] {
Range.range(BIGINT, -20L, true, -10L, false),
Range.range(BIGINT, -5L, true, 5L, true),
Range.range(BIGINT, 10L, false, 20L, true)})
.collect(Collectors.toList()));
}

private void assertSameSet(SortedRangeSet set1, SortedRangeSet set2, boolean removeSafeConstants)
throws Exception
{
Expand Down
Loading

0 comments on commit c4afd0b

Please sign in to comment.