From 83f62db4f6d17a2ee81450952424cfeb20f9c5ea Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 19 Jun 2024 13:16:16 -0700 Subject: [PATCH 01/18] fix!: statistical functions should return `null` when provided a vector of only `null` values (#5606) * Updated Numeric calculations and tests, corrected median and percentile to ignore NULL values. * Correct Numeric avg() function and tests. * UpdateBy corrections for all-NULL vectors * Agg corrections for avg,var,std * Better testing and coverage, fix to wvar() and misc cleanup. --- engine/function/src/templates/Numeric.ftl | 212 ++++++++++++++++-- engine/function/src/templates/TestNumeric.ftl | 157 +++++++++++-- .../table/impl/by/ByteChunkedAvgOperator.java | 10 +- .../table/impl/by/ByteChunkedVarOperator.java | 30 ++- .../table/impl/by/CharChunkedAvgOperator.java | 10 +- .../table/impl/by/CharChunkedVarOperator.java | 30 ++- .../impl/by/DoubleChunkedAvgOperator.java | 7 +- .../impl/by/DoubleChunkedReAvgOperator.java | 6 +- .../impl/by/DoubleChunkedVarOperator.java | 33 ++- .../impl/by/FloatChunkedAvgOperator.java | 7 +- .../impl/by/FloatChunkedReAvgOperator.java | 6 +- .../impl/by/FloatChunkedReVarOperator.java | 8 +- .../impl/by/FloatChunkedVarOperator.java | 33 ++- .../table/impl/by/IntChunkedAvgOperator.java | 10 +- .../table/impl/by/IntChunkedVarOperator.java | 30 ++- .../table/impl/by/LongChunkedAvgOperator.java | 10 +- .../table/impl/by/LongChunkedVarOperator.java | 30 ++- .../impl/by/ShortChunkedAvgOperator.java | 10 +- .../impl/by/ShortChunkedVarOperator.java | 30 ++- .../rollingavg/ByteRollingAvgOperator.java | 4 +- .../rollingavg/CharRollingAvgOperator.java | 4 +- .../rollingavg/DoubleRollingAvgOperator.java | 2 +- .../rollingavg/FloatRollingAvgOperator.java | 2 +- .../rollingavg/IntRollingAvgOperator.java | 4 +- .../rollingavg/LongRollingAvgOperator.java | 4 +- .../rollingavg/ShortRollingAvgOperator.java | 4 +- .../rollingstd/ByteRollingStdOperator.java | 7 +- .../rollingstd/CharRollingStdOperator.java | 7 +- .../rollingstd/DoubleRollingStdOperator.java | 7 +- .../rollingstd/FloatRollingStdOperator.java | 7 +- .../rollingstd/IntRollingStdOperator.java | 7 +- .../rollingstd/LongRollingStdOperator.java | 7 +- .../rollingstd/ShortRollingStdOperator.java | 7 +- .../table/impl/QueryTableAggregationTest.java | 12 +- ...leAggregationTestFormulaStaticMethods.java | 21 +- .../table/impl/updateby/TestRollingAvg.java | 11 +- .../table/impl/updateby/TestRollingCount.java | 10 +- .../impl/updateby/TestRollingFormula.java | 36 +-- .../table/impl/updateby/TestRollingGroup.java | 8 + .../impl/updateby/TestRollingMinMax.java | 8 + .../impl/updateby/TestRollingProduct.java | 11 +- .../table/impl/updateby/TestRollingStd.java | 11 +- .../table/impl/updateby/TestRollingSum.java | 2 +- .../table/impl/updateby/TestRollingWAvg.java | 8 + 44 files changed, 703 insertions(+), 177 deletions(-) diff --git a/engine/function/src/templates/Numeric.ftl b/engine/function/src/templates/Numeric.ftl index ccedc0d79d0..8710a195b83 100644 --- a/engine/function/src/templates/Numeric.ftl +++ b/engine/function/src/templates/Numeric.ftl @@ -14,6 +14,7 @@ import java.util.Arrays; import static io.deephaven.base.CompareUtils.compare; import static io.deephaven.util.QueryConstants.*; import static io.deephaven.function.Basic.*; +import static io.deephaven.function.Sort.*; import static io.deephaven.function.Cast.castDouble; /** @@ -364,20 +365,29 @@ public class Numeric { double sum = 0; double count = 0; + long nullCount = 0; try ( final ${pt.vectorIterator} vi = values.iterator() ) { while ( vi.hasNext() ) { final ${pt.primitive} c = vi.${pt.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(c)) { return Double.NaN; } + if (!isNull(c)) { sum += c; count++; + } else { + nullCount++; } } } + if (nullCount == values.size()) { + return NULL_DOUBLE; + } + return sum / count; } @@ -418,23 +428,32 @@ public class Numeric { double sum = 0; double count = 0; + long nullCount = 0; try ( final ${pt.vectorIterator} vi = values.iterator() ) { while ( vi.hasNext() ) { final ${pt.primitive} c = vi.${pt.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(c)) { return Double.NaN; } if (isInf(c)) { return Double.POSITIVE_INFINITY; } + if (!isNull(c)) { sum += Math.abs(c); count++; + } else { + nullCount++; } } } + if (nullCount == values.size()) { + return NULL_DOUBLE; + } + return sum / count; } @@ -485,20 +504,29 @@ public class Numeric { double sum = 0; double sum2 = 0; long count = 0; + long nullCount = 0; try ( final ${pt.vectorIterator} vi = values.iterator() ) { while ( vi.hasNext() ) { final ${pt.primitive} c = vi.${pt.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(c) || isInf(c)) { return Double.NaN; } + if (!isNull(c)) { sum += (double)c; sum2 += (double)c * (double)c; count++; + } else { + nullCount++; } } } + if (nullCount == values.size()) { + return NULL_DOUBLE; + } + // Return NaN if overflow or too few values to compute variance. if (count <= 1 || Double.isInfinite(sum) || Double.isInfinite(sum2)) { return Double.NaN; @@ -596,6 +624,8 @@ public class Numeric { double sum2 = 0; double count = 0; double count2 = 0; + long nullCount = 0; + long valueCount = 0; try ( final ${pt.vectorIterator} vi = values.iterator(); @@ -604,23 +634,34 @@ public class Numeric { while (vi.hasNext()) { final ${pt.primitive} c = vi.${pt.iteratorNext}(); final ${pt2.primitive} w = wi.${pt2.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(c) || isInf(c)) { return Double.NaN; } + + <#if pt2.valueType.isFloat > if (isNaN(w) || isInf(w)) { return Double.NaN; } + if (!isNull(c) && !isNull(w)) { sum += w * c; sum2 += w * c * c; count += w; count2 += w * w; + valueCount++; + } else { + nullCount++; } } } + if (nullCount == values.size()) { + return NULL_DOUBLE; + } + // Return NaN if overflow or too few values to compute variance. - if (count <= 1 || Double.isInfinite(sum) || Double.isInfinite(sum2)) { + if (valueCount <= 1 || Double.isInfinite(sum) || Double.isInfinite(sum2)) { return Double.NaN; } @@ -868,6 +909,11 @@ public class Numeric { throw new IllegalArgumentException("Incompatible input sizes: " + values.size() + ", " + weights.size()); } + final double s = wstd(values, weights); + if (s == NULL_DOUBLE) { + return NULL_DOUBLE; + } + // see https://stats.stackexchange.com/questions/25895/computing-standard-error-in-weighted-mean-estimation double sumw = 0; double sumw2 = 0; @@ -887,8 +933,7 @@ public class Numeric { } } - final double s = wstd(values, weights); - return s == NULL_DOUBLE ? NULL_DOUBLE : s * Math.sqrt(sumw2/sumw/sumw); + return s * Math.sqrt(sumw2/sumw/sumw); } @@ -996,7 +1041,15 @@ public class Numeric { } final double a = wavg(values, weights); + if (a == NULL_DOUBLE) { + return NULL_DOUBLE; + } + final double s = wste(values, weights); + if (s == NULL_DOUBLE) { + return NULL_DOUBLE; + } + return a / s; } @@ -1189,7 +1242,8 @@ public class Numeric { } /** - * Returns the median. + * Returns the median. {@code null} input values are ignored but {@code NaN} values will poison the computation, + * and {@code NaN} will be returned * * @param values values. * @return median. @@ -1199,7 +1253,8 @@ public class Numeric { } /** - * Returns the median. + * Returns the median. {@code null} input values are ignored but {@code NaN} values will poison the computation, + * and {@code NaN} will be returned * * @param values values. * @return median. @@ -1213,7 +1268,8 @@ public class Numeric { } /** - * Returns the median. + * Returns the median. {@code null} input values are ignored but {@code NaN} values will poison the computation, + * and {@code NaN} will be returned * * @param values values. * @return median. @@ -1226,18 +1282,71 @@ public class Numeric { int n = values.intSize("median"); if (n == 0) { - return Double.NaN; - } else { - ${pt.primitive}[] copy = values.copyToArray(); - Arrays.sort(copy); - if (n % 2 == 0) - return 0.5 * (copy[n / 2 - 1] + copy[n / 2]); - else return copy[n / 2]; + return NULL_DOUBLE; + } + + ${pt.primitive}[] sorted = values.copyToArray(); + Arrays.sort(sorted); + + <#if pt.valueType.isFloat > + if (isNaN(sorted[sorted.length - 1])) { + return Double.NaN; // Any NaN will pollute the result and NaN always sorted to the end. + } + + int nullStart = -1; + int nullCount = 0; + for (int i = 0; i < n; i++) { + final ${pt.primitive} val = sorted[i]; + if (val > ${pt.null}) { + break; // no more NULL possible + } + if (isNull(val)) { + nullCount++; + if (nullStart == -1) { + nullStart = i; + } + } + } + <#else> + int nullCount = 0; + for (int i = 0; i < n && isNull(sorted[i]); i++) { + nullCount++; } + + + if (nullCount == n) { + return NULL_DOUBLE; + } + if (nullCount > 0) { + n -= nullCount; + final int medianIndex = n / 2; + if (n % 2 == 0) { + <#if pt.valueType.isFloat > + final int idx1 = (medianIndex - 1) < nullStart ? (medianIndex - 1) : (medianIndex - 1) + nullCount; + final int idx2 = medianIndex < nullStart ? medianIndex : medianIndex + nullCount; + <#else> + final int idx1 = (medianIndex - 1) + nullCount; + final int idx2 = medianIndex + nullCount; + + return 0.5 * (sorted[idx1] + sorted[idx2]); + } + <#if pt.valueType.isFloat > + final int adjustedIndex = medianIndex < nullStart ? medianIndex : medianIndex + nullCount; + <#else> + final int adjustedIndex = medianIndex + nullCount; + + return sorted[adjustedIndex]; + } + final int medianIndex = n / 2; + if (n % 2 == 0) { + return 0.5 * (sorted[medianIndex - 1] + sorted[medianIndex]); + } + return sorted[medianIndex]; } /** - * Returns the percentile. + * Returns the percentile. {@code null} input values are ignored but {@code NaN} values will poison the computation, + * and {@code NaN} will be returned * * @param percentile percentile to compute. * @param values values. @@ -1252,7 +1361,8 @@ public class Numeric { } /** - * Returns the percentile. + * Returns the percentile. {@code null} input values are ignored but {@code NaN} values will poison the computation, + * and {@code NaN} will be returned * * @param percentile percentile to compute. * @param values values. @@ -1268,11 +1378,51 @@ public class Numeric { } int n = values.intSize("percentile"); - ${pt.primitive}[] copy = values.copyToArray(); - Arrays.sort(copy); + ${pt.primitive}[] sorted = values.copyToArray(); + Arrays.sort(sorted); + + <#if pt.valueType.isFloat > + if (isNaN(sorted[sorted.length - 1])) { + return ${pt.boxed}.NaN; // Any NaN will pollute the result and NaN always sorted to the end. + } + int nullStart = -1; + int nullCount = 0; + for (int i = 0; i < n; i++) { + final ${pt.primitive} val = sorted[i]; + if (val > ${pt.null}) { + break; // no more NULL possible + } + if (isNull(val)) { + nullCount++; + if (nullStart == -1) { + nullStart = i; + } + } + } + <#else> + int nullCount = 0; + for (int i = 0; i < n && isNull(sorted[i]); i++) { + nullCount++; + } + + + if (nullCount == n) { + return ${pt.null}; + } + if (nullCount > 0) { + n -= nullCount; + <#if pt.valueType.isFloat > + final int idx = (int) Math.round(percentile * (n - 1)); + final int adjustedIndex = idx < nullStart ? idx : idx + nullCount; + return sorted[adjustedIndex]; + <#else> + int idx = (int) Math.round(percentile * (n - 1)); + return sorted[idx + nullCount]; + + } int idx = (int) Math.round(percentile * (n - 1)); - return copy[idx]; + return sorted[idx]; } @@ -1344,6 +1494,7 @@ public class Numeric { double sum1 = 0; double sum01 = 0; double count = 0; + long nullCount = 0; try ( final ${pt.vectorIterator} v0i = values0.iterator(); @@ -1352,22 +1503,32 @@ public class Numeric { while (v0i.hasNext()) { final ${pt.primitive} v0 = v0i.${pt.iteratorNext}(); final ${pt2.primitive} v1 = v1i.${pt2.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(v0) || isInf(v0)) { return Double.NaN; } + + <#if pt2.valueType.isFloat > if (isNaN(v1) || isInf(v1)) { return Double.NaN; } + if (!isNull(v0) && !isNull(v1)) { sum0 += v0; sum1 += v1; sum01 += v0 * v1; count++; + } else { + nullCount++; } } } + if (nullCount == values0.size()) { + return NULL_DOUBLE; + } + return sum01 / count - sum0 * sum1 / count / count; } @@ -1438,6 +1599,7 @@ public class Numeric { double sum1Sq = 0; double sum01 = 0; double count = 0; + long nullCount = 0; try ( final ${pt.vectorIterator} v0i = values0.iterator(); @@ -1446,12 +1608,16 @@ public class Numeric { while (v0i.hasNext()) { final ${pt.primitive} v0 = v0i.${pt.iteratorNext}(); final ${pt2.primitive} v1 = v1i.${pt2.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(v0) || isInf(v0)) { return Double.NaN; } + + <#if pt2.valueType.isFloat > if (isNaN(v1) || isInf(v1)) { return Double.NaN; } + if (!isNull(v0) && !isNull(v1)) { sum0 += v0; @@ -1460,10 +1626,16 @@ public class Numeric { sum1Sq += v1 * v1; sum01 += v0 * v1; count++; + } else { + nullCount++; } } } + if (nullCount == values0.size()) { + return NULL_DOUBLE; + } + double cov = sum01 / count - sum0 * sum1 / count / count; double var0 = sum0Sq / count - sum0 * sum0 / count / count; double var1 = sum1Sq / count - sum1 * sum1 / count / count; @@ -2756,12 +2928,16 @@ public class Numeric { while (vi.hasNext()) { final ${pt.primitive} c = vi.${pt.iteratorNext}(); final ${pt2.primitive} w = wi.${pt2.iteratorNext}(); + <#if pt.valueType.isFloat > if (isNaN(c)) { return Double.NaN; } + + <#if pt2.valueType.isFloat > if (isNaN(w)) { return Double.NaN; } + if (!isNull(c) && !isNull(w)) { vsum += c * w; wsum += w; diff --git a/engine/function/src/templates/TestNumeric.ftl b/engine/function/src/templates/TestNumeric.ftl index 1805eb3a396..5e51d5445cd 100644 --- a/engine/function/src/templates/TestNumeric.ftl +++ b/engine/function/src/templates/TestNumeric.ftl @@ -91,25 +91,34 @@ public class TestNumeric extends BaseArrayTestCase { public void test${pt.boxed}Avg() { assertEquals(50.0, avg(new ${pt.primitive}[]{40, 50, 60})); assertEquals(45.5, avg(new ${pt.primitive}[]{40, 51})); - assertTrue(Double.isNaN(avg(new ${pt.primitive}[]{}))); - assertTrue(Double.isNaN(avg(new ${pt.primitive}[]{${pt.null}}))); + assertEquals(NULL_DOUBLE, avg(new ${pt.primitive}[]{})); + assertEquals(NULL_DOUBLE, avg(new ${pt.primitive}[]{${pt.null}})); assertEquals(10.0, avg(new ${pt.primitive}[]{5, ${pt.null}, 15})); assertEquals(NULL_DOUBLE, avg((${pt.primitive}[])null)); assertEquals(50.0, avg(new ${pt.boxed}[]{(${pt.primitive})40, (${pt.primitive})50, (${pt.primitive})60})); assertEquals(45.5, avg(new ${pt.boxed}[]{(${pt.primitive})40, (${pt.primitive})51})); - assertTrue(Double.isNaN(avg(new ${pt.boxed}[]{}))); - assertTrue(Double.isNaN(avg(new ${pt.boxed}[]{${pt.null}}))); + assertEquals(NULL_DOUBLE, avg(new ${pt.boxed}[]{})); + assertEquals(NULL_DOUBLE, avg(new ${pt.boxed}[]{${pt.null}})); assertEquals(10.0, avg(new ${pt.boxed}[]{(${pt.primitive})5, ${pt.null}, (${pt.primitive})15})); assertEquals(NULL_DOUBLE, avg((${pt.boxed}[])null)); assertEquals(50.0, avg(new ${pt.vectorDirect}(new ${pt.primitive}[]{40, 50, 60}))); assertEquals(45.5, avg(new ${pt.vectorDirect}(new ${pt.primitive}[]{40, 51}))); - assertTrue(Double.isNaN(avg(new ${pt.vectorDirect}()))); - assertTrue(Double.isNaN(avg(new ${pt.vectorDirect}(${pt.null})))); + assertEquals(NULL_DOUBLE, avg(new ${pt.vectorDirect}())); + assertEquals(NULL_DOUBLE, avg(new ${pt.vectorDirect}(${pt.null}))); assertEquals(10.0, avg(new ${pt.vectorDirect}(new ${pt.primitive}[]{5, ${pt.null}, 15}))); assertEquals(NULL_DOUBLE, avg((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, avg(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, avg(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + + <#if pt.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, avg(new ${pt.primitive}[]{40, ${pt.boxed}.NaN, 60})); + + // check that functions can be resolved with varargs assertEquals(45.0, avg((${pt.primitive})40, (${pt.primitive})50)); } @@ -117,25 +126,29 @@ public class TestNumeric extends BaseArrayTestCase { public void test${pt.boxed}AbsAvg() { assertEquals(50.0, absAvg(new ${pt.primitive}[]{40, (${pt.primitive}) 50, 60})); assertEquals(45.5, absAvg(new ${pt.primitive}[]{(${pt.primitive}) 40, 51})); - assertTrue(Double.isNaN(absAvg(new ${pt.primitive}[]{}))); - assertTrue(Double.isNaN(absAvg(new ${pt.primitive}[]{${pt.null}}))); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.primitive}[]{})); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.primitive}[]{${pt.null}})); assertEquals(10.0, absAvg(new ${pt.primitive}[]{(${pt.primitive}) 5, ${pt.null}, (${pt.primitive}) 15})); assertEquals(NULL_DOUBLE, absAvg((${pt.primitive}[])null)); assertEquals(50.0, absAvg(new ${pt.boxed}[]{(${pt.primitive})40, (${pt.primitive}) 50, (${pt.primitive})60})); assertEquals(45.5, absAvg(new ${pt.boxed}[]{(${pt.primitive}) 40, (${pt.primitive})51})); - assertTrue(Double.isNaN(absAvg(new ${pt.boxed}[]{}))); - assertTrue(Double.isNaN(absAvg(new ${pt.boxed}[]{${pt.null}}))); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.boxed}[]{})); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.boxed}[]{${pt.null}})); assertEquals(10.0, absAvg(new ${pt.boxed}[]{(${pt.primitive}) 5, ${pt.null}, (${pt.primitive}) 15})); assertEquals(NULL_DOUBLE, absAvg((${pt.boxed}[])null)); assertEquals(50.0, absAvg(new ${pt.vectorDirect}(new ${pt.primitive}[]{40, (${pt.primitive}) 50, 60}))); assertEquals(45.5, absAvg(new ${pt.vectorDirect}(new ${pt.primitive}[]{(${pt.primitive}) 40, 51}))); - assertTrue(Double.isNaN(absAvg(new ${pt.vectorDirect}()))); - assertTrue(Double.isNaN(absAvg(new ${pt.vectorDirect}(${pt.null})))); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.vectorDirect}())); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.vectorDirect}(${pt.null}))); assertEquals(10.0, absAvg(new ${pt.vectorDirect}((${pt.primitive}) 5, ${pt.null}, (${pt.primitive}) 15))); assertEquals(NULL_DOUBLE, absAvg((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, absAvg(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, absAvg(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + // check that functions can be resolved with varargs assertEquals(45.0, absAvg((${pt.primitive})40, (${pt.primitive})50)); } @@ -309,6 +322,18 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(var, var(new ${pt.vectorDirect}(v))); assertEquals(NULL_DOUBLE, var((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, var(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, var(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + + // verify size==1 + assertEquals(Double.NaN, var(new ${pt.primitive}[]{40})); + + <#if pt.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, var(new ${pt.primitive}[]{40, ${pt.boxed}.NaN, 60})); + + // check that functions can be resolved with varargs assertEquals(var, var((${pt.primitive})0, (${pt.primitive})40, ${pt.null}, (${pt.primitive})50, (${pt.primitive})60, (${pt.primitive}) -1, (${pt.primitive})0)); } @@ -326,6 +351,10 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(Math.sqrt(var(new ${pt.vectorDirect}(v))), std(new ${pt.vectorDirect}(v))); assertEquals(NULL_DOUBLE, std((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, std(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, std(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + // check that functions can be resolved with varargs assertEquals(std(v), std((${pt.primitive})0, (${pt.primitive})40, ${pt.null}, (${pt.primitive})50, (${pt.primitive})60, (${pt.primitive}) -1, (${pt.primitive})0)); } @@ -343,6 +372,10 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(std(new ${pt.vectorDirect}(v)) / Math.sqrt(count(new ${pt.vectorDirect}(v))), ste(new ${pt.vectorDirect}(v))); assertEquals(NULL_DOUBLE, ste((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, ste(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, ste(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + // check that functions can be resolved with varargs assertEquals(ste(v), ste((${pt.primitive})0, (${pt.primitive})40, ${pt.null}, (${pt.primitive})50, (${pt.primitive})60, (${pt.primitive}) -1, (${pt.primitive})0)); } @@ -360,6 +393,10 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(avg(new ${pt.vectorDirect}(v)) / ste(new ${pt.vectorDirect}(v)), tstat(new ${pt.vectorDirect}(v))); assertEquals(NULL_DOUBLE, tstat((${pt.vectorDirect})null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, tstat(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, tstat(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + // check that functions can be resolved with varargs assertEquals(tstat(v), tstat((${pt.primitive})0, (${pt.primitive})40, ${pt.null}, (${pt.primitive})50, (${pt.primitive})60, (${pt.primitive}) -1, (${pt.primitive})0)); } @@ -509,6 +546,19 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(NULL_DOUBLE, cov((${pt.vectorDirect})null, new ${pt2.vectorDirect}(b))); assertEquals(NULL_DOUBLE, cov((${pt.vectorDirect})null, (${pt2.vectorDirect})null)); + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, cov(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, cov(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, cov(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + + <#if pt.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, cov(new ${pt.primitive}[]{1, 2, ${pt.boxed}.NaN}, new ${pt2.primitive}[]{1, 2, 3})); + + <#if pt2.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, cov(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{1, 2, ${pt2.boxed}.NaN})); + try { cov(new ${pt.vectorDirect}(new ${pt.primitive}[]{1,2,3,${pt.null},5}), new ${pt2.vectorDirect}(new ${pt2.primitive}[]{4,5})); @@ -552,6 +602,20 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(NULL_DOUBLE, cor((${pt.vectorDirect})null, new ${pt2.vectorDirect}(b))); assertEquals(NULL_DOUBLE, cor((${pt.vectorDirect})null, (${pt2.vectorDirect})null)); + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, cor(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, cor(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, cor(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + + <#if pt.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, cor(new ${pt.primitive}[]{1, 2, ${pt.boxed}.NaN}, new ${pt2.primitive}[]{1, 2, 3})); + + <#if pt2.valueType.isFloat > + // verify the NaN short-circuit case + assertEquals(Double.NaN, cor(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{1, 2, ${pt2.boxed}.NaN})); + + try { cor(new ${pt.vectorDirect}(new ${pt.primitive}[]{1,2,3,${pt.null},5}), new ${pt2.vectorDirect}(new ${pt2.primitive}[]{4,5})); fail("Mismatched arguments"); @@ -1092,7 +1156,7 @@ public class TestNumeric extends BaseArrayTestCase { } public void test${pt.boxed}Median() { - assertEquals(Double.NaN, median(new ${pt.primitive}[]{})); + assertEquals(NULL_DOUBLE, median(new ${pt.primitive}[]{})); assertEquals(3.0, median(new ${pt.primitive}[]{4,2,3})); assertEquals(3.5, median(new ${pt.primitive}[]{5,4,2,3})); @@ -1106,6 +1170,23 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(3.5, median(new ${pt.vectorDirect}(new ${pt.primitive}[]{5,4,2,3}))); assertEquals(NULL_DOUBLE, median((${pt.vector}) null)); + // verify the all-null case returns null + assertEquals(NULL_DOUBLE, median(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(NULL_DOUBLE, median(new ${pt.boxed}[]{${pt.null}, ${pt.null}, ${pt.null}})); + + // verify the mixed-null cases + assertEquals(3.0, median(new ${pt.primitive}[]{(${pt.primitive})4,(${pt.primitive})2,(${pt.primitive})3,${pt.null},${pt.null},${pt.null}})); + assertEquals(3.5, median(new ${pt.primitive}[]{(${pt.primitive})4,(${pt.primitive})2,(${pt.primitive})3,(${pt.primitive})5, ${pt.null},${pt.null},${pt.null}})); + + assertEquals(3.0, median(new ${pt.boxed}[]{(${pt.primitive})4,(${pt.primitive})2,(${pt.primitive})3,${pt.null},${pt.null}})); + assertEquals(3.5, median(new ${pt.boxed}[]{(${pt.primitive})4,(${pt.primitive})2,(${pt.primitive})3,(${pt.primitive})5,${pt.null},${pt.null}})); + + <#if pt.valueType.isFloat > + assertEquals(Double.NaN, median(new ${pt.primitive}[]{4,2,3, ${pt.boxed}.NaN})); + assertEquals(3.0, median(new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + assertEquals(3.5, median(new ${pt.primitive}[]{4,2,3,5, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + + // check that functions can be resolved with varargs assertEquals(3.0, median((${pt.primitive})4, (${pt.primitive})2, (${pt.primitive})3)); } @@ -1135,6 +1216,28 @@ public class TestNumeric extends BaseArrayTestCase { // pass } + // verify the all-null case returns null + assertEquals(${pt.null}, percentile(0.00, new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(${pt.null}, percentile(0.25, new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + assertEquals(${pt.null}, percentile(0.50, new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}})); + + // verify the mixed-null cases + assertEquals((${pt.primitive})2, percentile(0.00, new ${pt.primitive}[]{4,2,3,${pt.null}})); + assertEquals((${pt.primitive})3, percentile(0.50, new ${pt.primitive}[]{4,2,3,${pt.null},${pt.null}})); + assertEquals((${pt.primitive})4, percentile(1.0, new ${pt.primitive}[]{4,2,3,${pt.null},${pt.null},${pt.null}})); + + // verify the empty array case + assertEquals(${pt.null}, percentile(0.00, new ${pt.vectorDirect}(new ${pt.primitive}[]{}))); + + <#if pt.valueType.isFloat > + assertEquals(${pt.boxed}.NaN, percentile(1.0, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.NaN})); + + assertEquals(${pt.boxed}.NEGATIVE_INFINITY, percentile(0.0, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + assertEquals((${pt.primitive})2, percentile(0.25, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + assertEquals((${pt.primitive})3, percentile(0.5, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + assertEquals((${pt.primitive})4, percentile(0.75, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + assertEquals(${pt.boxed}.POSITIVE_INFINITY, percentile(1.0, new ${pt.primitive}[]{4,2,3, ${pt.boxed}.POSITIVE_INFINITY, ${pt.null}, ${pt.null}, ${pt.boxed}.NEGATIVE_INFINITY})); + } public void test${pt.boxed}Wsum() { @@ -1286,6 +1389,21 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(var(new ${pt.primitive}[]{1,2,3}), wvar(new ${pt.primitive}[]{1,2,3,${pt.null},5}, new ${pt2.primitive}[]{1,1,1,7,${pt2.null}})); assertEquals(var(new ${pt.primitive}[]{1,2,3}), wvar(new ${pt.primitive}[]{1,2,3,${pt.null},5}, new ${pt2.primitive}[]{2,2,2,7,${pt2.null}})); + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, wvar(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, wvar(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, wvar(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + + // verify size==1 + assertEquals(Double.NaN, wvar(new ${pt.primitive}[]{1}, new ${pt2.primitive}[]{4})); + + <#if pt2.valueType.isFloat > + // verify NaN poisoning + assertEquals(Double.NaN, wvar(new ${pt.vectorDirect}(new ${pt.primitive}[]{1,2,3,${pt.null},5}), new ${pt2.vectorDirect}(new ${pt2.primitive}[]{4,5,6,Float.NaN,${pt2.null}}))); + + + // verify the zero-weight case returns null + assertEquals(Double.NaN, wvar(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{0, 0, 0})); } @@ -1316,6 +1434,10 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(NULL_DOUBLE, wstd((${pt.vector}) null, new ${pt2.vectorDirect}(new ${pt2.primitive}[]{4,5,6}))); assertEquals(NULL_DOUBLE, wstd(new ${pt.vectorDirect}(new ${pt.primitive}[]{1,2,3}), (${pt2.vector}) null)); + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, wstd(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, wstd(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, wstd(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); } @@ -1355,6 +1477,10 @@ public class TestNumeric extends BaseArrayTestCase { // pass } + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, wste(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, wste(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, wste(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); @@ -1382,6 +1508,11 @@ public class TestNumeric extends BaseArrayTestCase { assertEquals(NULL_DOUBLE, wtstat((${pt.vector}) null, new ${pt2.vectorDirect}(new ${pt2.primitive}[]{4,5,6}))); assertEquals(NULL_DOUBLE, wtstat(new ${pt.vectorDirect}(new ${pt.primitive}[]{1,2,3}), (${pt2.vector}) null)); + // verify the all-null cases return null + assertEquals(NULL_DOUBLE, wtstat(new ${pt.primitive}[]{1, 2, 3}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + assertEquals(NULL_DOUBLE, wtstat(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{1, 2, 3})); + assertEquals(NULL_DOUBLE, wtstat(new ${pt.primitive}[]{${pt.null}, ${pt.null}, ${pt.null}}, new ${pt2.primitive}[]{${pt2.null}, ${pt2.null}, ${pt2.null}})); + } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java index ac8a08af83c..34fdf0b5392 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java @@ -24,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative average operator. @@ -92,7 +93,7 @@ private boolean addChunk(ByteChunk values, long destination, i runningSum.set(destination, newSum); resultColumn.set(destination, (double) newSum / newCount); } else if (nonNullCount.onlyNullsUnsafe(destination)) { - resultColumn.set(destination, Double.NaN); + resultColumn.set(destination, NULL_DOUBLE); } else { return false; } @@ -110,8 +111,11 @@ private boolean removeChunk(ByteChunk values, long destination final long newCount = nonNullCount.addNonNullUnsafe(destination, -chunkNonNull.get()); final long newSum = minusLong(runningSum.getUnsafe(destination), chunkSum); runningSum.set(destination, newSum); - resultColumn.set(destination, (double) newSum / newCount); - + if (newCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else { + resultColumn.set(destination, (double) newSum / newCount); + } return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java index bedc7465bb8..a2f8f971ba9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -23,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -86,21 +88,27 @@ private boolean addChunk(ByteChunk values, long destination, i final double sum = SumByteChunk.sum2ByteChunk(values, chunkStart, chunkSize, chunkNonNull, sum2); if (chunkNonNull.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); final double newSum = plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } - } else if (nonNullCounter.getCountUnsafe(destination) <= 1) { - resultColumn.set(destination, Double.NaN); + } else { + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else if (totalNormalCount == 1) { + resultColumn.set(destination, Double.NaN); + } } return true; } @@ -114,12 +122,12 @@ private boolean removeChunk(ByteChunk values, long destination return false; } - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); final double newSum; final double newSum2; - if (nonNullCount == 0) { + if (totalNormalCount == 0) { newSum = newSum2 = 0; } else { newSum = plusDouble(sumSource.getUnsafe(destination), -sum); @@ -129,12 +137,16 @@ private boolean removeChunk(ByteChunk values, long destination sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; } - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java index 130c588f20e..b5c2c12aec7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java @@ -20,6 +20,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative average operator. @@ -88,7 +89,7 @@ private boolean addChunk(CharChunk values, long destination, i runningSum.set(destination, newSum); resultColumn.set(destination, (double) newSum / newCount); } else if (nonNullCount.onlyNullsUnsafe(destination)) { - resultColumn.set(destination, Double.NaN); + resultColumn.set(destination, NULL_DOUBLE); } else { return false; } @@ -106,8 +107,11 @@ private boolean removeChunk(CharChunk values, long destination final long newCount = nonNullCount.addNonNullUnsafe(destination, -chunkNonNull.get()); final long newSum = minusLong(runningSum.getUnsafe(destination), chunkSum); runningSum.set(destination, newSum); - resultColumn.set(destination, (double) newSum / newCount); - + if (newCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else { + resultColumn.set(destination, (double) newSum / newCount); + } return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java index bb653cce70e..51599f8a9fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -19,6 +20,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -82,21 +84,27 @@ private boolean addChunk(CharChunk values, long destination, i final double sum = SumCharChunk.sum2CharChunk(values, chunkStart, chunkSize, chunkNonNull, sum2); if (chunkNonNull.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); final double newSum = plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } - } else if (nonNullCounter.getCountUnsafe(destination) <= 1) { - resultColumn.set(destination, Double.NaN); + } else { + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else if (totalNormalCount == 1) { + resultColumn.set(destination, Double.NaN); + } } return true; } @@ -110,12 +118,12 @@ private boolean removeChunk(CharChunk values, long destination return false; } - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); final double newSum; final double newSum2; - if (nonNullCount == 0) { + if (totalNormalCount == 0) { newSum = newSum2 = 0; } else { newSum = plusDouble(sumSource.getUnsafe(destination), -sum); @@ -125,12 +133,16 @@ private boolean removeChunk(CharChunk values, long destination sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; } - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java index 15666e608e0..ed0313ada8a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java @@ -22,6 +22,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; class DoubleChunkedAvgOperator extends FpChunkedNonNormalCounter implements IterativeChunkedAggregationOperator { private final String name; @@ -141,6 +142,8 @@ private void updateResultWithNewSum(long destination, long totalNormal, long tot resultColumn.set(destination, Double.POSITIVE_INFINITY); } else if (totalNegativeInfinityCount > 0) { resultColumn.set(destination, Double.NEGATIVE_INFINITY); + } else if (totalNormal == 0) { + resultColumn.set(destination, NULL_DOUBLE); } else { resultColumn.set(destination, newSum / totalNormal); } @@ -148,12 +151,14 @@ private void updateResultWithNewSum(long destination, long totalNormal, long tot private void updateResultSumUnchanged(long destination, long totalNormal, long totalNanCount, long totalInfinityCount, long totalNegativeInfinityCount) { - if (totalNanCount > 0 || totalNormal == 0 || (totalInfinityCount > 0 && totalNegativeInfinityCount > 0)) { + if (totalNanCount > 0 || (totalInfinityCount > 0 && totalNegativeInfinityCount > 0)) { resultColumn.set(destination, Double.NaN); } else if (totalInfinityCount > 0) { resultColumn.set(destination, Double.POSITIVE_INFINITY); } else if (totalNegativeInfinityCount > 0) { resultColumn.set(destination, Double.NEGATIVE_INFINITY); + } else if (totalNormal == 0) { + resultColumn.set(destination, NULL_DOUBLE); } else { resultColumn.set(destination, runningSum.getUnsafe(destination) / totalNormal); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedReAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedReAvgOperator.java index 4bdffdf6183..a8610d282b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedReAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedReAvgOperator.java @@ -20,6 +20,8 @@ import java.util.Collections; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; + /** * Iterative average operator. */ @@ -127,12 +129,14 @@ private boolean updateResult(long destination) { private boolean updateResult(long destination, long nncValue, long nanValue, long picValue, long nicValue, double sumSumValue) { - if (nanValue > 0 || (picValue > 0 && nicValue > 0) || nncValue == 0) { + if (nanValue > 0 || (picValue > 0 && nicValue > 0)) { return !Double.isNaN(resultColumn.getAndSetUnsafe(destination, Double.NaN)); } else if (picValue > 0) { return resultColumn.getAndSetUnsafe(destination, Double.POSITIVE_INFINITY) != Double.POSITIVE_INFINITY; } else if (nicValue > 0) { return resultColumn.getAndSetUnsafe(destination, Double.NEGATIVE_INFINITY) != Double.NEGATIVE_INFINITY; + } else if (nncValue == 0) { + return resultColumn.getAndSetUnsafe(destination, NULL_DOUBLE) != NULL_DOUBLE; } else { final double newValue = (double) (sumSumValue / nncValue); return resultColumn.getAndSetUnsafe(destination, newValue) != newValue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java index db5991d75e4..42916045fc0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -23,6 +24,7 @@ import java.util.Map; import static io.deephaven.engine.table.impl.by.RollupConstants.*; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -95,14 +97,15 @@ private boolean addChunk(DoubleChunk values, long destination, final boolean forceNanResult = totalNegativeInfinities > 0 || totalPositiveInfinities > 0 || totalNanCount > 0; if (chunkNormalCount.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNormalCount.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNormalCount.get()); final double newSum = NullSafeAddition.plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = NullSafeAddition.plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (forceNanResult || nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (forceNanResult || totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { // If the sum or sumSquared has reached +/-Infinity, we are stuck with NaN forever. @@ -110,16 +113,24 @@ private boolean addChunk(DoubleChunk values, long destination, resultColumn.set(destination, Double.NaN); return true; } - final double variance = computeVariance(nonNullCount, newSum, newSum2); + final double variance = computeVariance(totalNormalCount, newSum, newSum2); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } return true; - } else if (forceNanResult || (nonNullCounter.getCountUnsafe(destination) <= 1)) { + } + if (forceNanResult) { + resultColumn.set(destination, Double.NaN); + return true; + } + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } else if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; - } else { - return false; } + return false; } private static double computeVariance(long nonNullCount, double newSum, double newSum2) { @@ -165,16 +176,17 @@ private boolean removeChunk(DoubleChunk values, long destinati } sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - } else if (totalNormalCount <= 1 || forceNanResult) { - resultColumn.set(destination, Double.NaN); - return true; } else { newSum = sumSource.getUnsafe(destination); newSum2 = sum2Source.getUnsafe(destination); } - if (totalNormalCount <= 1) { + + if (totalNormalCount == 1 || forceNanResult) { resultColumn.set(destination, Double.NaN); return true; + } else if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; } // If the sum has reach +/-Infinity, we are stuck with NaN forever. @@ -186,6 +198,7 @@ private boolean removeChunk(DoubleChunk values, long destinati // Perform the calculation in a way that minimizes the impact of FP error. final double variance = computeVariance(totalNormalCount, newSum, newSum2); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); + return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java index f227389aa6b..e3cf530f7d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java @@ -18,6 +18,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; class FloatChunkedAvgOperator extends FpChunkedNonNormalCounter implements IterativeChunkedAggregationOperator { private final String name; @@ -137,6 +138,8 @@ private void updateResultWithNewSum(long destination, long totalNormal, long tot resultColumn.set(destination, Double.POSITIVE_INFINITY); } else if (totalNegativeInfinityCount > 0) { resultColumn.set(destination, Double.NEGATIVE_INFINITY); + } else if (totalNormal == 0) { + resultColumn.set(destination, NULL_DOUBLE); } else { resultColumn.set(destination, newSum / totalNormal); } @@ -144,12 +147,14 @@ private void updateResultWithNewSum(long destination, long totalNormal, long tot private void updateResultSumUnchanged(long destination, long totalNormal, long totalNanCount, long totalInfinityCount, long totalNegativeInfinityCount) { - if (totalNanCount > 0 || totalNormal == 0 || (totalInfinityCount > 0 && totalNegativeInfinityCount > 0)) { + if (totalNanCount > 0 || (totalInfinityCount > 0 && totalNegativeInfinityCount > 0)) { resultColumn.set(destination, Double.NaN); } else if (totalInfinityCount > 0) { resultColumn.set(destination, Double.POSITIVE_INFINITY); } else if (totalNegativeInfinityCount > 0) { resultColumn.set(destination, Double.NEGATIVE_INFINITY); + } else if (totalNormal == 0) { + resultColumn.set(destination, NULL_DOUBLE); } else { resultColumn.set(destination, runningSum.getUnsafe(destination) / totalNormal); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReAvgOperator.java index 7c42869077f..4d3258a8d4f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReAvgOperator.java @@ -16,6 +16,8 @@ import java.util.Collections; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; + /** * Iterative average operator. */ @@ -123,12 +125,14 @@ private boolean updateResult(long destination) { private boolean updateResult(long destination, long nncValue, long nanValue, long picValue, long nicValue, double sumSumValue) { - if (nanValue > 0 || (picValue > 0 && nicValue > 0) || nncValue == 0) { + if (nanValue > 0 || (picValue > 0 && nicValue > 0)) { return !Float.isNaN(resultColumn.getAndSetUnsafe(destination, Float.NaN)); } else if (picValue > 0) { return resultColumn.getAndSetUnsafe(destination, Float.POSITIVE_INFINITY) != Float.POSITIVE_INFINITY; } else if (nicValue > 0) { return resultColumn.getAndSetUnsafe(destination, Float.NEGATIVE_INFINITY) != Float.NEGATIVE_INFINITY; + } else if (nncValue == 0) { + return resultColumn.getAndSetUnsafe(destination, NULL_FLOAT) != NULL_FLOAT; } else { final float newValue = (float) (sumSumValue / nncValue); return resultColumn.getAndSetUnsafe(destination, newValue) != newValue; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReVarOperator.java index 12df11fca34..7ce77d6c68f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedReVarOperator.java @@ -16,6 +16,8 @@ import java.util.Collections; import java.util.Map; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; + /** * Iterative average operator. */ @@ -91,7 +93,7 @@ private void updateResult(ReVarContext reVarContext, RowSequence destinationOk, nicSum.getChunk(reVarContext.nicContext, destinationOk).asLongChunk(); final int size = reVarContext.keyIndices.size(); - final boolean ordered = reVarContext.ordered;; + final boolean ordered = reVarContext.ordered; for (int ii = 0; ii < size; ++ii) { final boolean changed = updateResult(reVarContext.keyIndices.get(ii), nncSumChunk.get(ii), nanSumChunk.get(ii), @@ -131,8 +133,10 @@ private boolean updateResult(long destination) { private boolean updateResult(long destination, long nncValue, long nanValue, long picValue, long nicValue, double newSum, double newSum2) { - if (nanValue > 0 || picValue > 0 || nicValue > 0 || nncValue <= 1) { + if (nanValue > 0 || picValue > 0 || nicValue > 0 || nncValue == 1) { return !Double.isNaN(resultColumn.getAndSetUnsafe(destination, Double.NaN)); + } else if (nncValue == 0) { + return resultColumn.getAndSetUnsafe(destination, NULL_DOUBLE) != NULL_DOUBLE; } else { final double variance = (newSum2 - newSum * newSum / nncValue) / (nncValue - 1); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java index e3881adf0f2..3d843d604f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -19,6 +20,7 @@ import java.util.Map; import static io.deephaven.engine.table.impl.by.RollupConstants.*; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -91,14 +93,15 @@ private boolean addChunk(FloatChunk values, long destination, final boolean forceNanResult = totalNegativeInfinities > 0 || totalPositiveInfinities > 0 || totalNanCount > 0; if (chunkNormalCount.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNormalCount.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNormalCount.get()); final double newSum = NullSafeAddition.plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = NullSafeAddition.plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (forceNanResult || nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (forceNanResult || totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { // If the sum or sumSquared has reached +/-Infinity, we are stuck with NaN forever. @@ -106,16 +109,24 @@ private boolean addChunk(FloatChunk values, long destination, resultColumn.set(destination, Double.NaN); return true; } - final double variance = computeVariance(nonNullCount, newSum, newSum2); + final double variance = computeVariance(totalNormalCount, newSum, newSum2); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } return true; - } else if (forceNanResult || (nonNullCounter.getCountUnsafe(destination) <= 1)) { + } + if (forceNanResult) { + resultColumn.set(destination, Double.NaN); + return true; + } + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } else if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; - } else { - return false; } + return false; } private static double computeVariance(long nonNullCount, double newSum, double newSum2) { @@ -161,16 +172,17 @@ private boolean removeChunk(FloatChunk values, long destinatio } sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - } else if (totalNormalCount <= 1 || forceNanResult) { - resultColumn.set(destination, Double.NaN); - return true; } else { newSum = sumSource.getUnsafe(destination); newSum2 = sum2Source.getUnsafe(destination); } - if (totalNormalCount <= 1) { + + if (totalNormalCount == 1 || forceNanResult) { resultColumn.set(destination, Double.NaN); return true; + } else if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; } // If the sum has reach +/-Infinity, we are stuck with NaN forever. @@ -182,6 +194,7 @@ private boolean removeChunk(FloatChunk values, long destinatio // Perform the calculation in a way that minimizes the impact of FP error. final double variance = computeVariance(totalNormalCount, newSum, newSum2); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); + return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java index 81e66b75ff8..1adec564276 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java @@ -24,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative average operator. @@ -92,7 +93,7 @@ private boolean addChunk(IntChunk values, long destination, in runningSum.set(destination, newSum); resultColumn.set(destination, (double) newSum / newCount); } else if (nonNullCount.onlyNullsUnsafe(destination)) { - resultColumn.set(destination, Double.NaN); + resultColumn.set(destination, NULL_DOUBLE); } else { return false; } @@ -110,8 +111,11 @@ private boolean removeChunk(IntChunk values, long destination, final long newCount = nonNullCount.addNonNullUnsafe(destination, -chunkNonNull.get()); final long newSum = minusLong(runningSum.getUnsafe(destination), chunkSum); runningSum.set(destination, newSum); - resultColumn.set(destination, (double) newSum / newCount); - + if (newCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else { + resultColumn.set(destination, (double) newSum / newCount); + } return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java index 882dc975285..482c9af19d5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -23,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -86,21 +88,27 @@ private boolean addChunk(IntChunk values, long destination, in final double sum = SumIntChunk.sum2IntChunk(values, chunkStart, chunkSize, chunkNonNull, sum2); if (chunkNonNull.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); final double newSum = plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } - } else if (nonNullCounter.getCountUnsafe(destination) <= 1) { - resultColumn.set(destination, Double.NaN); + } else { + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else if (totalNormalCount == 1) { + resultColumn.set(destination, Double.NaN); + } } return true; } @@ -114,12 +122,12 @@ private boolean removeChunk(IntChunk values, long destination, return false; } - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); final double newSum; final double newSum2; - if (nonNullCount == 0) { + if (totalNormalCount == 0) { newSum = newSum2 = 0; } else { newSum = plusDouble(sumSource.getUnsafe(destination), -sum); @@ -129,12 +137,16 @@ private boolean removeChunk(IntChunk values, long destination, sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; } - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java index 8c58a17b6d7..ce71da4691e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java @@ -24,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative average operator. @@ -92,7 +93,7 @@ private boolean addChunk(LongChunk values, long destination, i runningSum.set(destination, newSum); resultColumn.set(destination, (double) newSum / newCount); } else if (nonNullCount.onlyNullsUnsafe(destination)) { - resultColumn.set(destination, Double.NaN); + resultColumn.set(destination, NULL_DOUBLE); } else { return false; } @@ -110,8 +111,11 @@ private boolean removeChunk(LongChunk values, long destination final long newCount = nonNullCount.addNonNullUnsafe(destination, -chunkNonNull.get()); final long newSum = minusLong(runningSum.getUnsafe(destination), chunkSum); runningSum.set(destination, newSum); - resultColumn.set(destination, (double) newSum / newCount); - + if (newCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else { + resultColumn.set(destination, (double) newSum / newCount); + } return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java index 2b7b9a14bfa..d5d4929a7ab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -23,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -86,21 +88,27 @@ private boolean addChunk(LongChunk values, long destination, i final double sum = SumLongChunk.sum2LongChunk(values, chunkStart, chunkSize, chunkNonNull, sum2); if (chunkNonNull.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); final double newSum = plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } - } else if (nonNullCounter.getCountUnsafe(destination) <= 1) { - resultColumn.set(destination, Double.NaN); + } else { + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else if (totalNormalCount == 1) { + resultColumn.set(destination, Double.NaN); + } } return true; } @@ -114,12 +122,12 @@ private boolean removeChunk(LongChunk values, long destination return false; } - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); final double newSum; final double newSum2; - if (nonNullCount == 0) { + if (totalNormalCount == 0) { newSum = newSum2 = 0; } else { newSum = plusDouble(sumSource.getUnsafe(destination), -sum); @@ -129,12 +137,16 @@ private boolean removeChunk(LongChunk values, long destination sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; } - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java index 40bb2fd41b5..27c3ebdfbd9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java @@ -24,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative average operator. @@ -92,7 +93,7 @@ private boolean addChunk(ShortChunk values, long destination, runningSum.set(destination, newSum); resultColumn.set(destination, (double) newSum / newCount); } else if (nonNullCount.onlyNullsUnsafe(destination)) { - resultColumn.set(destination, Double.NaN); + resultColumn.set(destination, NULL_DOUBLE); } else { return false; } @@ -110,8 +111,11 @@ private boolean removeChunk(ShortChunk values, long destinatio final long newCount = nonNullCount.addNonNullUnsafe(destination, -chunkNonNull.get()); final long newSum = minusLong(runningSum.getUnsafe(destination), chunkSum); runningSum.set(destination, newSum); - resultColumn.set(destination, (double) newSum / newCount); - + if (newCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else { + resultColumn.set(destination, (double) newSum / newCount); + } return true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java index 6d24d6f75d6..b8a77cfefc7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.by; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -23,6 +24,7 @@ import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; /** * Iterative variance operator. @@ -86,21 +88,27 @@ private boolean addChunk(ShortChunk values, long destination, final double sum = SumShortChunk.sum2ShortChunk(values, chunkStart, chunkSize, chunkNonNull, sum2); if (chunkNonNull.get() > 0) { - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, chunkNonNull.get()); final double newSum = plusDouble(sumSource.getUnsafe(destination), sum); final double newSum2 = plusDouble(sum2Source.getUnsafe(destination), sum2.doubleValue()); sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + Assert.neqZero(totalNormalCount, "totalNormalCount"); + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); } else { - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); } - } else if (nonNullCounter.getCountUnsafe(destination) <= 1) { - resultColumn.set(destination, Double.NaN); + } else { + final long totalNormalCount = nonNullCounter.getCountUnsafe(destination); + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + } else if (totalNormalCount == 1) { + resultColumn.set(destination, Double.NaN); + } } return true; } @@ -114,12 +122,12 @@ private boolean removeChunk(ShortChunk values, long destinatio return false; } - final long nonNullCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); + final long totalNormalCount = nonNullCounter.addNonNullUnsafe(destination, -chunkNonNull.get()); final double newSum; final double newSum2; - if (nonNullCount == 0) { + if (totalNormalCount == 0) { newSum = newSum2 = 0; } else { newSum = plusDouble(sumSource.getUnsafe(destination), -sum); @@ -129,12 +137,16 @@ private boolean removeChunk(ShortChunk values, long destinatio sumSource.set(destination, newSum); sum2Source.set(destination, newSum2); - if (nonNullCount <= 1) { + if (totalNormalCount == 0) { + resultColumn.set(destination, NULL_DOUBLE); + return true; + } + if (totalNormalCount == 1) { resultColumn.set(destination, Double.NaN); return true; } - final double variance = (newSum2 - (newSum * newSum / nonNullCount)) / (nonNullCount - 1); + final double variance = (newSum2 - (newSum * newSum / totalNormalCount)) / (totalNormalCount - 1); resultColumn.set(destination, std ? Math.sqrt(variance) : variance); return true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ByteRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ByteRollingAvgOperator.java index adc9ca532d4..8a3ec4ccd8f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ByteRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ByteRollingAvgOperator.java @@ -87,12 +87,12 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (byteWindowValues.size() == 0) { + if (byteWindowValues.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { final int count = byteWindowValues.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, curVal / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/CharRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/CharRollingAvgOperator.java index e5620d15cc2..97dc280c981 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/CharRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/CharRollingAvgOperator.java @@ -82,12 +82,12 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (charWindowValues.size() == 0) { + if (charWindowValues.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { final int count = charWindowValues.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, curVal / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/DoubleRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/DoubleRollingAvgOperator.java index 3e53befcd5c..7bc1cf11e02 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/DoubleRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/DoubleRollingAvgOperator.java @@ -87,7 +87,7 @@ public void writeToOutputChunk(int outIdx) { } else { final int count = aggSum.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, aggSum.evaluate() / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/FloatRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/FloatRollingAvgOperator.java index 4f4eaa67053..0dcb3855312 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/FloatRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/FloatRollingAvgOperator.java @@ -83,7 +83,7 @@ public void writeToOutputChunk(int outIdx) { } else { final int count = aggSum.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, aggSum.evaluate() / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/IntRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/IntRollingAvgOperator.java index fa73349b73f..2f6c5192691 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/IntRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/IntRollingAvgOperator.java @@ -86,12 +86,12 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (intWindowValues.size() == 0) { + if (intWindowValues.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { final int count = intWindowValues.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, curVal / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/LongRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/LongRollingAvgOperator.java index 72c6e7b639f..81541eae9c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/LongRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/LongRollingAvgOperator.java @@ -86,12 +86,12 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (longWindowValues.size() == 0) { + if (longWindowValues.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { final int count = longWindowValues.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, curVal / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ShortRollingAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ShortRollingAvgOperator.java index 23512f1ab4d..f256a88c3a8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ShortRollingAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingavg/ShortRollingAvgOperator.java @@ -86,12 +86,12 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (shortWindowValues.size() == 0) { + if (shortWindowValues.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { final int count = shortWindowValues.size() - nullCount; if (count == 0) { - outputValues.set(outIdx, Double.NaN); + outputValues.set(outIdx, NULL_DOUBLE); } else { outputValues.set(outIdx, curVal / (double) count); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ByteRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ByteRollingStdOperator.java index 15c40b34176..a87a630b841 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ByteRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ByteRollingStdOperator.java @@ -110,9 +110,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/CharRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/CharRollingStdOperator.java index f622450770f..5ab4fb484de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/CharRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/CharRollingStdOperator.java @@ -105,9 +105,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/DoubleRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/DoubleRollingStdOperator.java index 1bd631bfd62..2a4b52b0493 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/DoubleRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/DoubleRollingStdOperator.java @@ -109,9 +109,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/FloatRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/FloatRollingStdOperator.java index f15227cd488..0653990e4e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/FloatRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/FloatRollingStdOperator.java @@ -109,9 +109,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/IntRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/IntRollingStdOperator.java index 2c71b961993..5ffe50e2657 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/IntRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/IntRollingStdOperator.java @@ -109,9 +109,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/LongRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/LongRollingStdOperator.java index b4706c05d75..ba6527c6682 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/LongRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/LongRollingStdOperator.java @@ -109,9 +109,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ShortRollingStdOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ShortRollingStdOperator.java index 193216665f4..4ef7f286514 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ShortRollingStdOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingstd/ShortRollingStdOperator.java @@ -109,9 +109,14 @@ public void pop(int count) { @Override public void writeToOutputChunk(int outIdx) { - if (valueBuffer.size() == 0) { + if (valueBuffer.isEmpty()) { outputValues.set(outIdx, NULL_DOUBLE); } else { + if (nullCount == valueBuffer.size()) { + outputValues.set(outIdx, NULL_DOUBLE); + return; + } + final int count = valueBuffer.size() - nullCount; if (count <= 1) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index f573f40547e..138cc91b59c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -1646,9 +1646,7 @@ private String avgExpr(String c) { return c + "=" + c + "_Count == 0 ? null : " + c + "_Sum.divide(java.math.BigDecimal.valueOf(" + c + "_Count), java.math.BigDecimal.ROUND_HALF_UP)"; } - // I would expect us to return a null for an average of nothing, but we instead return a NaN - // return c + "=" + c + "_Count == 0 ? null : ((double)" + c + "_Sum / (double)" + c + "_Count)"; - return c + "=((double)(" + c + "_Count == 0 ? 0.0 : " + c + "_Sum) / (double)" + c + "_Count)"; + return c + "=" + c + "_Count == 0 ? null : ((double)" + c + "_Sum / (double)" + c + "_Count)"; } @Test @@ -1886,9 +1884,9 @@ public void testAvgInfinities() { TableTools.show(result.meta()); TestCase.assertEquals(1, result.size()); double avg = result.getColumnSource("IntCol", double.class).getDouble(result.getRowSet().firstRowKey()); - TestCase.assertEquals(Double.NaN, avg); + TestCase.assertEquals(NULL_DOUBLE, avg); double avgF = result.getColumnSource("FloatCol", double.class).getDouble(result.getRowSet().firstRowKey()); - TestCase.assertEquals(Double.NaN, avgF); + TestCase.assertEquals(NULL_DOUBLE, avgF); final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); updateGraph.runWithinUnitTestCycle(() -> { @@ -1965,9 +1963,9 @@ public void testVarInfinities() { TableTools.show(result.meta()); TestCase.assertEquals(1, result.size()); double var = result.getColumnSource("IntCol", double.class).getDouble(result.getRowSet().firstRowKey()); - TestCase.assertEquals(Double.NaN, var); + TestCase.assertEquals(NULL_DOUBLE, var); double varF = result.getColumnSource("FloatCol", double.class).getDouble(result.getRowSet().firstRowKey()); - TestCase.assertEquals(Double.NaN, varF); + TestCase.assertEquals(NULL_DOUBLE, varF); final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); updateGraph.runWithinUnitTestCycle(() -> { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTestFormulaStaticMethods.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTestFormulaStaticMethods.java index e25376a5579..c0633350819 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTestFormulaStaticMethods.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTestFormulaStaticMethods.java @@ -14,6 +14,8 @@ import java.math.BigDecimal; import java.math.BigInteger; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; + public class QueryTableAggregationTestFormulaStaticMethods { public static ByteVector abs(ByteVector values) { final byte[] result = new byte[values.intSize()]; @@ -357,6 +359,7 @@ public static double varChar(CharVector values) { double sum = 0; double sum2 = 0; int count = 0; + int nullCount = 0; for (int ii = 0; ii < values.size(); ++ii) { final char c = values.get(ii); @@ -364,12 +367,26 @@ public static double varChar(CharVector values) { sum += c; sum2 += c * c; count++; + } else { + nullCount++; } } + if (nullCount == values.size()) { + return NULL_DOUBLE; + } return (sum2 - sum * sum / count) / (count - 1); } + public static double stdChar(CharVector values) { + if (values == null) { + return NULL_DOUBLE; + } + + final double v = varChar(values); + return v == NULL_DOUBLE ? NULL_DOUBLE : Math.sqrt(v); + } + public static BigDecimal varBigInt(ObjectVector values) { BigInteger sum = BigInteger.ZERO; BigInteger sum2 = BigInteger.ZERO; @@ -614,11 +631,13 @@ static String varFunction(String col) { static String stdFunction(String col) { switch (col) { + case "charCol": + return QueryTableAggregationTestFormulaStaticMethods.class.getCanonicalName() + ".stdChar(" + col + ")"; case "bigI": case "bigD": return "io.deephaven.util.BigDecimalUtils.sqrt(" + varFunction(col) + ", 10)"; default: - return "Math.sqrt(" + varFunction(col) + ")"; + return "std(" + col + ")"; } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingAvg.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingAvg.java index 05b93595eab..3c9fdfec4d8 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingAvg.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingAvg.java @@ -78,8 +78,7 @@ public class TestRollingAvg extends BaseUpdateByTest { private String[] getFormulas(String[] columns) { return Arrays.stream(columns) - // Force null instead of NaN when vector size == 0 - .map(c -> String.format("%s=%s.size() == 0 ? null : avg(%s)", c, c, c)) + .map(c -> String.format("%s=avg(%s)", c, c)) .toArray(String[]::new); } @@ -290,6 +289,14 @@ private void doTestStaticBucketedTimedBigNumbers(final QueryTable t, final Durat // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCount.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCount.java index d80c33edeb9..8fe83dfb661 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCount.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCount.java @@ -74,8 +74,7 @@ public class TestRollingCount extends BaseUpdateByTest { private String[] getFormulas(String[] columns) { return Arrays.stream(columns) - // Force null instead of NaN when vector size == 0 - .map(c -> String.format("%s=count(%s)", c, c, c)) + .map(c -> String.format("%s=count(%s)", c, c)) .toArray(String[]::new); } @@ -235,6 +234,13 @@ private void doTestStaticBucketedTimedBigNumbers(final QueryTable t, final Durat // endregion Object Helper functions // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } @Test public void testStaticZeroKeyRev() { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingFormula.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingFormula.java index 685f19dde43..7238a47c1d0 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingFormula.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingFormula.java @@ -111,6 +111,14 @@ public static BigInteger sumBigInteger(ObjectVector bigIntegerObject // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; @@ -263,12 +271,7 @@ private void doTestStaticZeroKey(final int prevTicks, final int postTicks) { //////////////////////////////////////////////////////////////////////////////////////////////////// actual = t.updateBy(UpdateByOperation.RollingFormula(prevTicks, postTicks, "avg(x)", "x", primitiveColumns)); - - // avg return Double.NaN when the window is empty, so we should adjust our comparison table. - updateStrings = Arrays.stream(primitiveColumns).map(c -> c + "=isNull(" + c + ") ? Double.NaN : " + c) - .toArray(String[]::new); - expected = t.updateBy(UpdateByOperation.RollingAvg(prevTicks, postTicks, primitiveColumns)) - .update(updateStrings); + expected = t.updateBy(UpdateByOperation.RollingAvg(prevTicks, postTicks, primitiveColumns)); TstUtils.assertTableEquals(expected, actual, TableDiff.DiffItems.DoublesExact); @@ -386,12 +389,7 @@ private void doTestStaticZeroKeyTimed(final Duration prevTime, final Duration po actual = t .updateBy(UpdateByOperation.RollingFormula("ts", prevTime, postTime, "avg(x)", "x", primitiveColumns)); - - // avg return Double.NaN when the window is empty, so we should adjust our comparison table. - updateStrings = Arrays.stream(primitiveColumns).map(c -> c + "=isNull(" + c + ") ? Double.NaN : " + c) - .toArray(String[]::new); - expected = t.updateBy(UpdateByOperation.RollingAvg("ts", prevTime, postTime, primitiveColumns)) - .update(updateStrings); + expected = t.updateBy(UpdateByOperation.RollingAvg("ts", prevTime, postTime, primitiveColumns)); TstUtils.assertTableEquals(expected, actual, TableDiff.DiffItems.DoublesExact); @@ -601,12 +599,7 @@ private void doTestStaticBucketed(boolean grouped, int prevTicks, int postTicks) actual = t.updateBy(UpdateByOperation.RollingFormula(prevTicks, postTicks, "avg(x)", "x", primitiveColumns), "Sym"); - - // avg return Double.NaN when the window is empty, so we should adjust our comparison table. - updateStrings = Arrays.stream(primitiveColumns).map(c -> c + "=isNull(" + c + ") ? Double.NaN : " + c) - .toArray(String[]::new); - expected = t.updateBy(UpdateByOperation.RollingAvg(prevTicks, postTicks, primitiveColumns), "Sym") - .update(updateStrings); + expected = t.updateBy(UpdateByOperation.RollingAvg(prevTicks, postTicks, primitiveColumns), "Sym"); TstUtils.assertTableEquals(expected, actual, TableDiff.DiffItems.DoublesExact); @@ -726,12 +719,7 @@ private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Durat actual = t.updateBy(UpdateByOperation.RollingFormula("ts", prevTime, postTime, "avg(x)", "x", primitiveColumns), "Sym"); - - // avg return Double.NaN when the window is empty, so we should adjust our comparison table. - updateStrings = Arrays.stream(primitiveColumns).map(c -> c + "=isNull(" + c + ") ? Double.NaN : " + c) - .toArray(String[]::new); - expected = t.updateBy(UpdateByOperation.RollingAvg("ts", prevTime, postTime, primitiveColumns), "Sym") - .update(updateStrings); + expected = t.updateBy(UpdateByOperation.RollingAvg("ts", prevTime, postTime, primitiveColumns), "Sym"); TstUtils.assertTableEquals(expected, actual, TableDiff.DiffItems.DoublesExact); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingGroup.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingGroup.java index e9faa243e10..2c6befbb805 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingGroup.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingGroup.java @@ -64,6 +64,14 @@ public class TestRollingGroup extends BaseUpdateByTest { // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingMinMax.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingMinMax.java index d5598242124..c8d51cd5394 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingMinMax.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingMinMax.java @@ -473,6 +473,14 @@ private void doTestStaticBucketedTimedBigNumbers(final QueryTable t, final Durat // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java index 609035daa38..8107ce5e716 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java @@ -91,8 +91,7 @@ public EnumSet diffItems() { private String[] getFormulas(String[] columns) { return Arrays.stream(columns) - // Force null instead of NaN when vector size == 0 - .map(c -> String.format("%s=%s.size() == 0 ? null : product(%s)", c, c, c)) + .map(c -> String.format("%s=product(%s)", c, c, c)) .toArray(String[]::new); } @@ -370,6 +369,14 @@ private void doTestStaticBucketedTimedBigNumbers(final QueryTable t, final Durat // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 10; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingStd.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingStd.java index 787eb3d1bda..52613920d6e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingStd.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingStd.java @@ -78,8 +78,7 @@ public class TestRollingStd extends BaseUpdateByTest { private String[] getFormulas(String[] columns) { return Arrays.stream(columns) - // Force null instead of NaN when vector size == 0 - .map(c -> String.format("%s=%s.size() == 0 ? null : std(%s)", c, c, c)) + .map(c -> String.format("%s=std(%s)", c, c)) .toArray(String[]::new); } @@ -316,6 +315,14 @@ private void doTestStaticBucketedTimedBigNumbers(final QueryTable t, final Durat // region Static Zero Key Tests + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStaticZeroKey(prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java index c985d4dd766..66a10af9287 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingSum.java @@ -62,7 +62,7 @@ public EnumSet diffItems() { // region Static Zero Key Tests @Test - public void testStaticZeroKeyWithAllNullWindows() { + public void testStaticZeroKeyAllNullVector() { final QueryTable t = createTestTable(10000, false, false, false, 0x31313131).t; t.setRefreshing(false); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingWAvg.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingWAvg.java index d8d9ffa0cb9..33b93422407 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingWAvg.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingWAvg.java @@ -682,6 +682,14 @@ private void doTestStaticTimed(boolean bucketed, Duration prevTime, Duration pos doTestStaticTimedBigNumbers(t, prevTime, postTime, bucketed, weightCol, "wavgBigIntBigDec", "wavgBigDecBigDec"); } + @Test + public void testStaticZeroKeyAllNullVector() { + final int prevTicks = 1; + final int postTicks = 0; + + doTestStatic(false, prevTicks, postTicks); + } + @Test public void testStaticZeroKeyRev() { final int prevTicks = 100; From b900e46ab3816176f2e1e5a6d20611bba3b7ba69 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Thu, 20 Jun 2024 17:32:11 -0500 Subject: [PATCH 02/18] Optimizations for BigDecimal and BigInteger parquet reads (#5638) --- extensions/parquet/base/build.gradle | 1 + .../base}/BigDecimalParquetBytesCodec.java | 68 ++++++++++++--- .../base}/BigIntegerParquetBytesCodec.java | 6 +- .../parquet/base/PageMaterializer.java | 37 ++++++++ .../BigDecimalFromBytesMaterializer.java | 56 ++++++++++++ .../BigDecimalFromIntMaterializer.java | 55 ++++++++++++ .../BigDecimalFromLongMaterializer.java | 51 +++++++++++ .../materializers/BigIntegerMaterializer.java | 60 +++++++++++++ .../base/materializers/BlobMaterializer.java | 28 +----- .../base/materializers/BoolMaterializer.java | 6 +- .../base/materializers/ByteMaterializer.java | 6 +- .../base/materializers/CharMaterializer.java | 6 +- .../materializers/DoubleMaterializer.java | 6 +- .../base/materializers/FloatMaterializer.java | 6 +- .../InstantFromInt96Materializer.java | 2 +- .../InstantNanosFromMicrosMaterializer.java | 2 +- .../InstantNanosFromMillisMaterializer.java | 2 +- .../base/materializers/IntMaterializer.java | 6 +- .../materializers/LocalDateMaterializer.java | 27 +----- .../LocalDateTimeFromMicrosMaterializer.java | 7 +- .../LocalDateTimeFromMillisMaterializer.java | 7 +- .../LocalDateTimeFromNanosMaterializer.java | 7 +- .../LocalDateTimeMaterializerBase.java | 40 --------- .../LocalTimeFromMicrosMaterializer.java | 6 +- .../LocalTimeFromMillisMaterializer.java | 6 +- .../LocalTimeFromNanosMaterializer.java | 6 +- .../LongFromUnsignedIntMaterializer.java | 2 +- .../base/materializers/LongMaterializer.java | 2 +- .../materializers/LongMaterializerBase.java | 2 +- ...rBase.java => ObjectMaterializerBase.java} | 11 ++- .../base/materializers/ShortMaterializer.java | 6 +- .../materializers/StringMaterializer.java | 32 +------ .../parquet/table/ParquetSchemaReader.java | 9 +- .../io/deephaven/parquet/table/TypeInfos.java | 3 + .../table/location/ParquetColumnLocation.java | 86 +++++++++---------- .../pagestore/topage/ToBigDecimalBase.java | 39 --------- .../topage/ToBigDecimalFromIntPage.java | 42 --------- .../topage/ToBigDecimalFromLongPage.java | 42 --------- .../pagestore/topage/ToBigDecimalPage.java | 64 ++++++++++++++ .../pagestore/topage/ToBigIntegerPage.java | 60 +++++++++++++ .../table/transfer/TransferObject.java | 10 ++- .../BigDecimalParquetBytesCodecTest.java | 3 +- .../table/ParquetTableReadWriteTest.java | 49 ++++++++++- .../ReferenceDecimalLogicalType.parquet | 3 + .../ReferenceDecimalLogicalType2.parquet | 3 + .../ReplicatePageMaterializers.java | 82 ++++++------------ .../replicators/ReplicateToPage.java | 24 ++++-- 47 files changed, 659 insertions(+), 425 deletions(-) rename extensions/parquet/{table/src/main/java/io/deephaven/parquet/table => base/src/main/java/io/deephaven/parquet/base}/BigDecimalParquetBytesCodec.java (61%) rename extensions/parquet/{table/src/main/java/io/deephaven/parquet/table => base/src/main/java/io/deephaven/parquet/base}/BigIntegerParquetBytesCodec.java (96%) create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromBytesMaterializer.java create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromIntMaterializer.java create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromLongMaterializer.java create mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigIntegerMaterializer.java delete mode 100644 extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeMaterializerBase.java rename extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/{LocalTimeMaterializerBase.java => ObjectMaterializerBase.java} (68%) delete mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalBase.java delete mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromIntPage.java delete mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromLongPage.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalPage.java create mode 100644 extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigIntegerPage.java create mode 100644 extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType.parquet create mode 100644 extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType2.parquet diff --git a/extensions/parquet/base/build.gradle b/extensions/parquet/base/build.gradle index db854924faf..6f50b1feeee 100644 --- a/extensions/parquet/base/build.gradle +++ b/extensions/parquet/base/build.gradle @@ -15,6 +15,7 @@ dependencies { implementation project(':Util') implementation project(':engine-time') implementation project(':Configuration') + implementation project(':DataStructures') implementation depCommonsIo compileOnly depAnnotations diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodec.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigDecimalParquetBytesCodec.java similarity index 61% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodec.java rename to extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigDecimalParquetBytesCodec.java index 940d0643c4c..d158d9f3456 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodec.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigDecimalParquetBytesCodec.java @@ -1,10 +1,11 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -package io.deephaven.parquet.table; +package io.deephaven.parquet.base; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.util.codec.ObjectCodec; +import org.apache.parquet.schema.PrimitiveType; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -14,12 +15,63 @@ import java.nio.ByteBuffer; public class BigDecimalParquetBytesCodec implements ObjectCodec { + + private static final int MIN_DECIMAL_INT_PRECISION = 1; + private static final int MAX_DECIMAL_INT_PRECISION = 9; + private static final int MIN_DECIMAL_LONG_PRECISION = 1; + private static final int MAX_DECIMAL_LONG_PRECISION = 18; + private final int precision; private final int scale; private final int encodedSizeInBytes; private final RoundingMode roundingMode; private final byte[] nullBytes; + /** + * Verify that the precision and scale are valid. + * + * @throws IllegalArgumentException if the provided precision and/or scale is invalid + */ + public static void verifyPrecisionAndScale(final int precision, final int scale) { + if (precision <= 0) { + throw new IllegalArgumentException(String.format("precision (=%d) should be > 0", precision)); + } + if (scale < 0) { + throw new IllegalArgumentException(String.format("scale (=%d) should be >= 0", scale)); + } + if (scale > precision) { + throw new IllegalArgumentException( + String.format("scale (=%d) is greater than precision (=%d)", scale, precision)); + } + } + + /** + * Verify that the precision and scale are valid for the given primitive type. + * + * @throws IllegalArgumentException if the provided precision and/or scale is invalid + */ + public static void verifyPrecisionAndScale(final int precision, final int scale, + final PrimitiveType.PrimitiveTypeName primitiveType) { + verifyPrecisionAndScale(precision, scale); + if (primitiveType == PrimitiveType.PrimitiveTypeName.INT32) { + if (precision < MIN_DECIMAL_INT_PRECISION || precision > MAX_DECIMAL_INT_PRECISION) { + throw new IllegalArgumentException( + String.format( + "Column with decimal logical type and INT32 primitive type should have precision in " + + "range [%d, %d], found column with precision %d", + MIN_DECIMAL_INT_PRECISION, MAX_DECIMAL_INT_PRECISION, precision)); + } + } else if (primitiveType == PrimitiveType.PrimitiveTypeName.INT64) { + if (precision < MIN_DECIMAL_LONG_PRECISION || precision > MAX_DECIMAL_LONG_PRECISION) { + throw new IllegalArgumentException( + String.format( + "Column with decimal logical type and INT64 primitive type should have precision in " + + "range [%d, %d], found column with precision %d", + MIN_DECIMAL_LONG_PRECISION, MAX_DECIMAL_LONG_PRECISION, precision)); + } + } + } + /** * * @param precision @@ -31,15 +83,7 @@ public class BigDecimalParquetBytesCodec implements ObjectCodec { */ public BigDecimalParquetBytesCodec(final int precision, final int scale, final int encodedSizeInBytes, final RoundingMode roundingMode) { - if (precision <= 0) { - throw new IllegalArgumentException("precision (=" + precision + ") should be > 0"); - } - if (scale < 0) { - throw new IllegalArgumentException("scale (=" + scale + ") should be >= 0"); - } - if (scale > precision) { - throw new IllegalArgumentException("scale (=" + scale + ") is greater than precision (=" + precision + ")"); - } + verifyPrecisionAndScale(precision, scale); this.precision = precision; this.scale = scale; this.encodedSizeInBytes = encodedSizeInBytes; @@ -58,6 +102,10 @@ public BigDecimalParquetBytesCodec(final int precision, final int scale, final i this(precision, scale, encodedSizeInBytes, RoundingMode.HALF_UP); } + public BigDecimalParquetBytesCodec(final int precision, final int scale) { + this(precision, scale, -1); + } + // Given how parquet encoding works for nulls, the actual value provided for a null is irrelevant. @Override public boolean isNullable() { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigIntegerParquetBytesCodec.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigIntegerParquetBytesCodec.java similarity index 96% rename from extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigIntegerParquetBytesCodec.java rename to extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigIntegerParquetBytesCodec.java index c8cd436e566..0a11a7e1023 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/BigIntegerParquetBytesCodec.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/BigIntegerParquetBytesCodec.java @@ -1,7 +1,7 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -package io.deephaven.parquet.table; +package io.deephaven.parquet.base; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.util.codec.ObjectCodec; @@ -34,6 +34,10 @@ public BigIntegerParquetBytesCodec(final int encodedSizeInBytes) { } } + public BigIntegerParquetBytesCodec() { + this(-1); + } + // Given how parquet encoding works for nulls, the actual value provided for a null is irrelevant. @Override public boolean isNullable() { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PageMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PageMaterializer.java index c4adfe39069..8aca259445b 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PageMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/PageMaterializer.java @@ -8,8 +8,27 @@ import org.apache.parquet.schema.PrimitiveType; import org.jetbrains.annotations.NotNull; +import java.math.BigDecimal; +import java.math.BigInteger; + +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; + public interface PageMaterializer { + /** + * Get the internal type used by Deephaven to represent a Parquet + * {@link LogicalTypeAnnotation.DecimalLogicalTypeAnnotation Decimal} logical type + */ + static Class resolveDecimalLogicalType( + final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { + // This pair of values (precision=1, scale=0) is set at write time as a marker so that we can recover + // the fact that the type is a BigInteger, not a BigDecimal when the fies are read. + if (decimalLogicalType.getPrecision() == 1 && decimalLogicalType.getScale() == 0) { + return BigInteger.class; + } + return BigDecimal.class; + } + static PageMaterializerFactory factoryForType(@NotNull final PrimitiveType primitiveType) { final PrimitiveType.PrimitiveTypeName primitiveTypeName = primitiveType.getPrimitiveTypeName(); final LogicalTypeAnnotation logicalTypeAnnotation = primitiveType.getLogicalTypeAnnotation(); @@ -47,6 +66,10 @@ static PageMaterializerFactory factoryForType(@NotNull final PrimitiveType primi } // isAdjustedToUTC parameter is ignored while reading LocalTime from Parquet files return LocalTimeFromMillisMaterializer.Factory; + } else if (logicalTypeAnnotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalTypeAnnotation; + return new BigDecimalFromIntMaterializer.Factory(decimalLogicalType.getScale()); } return IntMaterializer.Factory; case INT64: @@ -87,6 +110,10 @@ static PageMaterializerFactory factoryForType(@NotNull final PrimitiveType primi default: throw new IllegalArgumentException("Unsupported unit=" + timeLogicalType.getUnit()); } + } else if (logicalTypeAnnotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalTypeAnnotation; + return new BigDecimalFromLongMaterializer.Factory(decimalLogicalType.getScale()); } return LongMaterializer.Factory; case INT96: @@ -102,6 +129,16 @@ static PageMaterializerFactory factoryForType(@NotNull final PrimitiveType primi return StringMaterializer.Factory; } case FIXED_LEN_BYTE_ARRAY: // fall through + if (logicalTypeAnnotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalTypeAnnotation; + final int encodedSizeInBytes = primitiveTypeName == BINARY ? -1 : primitiveType.getTypeLength(); + if (resolveDecimalLogicalType(decimalLogicalType) == BigInteger.class) { + return new BigIntegerMaterializer.Factory(new BigIntegerParquetBytesCodec(encodedSizeInBytes)); + } + return new BigDecimalFromBytesMaterializer.Factory(new BigDecimalParquetBytesCodec( + decimalLogicalType.getPrecision(), decimalLogicalType.getScale(), encodedSizeInBytes)); + } return BlobMaterializer.Factory; default: throw new RuntimeException("Unexpected type name:" + primitiveTypeName); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromBytesMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromBytesMaterializer.java new file mode 100644 index 00000000000..5080ea35dd8 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromBytesMaterializer.java @@ -0,0 +1,56 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base.materializers; + +import io.deephaven.parquet.base.PageMaterializer; +import io.deephaven.parquet.base.PageMaterializerFactory; +import io.deephaven.util.codec.ObjectCodec; +import org.apache.parquet.column.values.ValuesReader; + +import java.math.BigDecimal; + +public class BigDecimalFromBytesMaterializer extends ObjectMaterializerBase implements PageMaterializer { + + public static final class Factory implements PageMaterializerFactory { + + final ObjectCodec codec; + + public Factory(ObjectCodec codec) { + this.codec = codec; + } + + @Override + public PageMaterializer makeMaterializerWithNulls(ValuesReader dataReader, Object nullValue, int numValues) { + return new BigDecimalFromBytesMaterializer(dataReader, (BigDecimal) nullValue, numValues, codec); + } + + @Override + public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int numValues) { + return new BigDecimalFromBytesMaterializer(dataReader, numValues, codec); + } + } + + private final ValuesReader dataReader; + private final ObjectCodec codec; + + private BigDecimalFromBytesMaterializer(ValuesReader dataReader, int numValues, + ObjectCodec codec) { + this(dataReader, null, numValues, codec); + } + + private BigDecimalFromBytesMaterializer(ValuesReader dataReader, BigDecimal nullValue, int numValues, + ObjectCodec codec) { + super(nullValue, new BigDecimal[numValues]); + this.dataReader = dataReader; + this.codec = codec; + } + + @Override + public void fillValues(int startIndex, int endIndex) { + for (int ii = startIndex; ii < endIndex; ii++) { + final byte[] bytes = dataReader.readBytes().getBytes(); + data[ii] = codec.decode(bytes, 0, bytes.length); + } + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromIntMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromIntMaterializer.java new file mode 100644 index 00000000000..333b0abc91e --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromIntMaterializer.java @@ -0,0 +1,55 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit BigDecimalFromLongMaterializer and run "./gradlew replicatePageMaterializers" to regenerate +// +// @formatter:off +package io.deephaven.parquet.base.materializers; + +import io.deephaven.parquet.base.PageMaterializer; +import io.deephaven.parquet.base.PageMaterializerFactory; +import org.apache.parquet.column.values.ValuesReader; + +import java.math.BigDecimal; + +public class BigDecimalFromIntMaterializer extends ObjectMaterializerBase implements PageMaterializer { + + public static final class Factory implements PageMaterializerFactory { + final int scale; + + public Factory(final int scale) { + this.scale = scale; + } + + @Override + public PageMaterializer makeMaterializerWithNulls(ValuesReader dataReader, Object nullValue, int numValues) { + return new BigDecimalFromIntMaterializer(dataReader, (BigDecimal) nullValue, numValues, scale); + } + + @Override + public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int numValues) { + return new BigDecimalFromIntMaterializer(dataReader, numValues, scale); + } + }; + + private final ValuesReader dataReader; + private final int scale; + + private BigDecimalFromIntMaterializer(ValuesReader dataReader, int numValues, int scale) { + this(dataReader, null, numValues, scale); + } + + private BigDecimalFromIntMaterializer(ValuesReader dataReader, BigDecimal nullValue, int numValues, int scale) { + super(nullValue, new BigDecimal[numValues]); + this.dataReader = dataReader; + this.scale = scale; + } + + @Override + public void fillValues(int startIndex, int endIndex) { + for (int ii = startIndex; ii < endIndex; ii++) { + data[ii] = BigDecimal.valueOf(dataReader.readInteger(), scale); + } + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromLongMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromLongMaterializer.java new file mode 100644 index 00000000000..ffbd17ba610 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigDecimalFromLongMaterializer.java @@ -0,0 +1,51 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.base.materializers; + +import io.deephaven.parquet.base.PageMaterializer; +import io.deephaven.parquet.base.PageMaterializerFactory; +import org.apache.parquet.column.values.ValuesReader; + +import java.math.BigDecimal; + +public class BigDecimalFromLongMaterializer extends ObjectMaterializerBase implements PageMaterializer { + + public static final class Factory implements PageMaterializerFactory { + final int scale; + + public Factory(final int scale) { + this.scale = scale; + } + + @Override + public PageMaterializer makeMaterializerWithNulls(ValuesReader dataReader, Object nullValue, int numValues) { + return new BigDecimalFromLongMaterializer(dataReader, (BigDecimal) nullValue, numValues, scale); + } + + @Override + public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int numValues) { + return new BigDecimalFromLongMaterializer(dataReader, numValues, scale); + } + }; + + private final ValuesReader dataReader; + private final int scale; + + private BigDecimalFromLongMaterializer(ValuesReader dataReader, int numValues, int scale) { + this(dataReader, null, numValues, scale); + } + + private BigDecimalFromLongMaterializer(ValuesReader dataReader, BigDecimal nullValue, int numValues, int scale) { + super(nullValue, new BigDecimal[numValues]); + this.dataReader = dataReader; + this.scale = scale; + } + + @Override + public void fillValues(int startIndex, int endIndex) { + for (int ii = startIndex; ii < endIndex; ii++) { + data[ii] = BigDecimal.valueOf(dataReader.readLong(), scale); + } + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigIntegerMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigIntegerMaterializer.java new file mode 100644 index 00000000000..36589a48a41 --- /dev/null +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BigIntegerMaterializer.java @@ -0,0 +1,60 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit BigDecimalFromBytesMaterializer and run "./gradlew replicatePageMaterializers" to regenerate +// +// @formatter:off +package io.deephaven.parquet.base.materializers; + +import io.deephaven.parquet.base.PageMaterializer; +import io.deephaven.parquet.base.PageMaterializerFactory; +import io.deephaven.util.codec.ObjectCodec; +import org.apache.parquet.column.values.ValuesReader; + +import java.math.BigInteger; + +public class BigIntegerMaterializer extends ObjectMaterializerBase implements PageMaterializer { + + public static final class Factory implements PageMaterializerFactory { + + final ObjectCodec codec; + + public Factory(ObjectCodec codec) { + this.codec = codec; + } + + @Override + public PageMaterializer makeMaterializerWithNulls(ValuesReader dataReader, Object nullValue, int numValues) { + return new BigIntegerMaterializer(dataReader, (BigInteger) nullValue, numValues, codec); + } + + @Override + public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int numValues) { + return new BigIntegerMaterializer(dataReader, numValues, codec); + } + } + + private final ValuesReader dataReader; + private final ObjectCodec codec; + + private BigIntegerMaterializer(ValuesReader dataReader, int numValues, + ObjectCodec codec) { + this(dataReader, null, numValues, codec); + } + + private BigIntegerMaterializer(ValuesReader dataReader, BigInteger nullValue, int numValues, + ObjectCodec codec) { + super(nullValue, new BigInteger[numValues]); + this.dataReader = dataReader; + this.codec = codec; + } + + @Override + public void fillValues(int startIndex, int endIndex) { + for (int ii = startIndex; ii < endIndex; ii++) { + final byte[] bytes = dataReader.readBytes().getBytes(); + data[ii] = codec.decode(bytes, 0, bytes.length); + } + } +} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BlobMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BlobMaterializer.java index 127ff034033..f796d94979e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BlobMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BlobMaterializer.java @@ -8,9 +8,7 @@ import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.api.Binary; -import java.util.Arrays; - -public class BlobMaterializer implements PageMaterializer { +public class BlobMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -24,24 +22,15 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; - - final Binary nullValue; - final Binary[] data; + private final ValuesReader dataReader; private BlobMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private BlobMaterializer(ValuesReader dataReader, Binary nullValue, int numValues) { + super(nullValue, new Binary[numValues]); this.dataReader = dataReader; - this.nullValue = nullValue; - this.data = new Binary[numValues]; - } - - @Override - public void fillNulls(int startIndex, int endIndex) { - Arrays.fill(data, startIndex, endIndex, nullValue); } @Override @@ -50,15 +39,4 @@ public void fillValues(int startIndex, int endIndex) { data[ii] = dataReader.readBytes(); } } - - @Override - public Object fillAll() { - fillValues(0, data.length); - return data; - } - - @Override - public Object data() { - return data; - } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BoolMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BoolMaterializer.java index 8852c0d4314..89871fbf277 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BoolMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/BoolMaterializer.java @@ -23,10 +23,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final byte nullValue; - final byte[] data; + private final byte nullValue; + private final byte[] data; private BoolMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, (byte) 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ByteMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ByteMaterializer.java index 181b9bf9848..66477aed7c5 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ByteMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ByteMaterializer.java @@ -27,10 +27,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final byte nullValue; - final byte[] data; + private final byte nullValue; + private final byte[] data; private ByteMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, (byte) 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/CharMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/CharMaterializer.java index 43b80ccbcd2..ee752bdecd8 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/CharMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/CharMaterializer.java @@ -23,10 +23,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final char nullValue; - final char[] data; + private final char nullValue; + private final char[] data; private CharMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, (char) 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/DoubleMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/DoubleMaterializer.java index 8278e0f8695..2e373e20172 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/DoubleMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/DoubleMaterializer.java @@ -27,10 +27,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final double nullValue; - final double[] data; + private final double nullValue; + private final double[] data; private DoubleMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/FloatMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/FloatMaterializer.java index 5b17ffbc316..fff9142984a 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/FloatMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/FloatMaterializer.java @@ -23,10 +23,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final float nullValue; - final float[] data; + private final float nullValue; + private final float[] data; private FloatMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantFromInt96Materializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantFromInt96Materializer.java index 912910f232a..4782303467d 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantFromInt96Materializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantFromInt96Materializer.java @@ -46,7 +46,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num setReferenceTimeZone(referenceTimeZone); } - final ValuesReader dataReader; + private final ValuesReader dataReader; private InstantFromInt96Materializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMicrosMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMicrosMaterializer.java index 5c82c8f4a45..751bf7ab1b0 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMicrosMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMicrosMaterializer.java @@ -22,7 +22,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private InstantNanosFromMicrosMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMillisMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMillisMaterializer.java index 4f94a5dd9f8..9e7907a0f82 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMillisMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/InstantNanosFromMillisMaterializer.java @@ -26,7 +26,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private InstantNanosFromMillisMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/IntMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/IntMaterializer.java index f0a048ca53e..847ea3a3e18 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/IntMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/IntMaterializer.java @@ -27,10 +27,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final int nullValue; - final int[] data; + private final int nullValue; + private final int[] data; private IntMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateMaterializer.java index 18adfd60971..202e54d83cb 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateMaterializer.java @@ -9,9 +9,8 @@ import org.apache.parquet.column.values.ValuesReader; import java.time.LocalDate; -import java.util.Arrays; -public class LocalDateMaterializer implements PageMaterializer { +public class LocalDateMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -25,24 +24,15 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; - - final LocalDate nullValue; - final LocalDate[] data; + private final ValuesReader dataReader; private LocalDateMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private LocalDateMaterializer(ValuesReader dataReader, LocalDate nullValue, int numValues) { + super(nullValue, new LocalDate[numValues]); this.dataReader = dataReader; - this.nullValue = nullValue; - this.data = new LocalDate[numValues]; - } - - @Override - public void fillNulls(int startIndex, int endIndex) { - Arrays.fill(data, startIndex, endIndex, nullValue); } @Override @@ -51,15 +41,4 @@ public void fillValues(int startIndex, int endIndex) { data[ii] = DateTimeUtils.epochDaysAsIntToLocalDate(dataReader.readInteger()); } } - - @Override - public Object fillAll() { - fillValues(0, data.length); - return data; - } - - @Override - public Object data() { - return data; - } } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMicrosMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMicrosMaterializer.java index 212c2df3a81..022586e6a73 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMicrosMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMicrosMaterializer.java @@ -14,7 +14,8 @@ import java.time.LocalDateTime; -public class LocalDateTimeFromMicrosMaterializer extends LocalDateTimeMaterializerBase implements PageMaterializer { +public class LocalDateTimeFromMicrosMaterializer extends ObjectMaterializerBase + implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -28,7 +29,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalDateTimeFromMicrosMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); @@ -36,7 +37,7 @@ private LocalDateTimeFromMicrosMaterializer(ValuesReader dataReader, int numValu private LocalDateTimeFromMicrosMaterializer(ValuesReader dataReader, LocalDateTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalDateTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMillisMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMillisMaterializer.java index 0bf6ad7fb89..7f2f5ec1f92 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMillisMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromMillisMaterializer.java @@ -10,7 +10,8 @@ import java.time.LocalDateTime; -public class LocalDateTimeFromMillisMaterializer extends LocalDateTimeMaterializerBase implements PageMaterializer { +public class LocalDateTimeFromMillisMaterializer extends ObjectMaterializerBase + implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -24,7 +25,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalDateTimeFromMillisMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); @@ -32,7 +33,7 @@ private LocalDateTimeFromMillisMaterializer(ValuesReader dataReader, int numValu private LocalDateTimeFromMillisMaterializer(ValuesReader dataReader, LocalDateTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalDateTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromNanosMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromNanosMaterializer.java index 664887b2aa4..ae3f66bd813 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromNanosMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeFromNanosMaterializer.java @@ -14,7 +14,8 @@ import java.time.LocalDateTime; -public class LocalDateTimeFromNanosMaterializer extends LocalDateTimeMaterializerBase implements PageMaterializer { +public class LocalDateTimeFromNanosMaterializer extends ObjectMaterializerBase + implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -28,7 +29,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalDateTimeFromNanosMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); @@ -36,7 +37,7 @@ private LocalDateTimeFromNanosMaterializer(ValuesReader dataReader, int numValue private LocalDateTimeFromNanosMaterializer(ValuesReader dataReader, LocalDateTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalDateTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeMaterializerBase.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeMaterializerBase.java deleted file mode 100644 index 60118a41f68..00000000000 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalDateTimeMaterializerBase.java +++ /dev/null @@ -1,40 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit LocalTimeMaterializerBase and run "./gradlew replicatePageMaterializers" to regenerate -// -// @formatter:off -package io.deephaven.parquet.base.materializers; - -import io.deephaven.parquet.base.PageMaterializer; - -import java.time.LocalDateTime; -import java.util.Arrays; - -abstract class LocalDateTimeMaterializerBase implements PageMaterializer { - - final LocalDateTime nullValue; - final LocalDateTime[] data; - - LocalDateTimeMaterializerBase(LocalDateTime nullValue, int numValues) { - this.nullValue = nullValue; - this.data = new LocalDateTime[numValues]; - } - - @Override - public final void fillNulls(int startIndex, int endIndex) { - Arrays.fill(data, startIndex, endIndex, nullValue); - } - - @Override - public final Object fillAll() { - fillValues(0, data.length); - return data; - } - - @Override - public final Object data() { - return data; - } -} diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMicrosMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMicrosMaterializer.java index 360fe334723..b276c808e4c 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMicrosMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMicrosMaterializer.java @@ -10,7 +10,7 @@ import java.time.LocalTime; -public class LocalTimeFromMicrosMaterializer extends LocalTimeMaterializerBase implements PageMaterializer { +public class LocalTimeFromMicrosMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -24,14 +24,14 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalTimeFromMicrosMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private LocalTimeFromMicrosMaterializer(ValuesReader dataReader, LocalTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMillisMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMillisMaterializer.java index da3d799aac6..aed1d7e6308 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMillisMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromMillisMaterializer.java @@ -14,7 +14,7 @@ import java.time.LocalTime; -public class LocalTimeFromMillisMaterializer extends LocalTimeMaterializerBase implements PageMaterializer { +public class LocalTimeFromMillisMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -28,14 +28,14 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalTimeFromMillisMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private LocalTimeFromMillisMaterializer(ValuesReader dataReader, LocalTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromNanosMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromNanosMaterializer.java index 81b0c8996b2..75e707442c9 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromNanosMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeFromNanosMaterializer.java @@ -14,7 +14,7 @@ import java.time.LocalTime; -public class LocalTimeFromNanosMaterializer extends LocalTimeMaterializerBase implements PageMaterializer { +public class LocalTimeFromNanosMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -28,14 +28,14 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LocalTimeFromNanosMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private LocalTimeFromNanosMaterializer(ValuesReader dataReader, LocalTime nullValue, int numValues) { - super(nullValue, numValues); + super(nullValue, new LocalTime[numValues]); this.dataReader = dataReader; } diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongFromUnsignedIntMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongFromUnsignedIntMaterializer.java index 6db570f83f6..add4270f164 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongFromUnsignedIntMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongFromUnsignedIntMaterializer.java @@ -21,7 +21,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LongFromUnsignedIntMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializer.java index a8eb22eb0e4..9f57f184ddb 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializer.java @@ -21,7 +21,7 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; private LongMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializerBase.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializerBase.java index 7edfa3b8423..7eaad312ab6 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializerBase.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LongMaterializerBase.java @@ -9,7 +9,7 @@ abstract class LongMaterializerBase implements PageMaterializer { - final long nullValue; + private final long nullValue; final long[] data; LongMaterializerBase(long nullValue, int numValues) { diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeMaterializerBase.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ObjectMaterializerBase.java similarity index 68% rename from extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeMaterializerBase.java rename to extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ObjectMaterializerBase.java index 320fed412b6..7ca9e4600e8 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/LocalTimeMaterializerBase.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ObjectMaterializerBase.java @@ -5,17 +5,16 @@ import io.deephaven.parquet.base.PageMaterializer; -import java.time.LocalTime; import java.util.Arrays; -abstract class LocalTimeMaterializerBase implements PageMaterializer { +abstract class ObjectMaterializerBase implements PageMaterializer { - final LocalTime nullValue; - final LocalTime[] data; + private final TYPE nullValue; + final TYPE[] data; - LocalTimeMaterializerBase(LocalTime nullValue, int numValues) { + ObjectMaterializerBase(TYPE nullValue, TYPE[] data) { this.nullValue = nullValue; - this.data = new LocalTime[numValues]; + this.data = data; } @Override diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ShortMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ShortMaterializer.java index bbfc0613e77..cff03881692 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ShortMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/ShortMaterializer.java @@ -27,10 +27,10 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; + private final ValuesReader dataReader; - final short nullValue; - final short[] data; + private final short nullValue; + private final short[] data; private ShortMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, (short) 0, numValues); diff --git a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/StringMaterializer.java b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/StringMaterializer.java index 759f8ab10d3..b594aeb351e 100644 --- a/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/StringMaterializer.java +++ b/extensions/parquet/base/src/main/java/io/deephaven/parquet/base/materializers/StringMaterializer.java @@ -1,19 +1,13 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit FloatMaterializer and run "./gradlew replicatePageMaterializers" to regenerate -// -// @formatter:off package io.deephaven.parquet.base.materializers; import io.deephaven.parquet.base.PageMaterializer; import io.deephaven.parquet.base.PageMaterializerFactory; import org.apache.parquet.column.values.ValuesReader; -import java.util.Arrays; - -public class StringMaterializer implements PageMaterializer { +public class StringMaterializer extends ObjectMaterializerBase implements PageMaterializer { public static final PageMaterializerFactory Factory = new PageMaterializerFactory() { @Override @@ -27,24 +21,15 @@ public PageMaterializer makeMaterializerNonNull(ValuesReader dataReader, int num } }; - final ValuesReader dataReader; - - final String nullValue; - final String[] data; + private final ValuesReader dataReader; private StringMaterializer(ValuesReader dataReader, int numValues) { this(dataReader, null, numValues); } private StringMaterializer(ValuesReader dataReader, String nullValue, int numValues) { + super(nullValue, new String[numValues]); this.dataReader = dataReader; - this.nullValue = nullValue; - this.data = new String[numValues]; - } - - @Override - public void fillNulls(int startIndex, int endIndex) { - Arrays.fill(data, startIndex, endIndex, nullValue); } @Override @@ -53,15 +38,4 @@ public void fillValues(int startIndex, int endIndex) { data[ii] = dataReader.readBytes().toStringUsingUTF8(); } } - - @Override - public Object fillAll() { - fillValues(0, data.length); - return data; - } - - @Override - public Object data() { - return data; - } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java index 466b7ee1096..4a61cbef58a 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java @@ -35,7 +35,6 @@ import org.jetbrains.annotations.NotNull; import java.io.IOException; -import java.math.BigInteger; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -44,6 +43,7 @@ import java.util.function.BiFunction; import java.util.function.Supplier; +import static io.deephaven.parquet.base.PageMaterializer.resolveDecimalLogicalType; import static io.deephaven.parquet.base.ParquetUtils.METADATA_KEY; public class ParquetSchemaReader { @@ -402,12 +402,7 @@ public Optional> visit(final LogicalTypeAnnotation.EnumLogicalTypeAnnot @Override public Optional> visit( final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { - // This pair of values (precision=1, scale=0) is set at write tiem as a marker so that we can recover - // the fact that the type is a BigInteger, not a BigDecimal when the fies are read. - if (decimalLogicalType.getPrecision() == 1 && decimalLogicalType.getScale() == 0) { - return Optional.of(BigInteger.class); - } - return Optional.of(java.math.BigDecimal.class); + return Optional.of(resolveDecimalLogicalType(decimalLogicalType)); } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java index 12a53b15be0..eecfdde8d1c 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.CodecLookup; import io.deephaven.engine.table.impl.dataindex.RowSetCodec; +import io.deephaven.parquet.base.PageMaterializer; import io.deephaven.stringset.StringSet; import io.deephaven.util.codec.ExternalizableCodec; import io.deephaven.util.codec.SerializableCodec; @@ -411,6 +412,8 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean * We will encode BigIntegers as Decimal types. Parquet has no special type for BigIntegers, but we can maintain * external compatibility by encoding them as fixed length decimals of scale 1. Internally, we'll record that we * wrote this as a decimal, so we can properly decode it back to BigInteger. + * + * @see PageMaterializer#resolveDecimalLogicalType */ private enum BigIntegerType implements TypeInfo { INSTANCE; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java index bad7cad841e..a131d8d8d06 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java @@ -15,9 +15,9 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.AbstractColumnLocation; import io.deephaven.engine.table.impl.sources.regioned.*; +import io.deephaven.parquet.base.BigDecimalParquetBytesCodec; +import io.deephaven.parquet.base.BigIntegerParquetBytesCodec; import io.deephaven.parquet.base.ColumnChunkReader; -import io.deephaven.parquet.table.BigDecimalParquetBytesCodec; -import io.deephaven.parquet.table.BigIntegerParquetBytesCodec; import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; @@ -45,6 +45,9 @@ import java.util.stream.IntStream; import java.util.stream.Stream; +import static io.deephaven.parquet.base.BigDecimalParquetBytesCodec.verifyPrecisionAndScale; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; + final class ParquetColumnLocation extends AbstractColumnLocation { private static final String IMPLEMENTATION_NAME = ParquetColumnLocation.class.getSimpleName(); @@ -441,29 +444,29 @@ private static class LogicalTypeVisitor private final String name; private final ColumnChunkReader columnChunkReader; - private final Class componentType; + private final Class pageType; LogicalTypeVisitor(@NotNull final String name, @NotNull final ColumnChunkReader columnChunkReader, - final Class componentType) { + final Class pageType) { this.name = name; this.columnChunkReader = columnChunkReader; - this.componentType = componentType; + this.pageType = pageType; } @Override public Optional> visit( final LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { return Optional - .of(ToStringPage.create(componentType, columnChunkReader.getDictionarySupplier())); + .of(ToStringPage.create(pageType, columnChunkReader.getDictionarySupplier())); } @Override public Optional> visit( final LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { if (timestampLogicalType.isAdjustedToUTC()) { - return Optional.of(ToInstantPage.create(componentType)); + return Optional.of(ToInstantPage.create(pageType)); } - return Optional.of(ToLocalDateTimePage.create(componentType)); + return Optional.of(ToLocalDateTimePage.create(pageType)); } @Override @@ -471,21 +474,21 @@ private static class LogicalTypeVisitor if (intLogicalType.isSigned()) { switch (intLogicalType.getBitWidth()) { case 8: - return Optional.of(ToBytePage.create(componentType)); + return Optional.of(ToBytePage.create(pageType)); case 16: - return Optional.of(ToShortPage.create(componentType)); + return Optional.of(ToShortPage.create(pageType)); case 32: - return Optional.of(ToIntPage.create(componentType)); + return Optional.of(ToIntPage.create(pageType)); case 64: - return Optional.of(ToLongPage.create(componentType)); + return Optional.of(ToLongPage.create(pageType)); } } else { switch (intLogicalType.getBitWidth()) { case 8: case 16: - return Optional.of(ToCharPage.create(componentType)); + return Optional.of(ToCharPage.create(pageType)); case 32: - return Optional.of(ToLongPage.create(componentType)); + return Optional.of(ToLongPage.create(pageType)); } } return Optional.empty(); @@ -493,12 +496,12 @@ private static class LogicalTypeVisitor @Override public Optional> visit(final LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { - return Optional.of(ToLocalDatePage.create(componentType)); + return Optional.of(ToLocalDatePage.create(pageType)); } @Override public Optional> visit(final LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { - return Optional.of(ToLocalTimePage.create(componentType)); + return Optional.of(ToLocalTimePage.create(pageType)); } @Override @@ -506,37 +509,28 @@ private static class LogicalTypeVisitor final LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) { final PrimitiveType type = columnChunkReader.getType(); final PrimitiveType.PrimitiveTypeName typeName = type.getPrimitiveTypeName(); - switch (typeName) { - case INT32: - return Optional.of(ToBigDecimalFromIntPage.create( - componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case INT64: - return Optional.of(ToBigDecimalFromLongPage.create( - componentType, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - final int encodedSizeInBytes = - (typeName == PrimitiveType.PrimitiveTypeName.BINARY) ? -1 : type.getTypeLength(); - if (BigDecimal.class.equals(componentType)) { - return Optional.of( - ToObjectPage.create( - BigDecimal.class, - new BigDecimalParquetBytesCodec( - decimalLogicalType.getPrecision(), decimalLogicalType.getScale(), - encodedSizeInBytes), - columnChunkReader.getDictionarySupplier())); - } else if (BigInteger.class.equals(componentType)) { - return Optional.of( - ToObjectPage.create( - BigInteger.class, - new BigIntegerParquetBytesCodec(encodedSizeInBytes), - columnChunkReader.getDictionarySupplier())); - } - - // We won't blow up here, Maybe someone will provide us a codec instead. - default: - return Optional.empty(); + final int encodedSizeInBytes = typeName == BINARY ? -1 : type.getTypeLength(); + if (BigDecimal.class.equals(pageType)) { + final int precision = decimalLogicalType.getPrecision(); + final int scale = decimalLogicalType.getScale(); + try { + verifyPrecisionAndScale(precision, scale, typeName); + } catch (final IllegalArgumentException exception) { + throw new TableDataException( + "Invalid scale and precision for column " + name + ": " + exception.getMessage()); + } + return Optional.of(ToBigDecimalPage.create( + pageType, + new BigDecimalParquetBytesCodec(precision, scale, encodedSizeInBytes), + columnChunkReader.getDictionarySupplier())); + } else if (BigInteger.class.equals(pageType)) { + return Optional.of(ToBigIntegerPage.create( + pageType, + new BigIntegerParquetBytesCodec(encodedSizeInBytes), + columnChunkReader.getDictionarySupplier())); } + // We won't blow up here, Maybe someone will provide us a codec instead. + return Optional.empty(); } } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalBase.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalBase.java deleted file mode 100644 index d2a63f2593a..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalBase.java +++ /dev/null @@ -1,39 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table.pagestore.topage; - -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.attributes.Any; -import org.jetbrains.annotations.NotNull; - -import java.math.BigDecimal; - -public abstract class ToBigDecimalBase implements ToPage { - protected final byte scale; - - protected ToBigDecimalBase(@NotNull final Class nativeType, final int precision, final int scale) { - if (!BigDecimal.class.equals(nativeType)) { - throw new IllegalArgumentException( - "The native type for a BigDecimal column is " + nativeType.getCanonicalName()); - } - - this.scale = (byte) scale; - if (((int) this.scale) != scale) { - throw new IllegalArgumentException( - "precision=" + precision + " and scale=" + scale + " can't be represented"); - } - } - - @NotNull - @Override - public Class getNativeType() { - return BigDecimal.class; - } - - @Override - @NotNull - public final ChunkType getChunkType() { - return ChunkType.Object; - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromIntPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromIntPage.java deleted file mode 100644 index 347f942894b..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromIntPage.java +++ /dev/null @@ -1,42 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table.pagestore.topage; - -import io.deephaven.chunk.attributes.Any; -import org.jetbrains.annotations.NotNull; - -import java.math.BigDecimal; - -import static io.deephaven.util.QueryConstants.NULL_INT_BOXED; - -public class ToBigDecimalFromIntPage extends ToBigDecimalBase { - - public static ToPage create( - @NotNull final Class nativeType, - final int precision, - final int scale) { - return new ToBigDecimalFromIntPage(nativeType, precision, scale); - } - - protected ToBigDecimalFromIntPage(@NotNull final Class nativeType, final int precision, final int scale) { - super(nativeType, precision, scale); - } - - @Override - public BigDecimal[] convertResult(@NotNull final Object result) { - final int[] in = (int[]) result; - final int resultLength = in.length; - final BigDecimal[] out = new BigDecimal[resultLength]; - for (int ri = 0; ri < resultLength; ++ri) { - out[ri] = BigDecimal.valueOf(in[ri], scale); - } - return out; - } - - @Override - @NotNull - public final Object nullValue() { - return NULL_INT_BOXED; - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromLongPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromLongPage.java deleted file mode 100644 index 84c9bad2808..00000000000 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalFromLongPage.java +++ /dev/null @@ -1,42 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table.pagestore.topage; - -import io.deephaven.chunk.attributes.Any; -import org.jetbrains.annotations.NotNull; - -import java.math.BigDecimal; - -import static io.deephaven.util.QueryConstants.NULL_LONG_BOXED; - -public class ToBigDecimalFromLongPage extends ToBigDecimalBase { - - public static ToPage create( - @NotNull final Class nativeType, - final int precision, - final int scale) { - return new ToBigDecimalFromLongPage(nativeType, precision, scale); - } - - protected ToBigDecimalFromLongPage(@NotNull final Class nativeType, final int precision, final int scale) { - super(nativeType, precision, scale); - } - - @Override - public BigDecimal[] convertResult(@NotNull final Object result) { - final long[] in = (long[]) result; - final int resultLength = in.length; - final BigDecimal[] out = new BigDecimal[resultLength]; - for (int ri = 0; ri < resultLength; ++ri) { - out[ri] = BigDecimal.valueOf(in[ri], scale); - } - return out; - } - - @Override - @NotNull - public final Object nullValue() { - return NULL_LONG_BOXED; - } -} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalPage.java new file mode 100644 index 00000000000..0e60264aa3e --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigDecimalPage.java @@ -0,0 +1,64 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit ToBigIntegerPage and run "./gradlew replicateToPage" to regenerate +// +// @formatter:off +package io.deephaven.parquet.table.pagestore.topage; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.attributes.Any; +import io.deephaven.parquet.base.BigDecimalParquetBytesCodec; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.codec.ObjectCodec; +import org.apache.parquet.column.Dictionary; +import org.jetbrains.annotations.NotNull; + +import java.math.BigDecimal; +import java.util.function.Function; + +public class ToBigDecimalPage implements ToPage { + + private static final ToBigDecimalPage INSTANCE = new ToBigDecimalPage<>(); + + public static ToPage create( + final Class nativeType, + @NotNull final ObjectCodec codec, + final Function dictionarySupplier) { + if (nativeType == null || BigDecimal.class.equals(nativeType)) { + if (dictionarySupplier == null) { + // noinspection unchecked + return (ToPage) INSTANCE; + } + // Note that dictionary supplier is never null, even if it points to a NULL_DICTIONARY. + // So we always use the following dictionary version of ToPage but internally, we check if the dictionary is + // NULL and fall back to the default implementation. + return new ToPageWithDictionary<>( + BigDecimal.class, + new ChunkDictionary<>( + (dictionary, key) -> { + final byte[] bytes = dictionary.decodeToBinary(key).getBytes(); + return codec.decode(bytes, 0, bytes.length); + }, + dictionarySupplier), + INSTANCE::convertResult); + } + throw new IllegalArgumentException( + "The native type for a BigDecimal column is " + nativeType.getCanonicalName()); + } + + private ToBigDecimalPage() {} + + @Override + @NotNull + public final Class getNativeType() { + return BigDecimal.class; + } + + @Override + @NotNull + public final ChunkType getChunkType() { + return ChunkType.Object; + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigIntegerPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigIntegerPage.java new file mode 100644 index 00000000000..7b38928009d --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToBigIntegerPage.java @@ -0,0 +1,60 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.table.pagestore.topage; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.attributes.Any; +import io.deephaven.parquet.base.BigIntegerParquetBytesCodec; +import io.deephaven.util.channel.SeekableChannelContext; +import io.deephaven.util.codec.ObjectCodec; +import org.apache.parquet.column.Dictionary; +import org.jetbrains.annotations.NotNull; + +import java.math.BigInteger; +import java.util.function.Function; + +public class ToBigIntegerPage implements ToPage { + + private static final ToBigIntegerPage INSTANCE = new ToBigIntegerPage<>(); + + public static ToPage create( + final Class nativeType, + @NotNull final ObjectCodec codec, + final Function dictionarySupplier) { + if (nativeType == null || BigInteger.class.equals(nativeType)) { + if (dictionarySupplier == null) { + // noinspection unchecked + return (ToPage) INSTANCE; + } + // Note that dictionary supplier is never null, even if it points to a NULL_DICTIONARY. + // So we always use the following dictionary version of ToPage but internally, we check if the dictionary is + // NULL and fall back to the default implementation. + return new ToPageWithDictionary<>( + BigInteger.class, + new ChunkDictionary<>( + (dictionary, key) -> { + final byte[] bytes = dictionary.decodeToBinary(key).getBytes(); + return codec.decode(bytes, 0, bytes.length); + }, + dictionarySupplier), + INSTANCE::convertResult); + } + throw new IllegalArgumentException( + "The native type for a BigInteger column is " + nativeType.getCanonicalName()); + } + + private ToBigIntegerPage() {} + + @Override + @NotNull + public final Class getNativeType() { + return BigInteger.class; + } + + @Override + @NotNull + public final ChunkType getChunkType() { + return ChunkType.Object; + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java index a40fb3f086b..e87eff5aa85 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/TransferObject.java @@ -8,6 +8,8 @@ import io.deephaven.engine.table.impl.CodecLookup; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.util.BigDecimalUtils; +import io.deephaven.parquet.base.BigDecimalParquetBytesCodec; +import io.deephaven.parquet.base.BigIntegerParquetBytesCodec; import io.deephaven.parquet.table.*; import io.deephaven.util.SafeCloseable; import io.deephaven.util.codec.ObjectCodec; @@ -87,11 +89,11 @@ static TransferObject create( final BigDecimalUtils.PrecisionAndScale precisionAndScale = TypeInfos.getPrecisionAndScale( computedCache, columnName, tableRowSet, () -> bigDecimalColumnSource); final ObjectCodec codec = new BigDecimalParquetBytesCodec( - precisionAndScale.precision, precisionAndScale.scale, -1); + precisionAndScale.precision, precisionAndScale.scale); return new CodecTransfer<>(bigDecimalColumnSource, codec, tableRowSet, instructions.getTargetPageSize()); } if (columnType == BigInteger.class) { - return new CodecTransfer<>(columnSource, new BigIntegerParquetBytesCodec(-1), tableRowSet, + return new CodecTransfer<>(columnSource, new BigIntegerParquetBytesCodec(), tableRowSet, instructions.getTargetPageSize()); } if (columnType == LocalDate.class) { @@ -135,7 +137,7 @@ static TransferObject create( return new StringArrayTransfer(columnSource, tableRowSet, instructions.getTargetPageSize()); } if (componentType == BigInteger.class) { - return new CodecArrayTransfer<>(columnSource, new BigIntegerParquetBytesCodec(-1), + return new CodecArrayTransfer<>(columnSource, new BigIntegerParquetBytesCodec(), tableRowSet, instructions.getTargetPageSize()); } if (componentType == Instant.class) { @@ -181,7 +183,7 @@ static TransferObject create( return new StringVectorTransfer(columnSource, tableRowSet, instructions.getTargetPageSize()); } if (componentType == BigInteger.class) { - return new CodecVectorTransfer<>(columnSource, new BigIntegerParquetBytesCodec(-1), + return new CodecVectorTransfer<>(columnSource, new BigIntegerParquetBytesCodec(), tableRowSet, instructions.getTargetPageSize()); } if (componentType == Instant.class) { diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodecTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodecTest.java index 658a7f6ade6..527f1537941 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodecTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/BigDecimalParquetBytesCodecTest.java @@ -3,6 +3,7 @@ // package io.deephaven.parquet.table; +import io.deephaven.parquet.base.BigDecimalParquetBytesCodec; import org.junit.Test; import java.math.BigDecimal; @@ -68,7 +69,7 @@ private static BigInteger bi(int val) { } private static BigDecimalParquetBytesCodec codec(int precision, int scale) { - return new BigDecimalParquetBytesCodec(precision, scale, -1); + return new BigDecimalParquetBytesCodec(precision, scale); } private static void checkNoRounding(BigDecimalParquetBytesCodec codec, BigDecimal input, diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java index f95746ab392..67bcc373e4c 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java @@ -37,6 +37,7 @@ import io.deephaven.engine.util.BigDecimalUtils; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.util.file.TrackedFileHandleFactory; +import io.deephaven.parquet.base.BigDecimalParquetBytesCodec; import io.deephaven.parquet.base.InvalidParquetFileException; import io.deephaven.parquet.base.NullStatistics; import io.deephaven.parquet.table.location.ParquetTableLocation; @@ -1365,6 +1366,52 @@ public void testAllNonPartitioningColumnTypes() { } } + @Test + public void decimalLogicalTypeTest() { + final Table expected = TableTools.emptyTable(100_000).update( + "DecimalIntCol = java.math.BigDecimal.valueOf(ii*12, 5)", + "DecimalLongCol = java.math.BigDecimal.valueOf(ii*212, 8)"); + + { + // This reference file has Decimal logical type columns stored as INT32 and INT64 physical types + final String path = + ParquetTableReadWriteTest.class.getResource("/ReferenceDecimalLogicalType.parquet").getFile(); + final Table fromDisk = readParquetFileFromGitLFS(new File(path)); + final ParquetMetadata metadata = + new ParquetTableLocationKey(new File(path).toURI(), 0, null, ParquetInstructions.EMPTY) + .getMetadata(); + final List columnsMetadata = metadata.getFileMetaData().getSchema().getColumns(); + assertEquals("DECIMAL(7,5)", + columnsMetadata.get(0).getPrimitiveType().getLogicalTypeAnnotation().toString()); + assertEquals(PrimitiveType.PrimitiveTypeName.INT32, + columnsMetadata.get(0).getPrimitiveType().getPrimitiveTypeName()); + assertEquals("DECIMAL(12,8)", + columnsMetadata.get(1).getPrimitiveType().getLogicalTypeAnnotation().toString()); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, + columnsMetadata.get(1).getPrimitiveType().getPrimitiveTypeName()); + assertTableEquals(expected, fromDisk); + } + + { + // This reference file has Decimal logical type columns stored as FIXED_LEN_BYTE_ARRAY physical types + final String path = + ParquetTableReadWriteTest.class.getResource("/ReferenceDecimalLogicalType2.parquet").getFile(); + final Table fromDisk = readParquetFileFromGitLFS(new File(path)); + final ParquetMetadata metadata = + new ParquetTableLocationKey(new File(path).toURI(), 0, null, ParquetInstructions.EMPTY) + .getMetadata(); + final List columnsMetadata = metadata.getFileMetaData().getSchema().getColumns(); + assertEquals("DECIMAL(7,5)", + columnsMetadata.get(0).getPrimitiveType().getLogicalTypeAnnotation().toString()); + assertEquals(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + columnsMetadata.get(0).getPrimitiveType().getPrimitiveTypeName()); + assertEquals("DECIMAL(12,8)", + columnsMetadata.get(1).getPrimitiveType().getLogicalTypeAnnotation().toString()); + assertEquals(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + columnsMetadata.get(1).getPrimitiveType().getPrimitiveTypeName()); + assertTableEquals(expected, fromDisk); + } + } @Test public void testVectorColumns() { @@ -1529,7 +1576,7 @@ public void stringDictionaryTest() { */ private Table maybeFixBigDecimal(Table toFix) { final BigDecimalUtils.PrecisionAndScale pas = BigDecimalUtils.computePrecisionAndScale(toFix, "bdColumn"); - final BigDecimalParquetBytesCodec codec = new BigDecimalParquetBytesCodec(pas.precision, pas.scale, -1); + final BigDecimalParquetBytesCodec codec = new BigDecimalParquetBytesCodec(pas.precision, pas.scale); ExecutionContext.getContext() .getQueryScope() diff --git a/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType.parquet b/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType.parquet new file mode 100644 index 00000000000..f6cb4cf8507 --- /dev/null +++ b/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:e559539ce712961badc6f4af7586f02f281c7164b762ca39bacef962ad70c203 +size 891184 diff --git a/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType2.parquet b/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType2.parquet new file mode 100644 index 00000000000..e9aa389d375 --- /dev/null +++ b/extensions/parquet/table/src/test/resources/ReferenceDecimalLogicalType2.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:6b37b67a9b5aa942e23bec6fa9c9db3d1af847626c0a5f0d2c3e6d0c45cd53cd +size 1249011 diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicatePageMaterializers.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicatePageMaterializers.java index 489ebe04e78..13909dd32b3 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicatePageMaterializers.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicatePageMaterializers.java @@ -21,31 +21,17 @@ public class ReplicatePageMaterializers { private static final String CHAR_MATERIALIZER_PATH = MATERIALIZER_DIR + "CharMaterializer.java"; private static final String FLOAT_MATERIALIZER_PATH = MATERIALIZER_DIR + "FloatMaterializer.java"; - private static final String INT_MATERIALIZER_PATH = MATERIALIZER_DIR + "IntMaterializer.java"; - private static final String STRING_MATERIALIZER_PATH = MATERIALIZER_DIR + "StringMaterializer.java"; - - private static final String LOCAL_TIME_MATERIALIZER_BASE_PATH = - MATERIALIZER_DIR + "LocalTimeMaterializerBase.java"; - private static final String LOCAL_TIME_FROM_MILLIS_MATERIALIZER_PATH = - MATERIALIZER_DIR + "LocalTimeFromMillisMaterializer.java"; private static final String LOCAL_TIME_FROM_MICROS_MATERIALIZER_PATH = MATERIALIZER_DIR + "LocalTimeFromMicrosMaterializer.java"; - private static final String LOCAL_TIME_FROM_NANOS_MATERIALIZER_PATH = - MATERIALIZER_DIR + "LocalTimeFromNanosMaterializer.java"; - - private static final String LOCAL_DATE_TIME_MATERIALIZER_BASE_PATH = - MATERIALIZER_DIR + "LocalDateTimeMaterializerBase.java"; private static final String LOCAL_DATE_TIME_FROM_MILLIS_MATERIALIZER_PATH = MATERIALIZER_DIR + "LocalDateTimeFromMillisMaterializer.java"; - private static final String LOCAL_DATE_TIME_FROM_MICROS_MATERIALIZER_PATH = - MATERIALIZER_DIR + "LocalDateTimeFromMicrosMaterializer.java"; - private static final String LOCAL_DATE_TIME_FROM_NANOS_MATERIALIZER_PATH = - MATERIALIZER_DIR + "LocalDateTimeFromNanosMaterializer.java"; - - private static final String INSTANT_NANOS_FROM_MILLIS_MATERIALIZER_PATH = - MATERIALIZER_DIR + "InstantNanosFromMillisMaterializer.java"; private static final String INSTANT_NANOS_FROM_MICROS_MATERIALIZER_PATH = MATERIALIZER_DIR + "InstantNanosFromMicrosMaterializer.java"; + private static final String BIG_DECIMAL_FROM_LONG_MATERIALIZER_PATH = + MATERIALIZER_DIR + "BigDecimalFromLongMaterializer.java"; + private static final String BIG_DECIMAL_FROM_BYTES_MATERIALIZER_PATH = + MATERIALIZER_DIR + "BigDecimalFromBytesMaterializer.java"; + private static final String BIG_INTEGER_MATERIALIZER_PATH = MATERIALIZER_DIR + "BigIntegerMaterializer.java"; public static void main(String... args) throws IOException { charToShortAndByte(TASK, CHAR_MATERIALIZER_PATH, NO_EXCEPTIONS); @@ -59,16 +45,7 @@ public static void main(String... args) throws IOException { {"Float", "Int"}, {"float", "int"} }; - replaceAll(TASK, FLOAT_MATERIALIZER_PATH, INT_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); - - // Float -> String - pairs = new String[][] { - {"readFloat()", "readBytes().toStringUsingUTF8"}, - {"Float", "String"}, - {"float", "String"}, - {"dataReader, 0, numValues", "dataReader, null, numValues"} - }; - replaceAll(TASK, FLOAT_MATERIALIZER_PATH, STRING_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, FLOAT_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); // LocalTimeFromMicros -> LocalTimeFromMillis // We change from Micros to Millis and not the other way since converting from Long to Integer has fewer @@ -78,56 +55,49 @@ public static void main(String... args) throws IOException { {"micros", "millis"}, {"readLong", "readInteger"}, }; - replaceAll(TASK, - LOCAL_TIME_FROM_MICROS_MATERIALIZER_PATH, - LOCAL_TIME_FROM_MILLIS_MATERIALIZER_PATH, - null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, LOCAL_TIME_FROM_MICROS_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); // LocalTimeFromMicros -> LocalTimeFromNanos pairs = new String[][] { {"Micros", "Nanos"}, {"micros", "nanos"}, }; - replaceAll(TASK, - LOCAL_TIME_FROM_MICROS_MATERIALIZER_PATH, - LOCAL_TIME_FROM_NANOS_MATERIALIZER_PATH, - null, NO_EXCEPTIONS, pairs); - - // LocalTimeBase -> LocalDateTimeBase - pairs = new String[][] { - {"LocalTime", "LocalDateTime"} - }; - replaceAll(TASK, - LOCAL_TIME_MATERIALIZER_BASE_PATH, - LOCAL_DATE_TIME_MATERIALIZER_BASE_PATH, - null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, LOCAL_TIME_FROM_MICROS_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); // LocalDateTimeFromMillis -> LocalDateTimeFromMicros pairs = new String[][] { {"Millis", "Micros"} }; - replaceAll(TASK, - LOCAL_DATE_TIME_FROM_MILLIS_MATERIALIZER_PATH, - LOCAL_DATE_TIME_FROM_MICROS_MATERIALIZER_PATH, - null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, LOCAL_DATE_TIME_FROM_MILLIS_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); // LocalDateTimeFromMillis -> LocalDateTimeFromNanos pairs = new String[][] { {"Millis", "Nanos"} }; - replaceAll(TASK, - LOCAL_DATE_TIME_FROM_MILLIS_MATERIALIZER_PATH, - LOCAL_DATE_TIME_FROM_NANOS_MATERIALIZER_PATH, - null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, LOCAL_DATE_TIME_FROM_MILLIS_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); // InstantNanosFromMicros -> InstantNanosFromMillis pairs = new String[][] { {"Micros", "Millis"}, {"micros", "millis"} }; + replaceAll(TASK, INSTANT_NANOS_FROM_MICROS_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); + + // BigDecimalFromLong -> BigDecimalFromInt + pairs = new String[][] { + {"readLong", "readInteger"}, + {"Long", "Int"} + }; + replaceAll(TASK, BIG_DECIMAL_FROM_LONG_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); + + // BigDecimal -> BigInteger + pairs = new String[][] { + {"BigDecimalFromBytes", "BigInteger"}, + {"BigDecimal", "BigInteger"} + }; replaceAll(TASK, - INSTANT_NANOS_FROM_MICROS_MATERIALIZER_PATH, - INSTANT_NANOS_FROM_MILLIS_MATERIALIZER_PATH, + BIG_DECIMAL_FROM_BYTES_MATERIALIZER_PATH, + BIG_INTEGER_MATERIALIZER_PATH, null, NO_EXCEPTIONS, pairs); } } diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateToPage.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateToPage.java index aa3e1a715be..991943678ab 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateToPage.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateToPage.java @@ -15,23 +15,33 @@ public class ReplicateToPage { private static final String TASK = "replicateToPage"; private static final String[] NO_EXCEPTIONS = new String[0]; + private static final String TO_PAGE_DIR = + "extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/"; + + private static final String TO_INT_PAGE_PATH = TO_PAGE_DIR + "ToIntPage.java"; + private static final String TO_LOCAL_DATE_PAGE_PATH = TO_PAGE_DIR + "ToLocalDatePage.java"; + private static final String TO_BIG_INTEGER_PAGE_PATH = TO_PAGE_DIR + "ToBigIntegerPage.java"; + public static void main(String... args) throws IOException { - intToAllButBoolean(TASK, - "extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToIntPage.java", - "interface"); + intToAllButBoolean(TASK, TO_INT_PAGE_PATH, "interface"); // LocalDate -> LocalDateTime - final String sourcePath = - "extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToLocalDatePage.java"; String[][] pairs = new String[][] { {"LocalDate", "LocalDateTime"} }; - replaceAll(TASK, sourcePath, null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, TO_LOCAL_DATE_PAGE_PATH, null, NO_EXCEPTIONS, pairs); // LocalDate -> LocalTime pairs = new String[][] { {"LocalDate", "LocalTime"} }; - replaceAll(TASK, sourcePath, null, NO_EXCEPTIONS, pairs); + replaceAll(TASK, TO_LOCAL_DATE_PAGE_PATH, null, NO_EXCEPTIONS, pairs); + + // BigInteger -> BigDecimal + pairs = new String[][] { + {"BigInteger", "BigDecimal"} + }; + replaceAll(TASK, TO_BIG_INTEGER_PAGE_PATH, null, NO_EXCEPTIONS, pairs); + } } From 57941cd702657c19ac2cd05df56b2a11ee10c268 Mon Sep 17 00:00:00 2001 From: arman-ddl <122062464+arman-ddl@users.noreply.github.com> Date: Thu, 20 Jun 2024 21:10:32 -0700 Subject: [PATCH 03/18] Allow extra headers in Barrage session creation (#5637) * Accept extra headers in Barrage session creation * Preserve extra header order * Update py/server/deephaven/barrage.py * Add extra headers to unit test * Move headers to own test * Force CI to rerun completely --------- Co-authored-by: Jianfeng Mao <4297243+jmao-denver@users.noreply.github.com> Co-authored-by: jianfengmao --- py/server/deephaven/barrage.py | 9 +++++++-- py/server/tests/test_barrage.py | 4 ++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/py/server/deephaven/barrage.py b/py/server/deephaven/barrage.py index 00e4a54d0bb..8680c718756 100644 --- a/py/server/deephaven/barrage.py +++ b/py/server/deephaven/barrage.py @@ -116,6 +116,7 @@ def barrage_session(host: str, auth_token: str = "", use_tls: bool = False, tls_root_certs: bytes = None, + extra_headers: Dict[str, str] = None ) -> BarrageSession: """Returns a Deephaven gRPC session to a remote server if a cached session is available; otherwise, creates a new session. @@ -135,6 +136,7 @@ def barrage_session(host: str, tls_root_certs (bytes): PEM encoded root certificates to use for TLS connection, or None to use system defaults. If not None implies use a TLS connection and the use_tls argument should have been passed as True. Defaults to None + extra_headers (Dict[str, str]): extra headers to set when configuring the gRPC channel. Defaults to None. Returns: a Deephaven Barrage session @@ -152,7 +154,7 @@ def barrage_session(host: str, else: target_uri = f"dh+plain://{target_uri}" - j_client_config = _build_client_config(target_uri, tls_root_certs) + j_client_config = _build_client_config(target_uri, tls_root_certs, extra_headers) auth = f"{auth_type} {auth_token}" try: @@ -209,9 +211,12 @@ def _get_barrage_session_direct(client_config: jpy.JType, auth: str) -> BarrageS return BarrageSession(j_barrage_session, j_channel) -def _build_client_config(target_uri: str, tls_root_certs: bytes) -> jpy.JType: +def _build_client_config(target_uri: str, tls_root_certs: bytes, extra_headers: Dict[str, str] = None) -> jpy.JType: j_client_config_builder = _JClientConfig.builder() j_client_config_builder.target(_JDeephavenTarget.of(_JURI(target_uri))) + if extra_headers: + for header, value in extra_headers.items(): + j_client_config_builder.putExtraHeaders(header, value) if tls_root_certs: j_ssl_config = _JSSLConfig.builder().trust( _JTrustCustom.ofX509(tls_root_certs, 0, len(tls_root_certs))).build() diff --git a/py/server/tests/test_barrage.py b/py/server/tests/test_barrage.py index b3411f60ad7..28c0a49b6aa 100644 --- a/py/server/tests/test_barrage.py +++ b/py/server/tests/test_barrage.py @@ -68,6 +68,10 @@ def test_barrage_session(self): with self.assertRaises(DHError): barrage_session(host="localhost", port=10000, auth_type="Basic", auth_token="user:password") + def test_barrage_session_with_extra_headers(self): + session = barrage_session(host="localhost", port=10000, auth_type="Anonymous", extra_headers={"envoy-prefix": "test"}) + self.assertIsNotNone(session) + def test_subscribe(self): session = barrage_session(host="localhost", port=10000, auth_type="Anonymous") t = session.subscribe(ticket=self.shared_ticket.bytes) From a9ebacf979463a0d96063a1eb17aeed801259fc0 Mon Sep 17 00:00:00 2001 From: Nate Bauernfeind Date: Fri, 21 Jun 2024 15:17:53 -0600 Subject: [PATCH 04/18] ErrorTransformer: prevent NPE (#5633) --- .../appmode/ApplicationTicketResolver.java | 26 ++++++++++---- .../server/console/ScopeTicketResolver.java | 36 ++++++++++++------- .../HierarchicalTableServiceGrpcImpl.java | 21 +++++++++++ .../PartitionedTableServiceGrpcImpl.java | 8 +++++ .../server/session/ExportTicketResolver.java | 2 +- .../server/session/SharedTicketResolver.java | 31 ++++++++++------ .../server/session/TicketResolver.java | 3 +- 7 files changed, 96 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/io/deephaven/server/appmode/ApplicationTicketResolver.java b/server/src/main/java/io/deephaven/server/appmode/ApplicationTicketResolver.java index cf76f5caa86..a2084a87ebd 100644 --- a/server/src/main/java/io/deephaven/server/appmode/ApplicationTicketResolver.java +++ b/server/src/main/java/io/deephaven/server/appmode/ApplicationTicketResolver.java @@ -17,7 +17,9 @@ import io.deephaven.server.session.SessionState; import io.deephaven.server.session.TicketResolverBase; import io.deephaven.server.session.TicketRouter; +import io.grpc.StatusRuntimeException; import org.apache.arrow.flight.impl.Flight; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import javax.inject.Inject; @@ -79,10 +81,12 @@ private SessionState.ExportObject resolve(final AppFieldId id, final Stri } final Field field = id.app.getField(id.fieldName); if (field == null) { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + "': field '" + getLogNameFor(id) + "' not found"); + throw newNotFoundSRE(logId, id); } Object value = authorization.transform(field.value()); + if (value == null) { + throw newNotFoundSRE(logId, id); + } // noinspection unchecked return SessionState.wrapAsExport((T) value); } @@ -100,16 +104,18 @@ public SessionState.ExportObject flightInfoFor( synchronized (id.app) { Field field = id.app.getField(id.fieldName); if (field == null) { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + "': field '" + getLogNameFor(id) + "' not found"); + throw newNotFoundSRE(logId, id); } Object value = field.value(); if (value instanceof Table) { + // may return null if the table is not authorized value = authorization.transform(value); + } + + if (value instanceof Table) { info = TicketRouter.getFlightInfo((Table) value, descriptor, flightTicketForName(id.app, id.fieldName)); } else { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + "': field '" + getLogNameFor(id) + "' is not a flight"); + throw newNotFoundSRE(logId, id); } } @@ -151,7 +157,10 @@ public void forAllFlightInfo(@Nullable SessionState session, Consumer { Object value = field.value(); if (value instanceof Table) { + // may return null if the table is not authorized value = authorization.transform(value); + } + if (value instanceof Table) { final Flight.FlightInfo info = TicketRouter.getFlightInfo((Table) value, descriptorForName(app, field.name()), flightTicketForName(app, field.name())); visitor.accept(info); @@ -202,6 +211,11 @@ public static Flight.FlightDescriptor descriptorForName(final ApplicationState a .build(); } + private @NotNull StatusRuntimeException newNotFoundSRE(String logId, AppFieldId id) { + return Exceptions.statusRuntimeException(Code.NOT_FOUND, + "Could not resolve '" + logId + "': field '" + getLogNameFor(id) + "' not found"); + } + private AppFieldId appFieldIdFor(final ByteBuffer ticket, final String logId) { if (ticket == null) { throw Exceptions.statusRuntimeException(Code.FAILED_PRECONDITION, diff --git a/server/src/main/java/io/deephaven/server/console/ScopeTicketResolver.java b/server/src/main/java/io/deephaven/server/console/ScopeTicketResolver.java index 135dd779ed3..2ee01446d39 100644 --- a/server/src/main/java/io/deephaven/server/console/ScopeTicketResolver.java +++ b/server/src/main/java/io/deephaven/server/console/ScopeTicketResolver.java @@ -18,7 +18,9 @@ import io.deephaven.server.session.SessionState; import io.deephaven.server.session.TicketResolverBase; import io.deephaven.server.session.TicketRouter; +import io.grpc.StatusRuntimeException; import org.apache.arrow.flight.impl.Flight; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import javax.inject.Inject; @@ -51,19 +53,20 @@ public SessionState.ExportObject flightInfoFor( // there is no mechanism to wait for a scope variable to resolve; require that the scope variable exists now final String scopeName = nameForDescriptor(descriptor, logId); - QueryScope queryScope = ExecutionContext.getContext().getQueryScope(); - Object scopeVar = queryScope.unwrapObject(queryScope.readParamValue(scopeName, null)); + final QueryScope queryScope = ExecutionContext.getContext().getQueryScope(); + final Object scopeVar = queryScope.unwrapObject(queryScope.readParamValue(scopeName, null)); if (scopeVar == null) { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + ": no table exists with name '" + scopeName + "'"); + throw newNotFoundSRE(logId, scopeName); } if (!(scopeVar instanceof Table)) { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + "': no table exists with name '" + scopeName + "'"); + throw newNotFoundSRE(logId, scopeName); } - Table transformed = authorization.transform((Table) scopeVar); - Flight.FlightInfo flightInfo = + final Table transformed = authorization.transform((Table) scopeVar); + if (transformed == null) { + throw newNotFoundSRE(logId, scopeName); + } + final Flight.FlightInfo flightInfo = TicketRouter.getFlightInfo(transformed, descriptor, flightTicketForName(scopeName)); return SessionState.wrapAsExport(flightInfo); @@ -72,8 +75,13 @@ public SessionState.ExportObject flightInfoFor( @Override public void forAllFlightInfo(@Nullable final SessionState session, final Consumer visitor) { final QueryScope queryScope = ExecutionContext.getContext().getQueryScope(); - queryScope.toMap(queryScope::unwrapObject, (n, t) -> t instanceof Table).forEach((name, table) -> visitor - .accept(TicketRouter.getFlightInfo((Table) table, descriptorForName(name), flightTicketForName(name)))); + queryScope.toMap(queryScope::unwrapObject, (n, t) -> t instanceof Table).forEach((name, table) -> { + final Table transformedTable = authorization.transform((Table) table); + if (transformedTable != null) { + visitor.accept(TicketRouter.getFlightInfo( + transformedTable, descriptorForName(name), flightTicketForName(name))); + } + }); } @Override @@ -101,8 +109,7 @@ private SessionState.ExportObject resolve(final String scopeName, final S export = authorization.transform(export); if (export == null) { - return SessionState.wrapAsFailedExport(Exceptions.statusRuntimeException(Code.FAILED_PRECONDITION, - "Could not resolve '" + logId + "': no variable exists with name '" + scopeName + "'")); + return SessionState.wrapAsFailedExport(newNotFoundSRE(logId, scopeName)); } return SessionState.wrapAsExport(export); @@ -268,4 +275,9 @@ public static Flight.FlightDescriptor ticketToDescriptor(final Flight.Ticket tic public static Flight.Ticket descriptorToTicket(final Flight.FlightDescriptor descriptor, final String logId) { return flightTicketForName(nameForDescriptor(descriptor, logId)); } + + private static @NotNull StatusRuntimeException newNotFoundSRE(String logId, String scopeName) { + return Exceptions.statusRuntimeException(Code.NOT_FOUND, + "Could not resolve '" + logId + ": variable '" + scopeName + "' not found"); + } } diff --git a/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableServiceGrpcImpl.java b/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableServiceGrpcImpl.java index ea9cbe2d881..08e6e9eeba6 100644 --- a/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableServiceGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableServiceGrpcImpl.java @@ -105,6 +105,10 @@ public void rollup( aggregations, includeConstituents, groupByColumns); final RollupTable transformedResult = authTransformation.transform(result); + if (transformedResult == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to rollup hierarchical table"); + } safelyComplete(responseObserver, RollupResponse.getDefaultInstance()); return transformedResult; }); @@ -161,6 +165,10 @@ public void tree( identifierColumn.name(), parentIdentifierColumn.name()); final TreeTable transformedResult = authTransformation.transform(result); + if (transformedResult == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to tree hierarchical table"); + } safelyComplete(responseObserver, TreeResponse.getDefaultInstance()); return transformedResult; }); @@ -262,6 +270,10 @@ public void apply( } final HierarchicalTable transformedResult = authTransformation.transform(result); + if (transformedResult == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to apply to hierarchical table"); + } safelyComplete(responseObserver, HierarchicalTableApplyResponse.getDefaultInstance()); return transformedResult; }); @@ -423,6 +435,10 @@ public void view( } final HierarchicalTableView transformedResult = authTransformation.transform(result); + if (transformedResult == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to view hierarchical table"); + } safelyComplete(responseObserver, HierarchicalTableViewResponse.getDefaultInstance()); return transformedResult; }); @@ -478,6 +494,11 @@ public void exportSource( authWiring.checkPermissionExportSource(session.getAuthContext(), request, List.of(result)); final Table transformedResult = authTransformation.transform(result); + if (transformedResult == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, + "Not authorized to export source from hierarchical table"); + } final ExportedTableCreationResponse response = ExportUtil.buildTableCreationResponse(request.getResultTableId(), transformedResult); safelyComplete(responseObserver, response); diff --git a/server/src/main/java/io/deephaven/server/partitionedtable/PartitionedTableServiceGrpcImpl.java b/server/src/main/java/io/deephaven/server/partitionedtable/PartitionedTableServiceGrpcImpl.java index 1cdab319c0d..7c45821873e 100644 --- a/server/src/main/java/io/deephaven/server/partitionedtable/PartitionedTableServiceGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/partitionedtable/PartitionedTableServiceGrpcImpl.java @@ -112,6 +112,10 @@ public void merge( merged = partitionedTable.get().merge(); } merged = authorizationTransformation.transform(merged); + if (merged == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to merge table."); + } final ExportedTableCreationResponse response = buildTableCreationResponse(request.getResultId(), merged); safelyComplete(responseObserver, response); @@ -187,6 +191,10 @@ public void getTable( table = authorizationTransformation.transform(table); final ExportedTableCreationResponse response = buildTableCreationResponse(request.getResultId(), table); + if (table == null) { + throw Exceptions.statusRuntimeException( + Code.FAILED_PRECONDITION, "Not authorized to get table."); + } safelyComplete(responseObserver, response); return table; }); diff --git a/server/src/main/java/io/deephaven/server/session/ExportTicketResolver.java b/server/src/main/java/io/deephaven/server/session/ExportTicketResolver.java index aa4430dcef7..b72448b5166 100644 --- a/server/src/main/java/io/deephaven/server/session/ExportTicketResolver.java +++ b/server/src/main/java/io/deephaven/server/session/ExportTicketResolver.java @@ -55,7 +55,7 @@ public SessionState.ExportObject flightInfoFor( } throw Exceptions.statusRuntimeException(Code.NOT_FOUND, - "Could not resolve '" + logId + "': flight '" + descriptor + "' is not a table"); + "Could not resolve '" + logId + "': flight '" + descriptor + "' not found"); }); } diff --git a/server/src/main/java/io/deephaven/server/session/SharedTicketResolver.java b/server/src/main/java/io/deephaven/server/session/SharedTicketResolver.java index 3408976f448..02230b4b45e 100644 --- a/server/src/main/java/io/deephaven/server/session/SharedTicketResolver.java +++ b/server/src/main/java/io/deephaven/server/session/SharedTicketResolver.java @@ -15,6 +15,7 @@ import io.deephaven.proto.util.Exceptions; import io.deephaven.proto.util.SharedTicketHelper; import io.deephaven.server.auth.AuthorizationProvider; +import io.grpc.StatusRuntimeException; import org.apache.arrow.flight.impl.Flight; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -63,22 +64,22 @@ public SessionState.ExportObject flightInfoFor( SessionState.ExportObject export = sharedVariables.get(sharedId); if (export == null) { - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, String.format( - "Could not resolve '%s': no shared ticket exists with id 0x%s", logId, toHexString(sharedId))); + throw newNotFoundSRE(logId, toHexString(sharedId)); } return session.nonExport() .require(export) .submit(() -> { - final Object result = export.get(); + Object result = export.get(); if (result instanceof Table) { - final Table table = (Table) authorization.transform(result); - return TicketRouter.getFlightInfo(table, descriptor, + result = authorization.transform(result); + } + if (result instanceof Table) { + return TicketRouter.getFlightInfo((Table) result, descriptor, FlightExportTicketHelper.descriptorToFlightTicket(descriptor, logId)); } - throw Exceptions.statusRuntimeException(Code.NOT_FOUND, String.format( - "Could not resolve '%s': flight '%s' is not a table", logId, descriptor)); + throw newNotFoundSRE(logId, toHexString(sharedId)); }); } @@ -109,16 +110,19 @@ private SessionState.ExportObject resolve( // noinspection unchecked final SessionState.ExportObject sharedVar = (SessionState.ExportObject) sharedVariables.get(sharedId); if (sharedVar == null) { - return SessionState.wrapAsFailedExport(Exceptions.statusRuntimeException(Code.NOT_FOUND, String.format( - "Could not resolve '%s': no shared ticket exists with id '%s'", logId, toHexString(sharedId)))); + return SessionState.wrapAsFailedExport(newNotFoundSRE(logId, toHexString(sharedId))); } // we need to wrap this in a new export object to hand off to the new session and defer checking permissions return session.nonExport() .require(sharedVar) .submit(() -> { - final T result = sharedVar.get(); - return authorization.transform(result); + T result = sharedVar.get(); + result = authorization.transform(result); + if (result == null) { + throw newNotFoundSRE(logId, toHexString(sharedId)); + } + return result; }); } @@ -284,4 +288,9 @@ public static Flight.FlightDescriptor ticketToDescriptor(final Flight.Ticket tic public static Flight.Ticket descriptorToTicket(final Flight.FlightDescriptor descriptor, final String logId) { return flightTicketForId(idForDescriptor(descriptor, logId).toByteArray()); } + + private static @NotNull StatusRuntimeException newNotFoundSRE(String logId, String sharedId) { + return Exceptions.statusRuntimeException(Code.NOT_FOUND, String.format( + "Could not resolve '%s': ticket '%s' not found", logId, sharedId)); + } } diff --git a/server/src/main/java/io/deephaven/server/session/TicketResolver.java b/server/src/main/java/io/deephaven/server/session/TicketResolver.java index b19dc68a7b4..cb74ffcc49f 100644 --- a/server/src/main/java/io/deephaven/server/session/TicketResolver.java +++ b/server/src/main/java/io/deephaven/server/session/TicketResolver.java @@ -24,7 +24,8 @@ interface Authorization { * transformations to requested resources. * * @param source the object to transform (such as by applying ACLs) - * @return an object that has been sanitized to be used by the current user + * @return an object that has been sanitized to be used by the current user; may return null if user does not + * have access to the resource */ T transform(T source); From c735716603b8501bced3ad8d7a98c6230a0ba0bf Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Fri, 21 Jun 2024 16:27:35 -0500 Subject: [PATCH 05/18] [BREAKING] Fix for OOM error when reading from S3 (#5613) Breaking changes to S3 API --- .../extensions/s3/S3ChannelContext.java | 81 ++------ .../extensions/s3/S3Instructions.java | 27 +-- .../io/deephaven/extensions/s3/S3Request.java | 173 ++++++++++-------- .../extensions/s3/S3RequestCache.java | 39 +++- .../extensions/s3/S3InstructionsTest.java | 25 --- py/server/deephaven/experimental/s3.py | 6 - 6 files changed, 143 insertions(+), 208 deletions(-) diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java index 15bbed956c9..fb58ee43632 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3ChannelContext.java @@ -8,7 +8,6 @@ import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.BaseSeekableChannelContext; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3Uri; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; @@ -16,7 +15,6 @@ import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Objects; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -32,7 +30,7 @@ final class S3ChannelContext extends BaseSeekableChannelContext implements Seeka static final long UNINITIALIZED_SIZE = -1; private static final long UNINITIALIZED_NUM_FRAGMENTS = -1; - final S3SeekableChannelProvider provider; + private final S3SeekableChannelProvider provider; final S3AsyncClient client; final S3Instructions instructions; @@ -48,12 +46,6 @@ final class S3ChannelContext extends BaseSeekableChannelContext implements Seeka */ final S3RequestCache sharedCache; - /** - * Used to cache recently fetched fragments as well as the ownership token for the request. This cache is local to - * the context and is used to keep the requests alive as long as the context is alive. - */ - private final S3Request.AcquiredRequest[] localCache; - /** * The size of the object in bytes, stored in context to avoid fetching multiple times */ @@ -72,7 +64,6 @@ final class S3ChannelContext extends BaseSeekableChannelContext implements Seeka this.provider = Objects.requireNonNull(provider); this.client = Objects.requireNonNull(client); this.instructions = Objects.requireNonNull(instructions); - this.localCache = new S3Request.AcquiredRequest[instructions.maxCacheSize()]; this.sharedCache = sharedCache; if (sharedCache.getFragmentSize() != instructions.fragmentSize()) { throw new IllegalArgumentException("Fragment size mismatch between shared cache and instructions, " @@ -121,32 +112,31 @@ int fill(final long position, final ByteBuffer dest) throws IOException { final int impliedReadAhead = (int) (lastFragmentIx - firstFragmentIx); final int desiredReadAhead = instructions.readAheadCount(); final long totalRemainingFragments = numFragments - firstFragmentIx - 1; - final int maxReadAhead = instructions.maxCacheSize() - 1; - readAhead = Math.min( - Math.max(impliedReadAhead, desiredReadAhead), - (int) Math.min(maxReadAhead, totalRemainingFragments)); + readAhead = Math.min(Math.max(impliedReadAhead, desiredReadAhead), totalRemainingFragments); } - final S3Request firstRequest = getOrCreateRequest(firstFragmentIx); - for (int i = 0; i < readAhead; ++i) { - getOrCreateRequest(firstFragmentIx + i + 1); + int filled; + { + // Hold a reference to the first request to ensure it is not evicted from the cache + final S3Request.Acquired acquiredRequest = getOrCreateRequest(firstFragmentIx); + for (int i = 0; i < readAhead; ++i) { + // Do not hold references to the read-ahead requests + getOrCreateRequest(firstFragmentIx + i + 1); + } + // blocking + filled = acquiredRequest.fill(position, dest); } - // blocking - int filled = firstRequest.fill(position, dest); for (int i = 0; dest.hasRemaining(); ++i) { - // Since we have already created requests for read ahead fragments, we can retrieve them from the local - // cache - final S3Request request = getRequestFromLocalCache(firstFragmentIx + i + 1); - if (request == null || !request.isDone()) { + final S3Request.Acquired readAheadRequest = sharedCache.getRequest(uri, firstFragmentIx + i + 1); + if (readAheadRequest == null || !readAheadRequest.isDone()) { break; } // non-blocking since we know isDone - filled += request.fill(position + filled, dest); + filled += readAheadRequest.fill(position + filled, dest); } return filled; } private void reset() { - releaseOutstanding(); // Reset the internal state uri = null; size = UNINITIALIZED_SIZE; @@ -162,49 +152,16 @@ public void close() { if (log.isDebugEnabled()) { log.debug().append("Closing context: ").append(ctxStr()).endl(); } - releaseOutstanding(); - } - - /** - * Release all outstanding requests associated with this context. Eventually, the request will be canceled when the - * objects are garbage collected. - */ - private void releaseOutstanding() { - Arrays.fill(localCache, null); } // -------------------------------------------------------------------------------------------------- - @Nullable - private S3Request getRequestFromLocalCache(final long fragmentIndex) { - return getRequestFromLocalCache(fragmentIndex, cacheIndex(fragmentIndex)); - } - - @Nullable - private S3Request getRequestFromLocalCache(final long fragmentIndex, final int cacheIdx) { - if (localCache[cacheIdx] != null && localCache[cacheIdx].request.isFragment(fragmentIndex)) { - return localCache[cacheIdx].request; - } - return null; - } - @NotNull - private S3Request getOrCreateRequest(final long fragmentIndex) { - final int cacheIdx = cacheIndex(fragmentIndex); - final S3Request locallyCached = getRequestFromLocalCache(fragmentIndex, cacheIdx); - if (locallyCached != null) { - return locallyCached; - } - final S3Request.AcquiredRequest sharedCacheRequest = sharedCache.getOrCreateRequest(uri, fragmentIndex, this); - // Cache the request and the ownership token locally - localCache[cacheIdx] = sharedCacheRequest; + private S3Request.Acquired getOrCreateRequest(final long fragmentIndex) { + final S3Request.Acquired cachedRequest = sharedCache.getOrCreateRequest(uri, fragmentIndex, this); // Send the request, if not sent already. The following method is idempotent, so we always call it. - sharedCacheRequest.request.sendRequest(); - return sharedCacheRequest.request; - } - - private int cacheIndex(final long fragmentIndex) { - return (int) (fragmentIndex % instructions.maxCacheSize()); + cachedRequest.send(); + return cachedRequest; } private long fragmentIndex(final long pos) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java index 7560e9377df..27d313a235c 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Instructions.java @@ -28,7 +28,6 @@ public abstract class S3Instructions implements LogOutputAppendable { private final static int DEFAULT_READ_AHEAD_COUNT = 32; private final static int DEFAULT_FRAGMENT_SIZE = 1 << 16; // 64 KiB private final static int MIN_FRAGMENT_SIZE = 8 << 10; // 8 KiB - private final static int DEFAULT_MAX_CACHE_SIZE = 256; private final static Duration DEFAULT_CONNECTION_TIMEOUT = Duration.ofSeconds(2); private final static Duration DEFAULT_READ_TIMEOUT = Duration.ofSeconds(2); @@ -73,17 +72,6 @@ public int fragmentSize() { return DEFAULT_FRAGMENT_SIZE; } - /** - * The maximum number of fragments to cache in memory, defaults to - * {@code Math.max(1 + readAheadCount(), DEFAULT_MAX_CACHE_SIZE)}, which is at least - * {@value #DEFAULT_MAX_CACHE_SIZE}. This caching is done at the deephaven layer for faster access to recently read - * fragments. Must be greater than or equal to {@code 1 + readAheadCount()}. - */ - @Default - public int maxCacheSize() { - return Math.max(1 + readAheadCount(), DEFAULT_MAX_CACHE_SIZE); - } - /** * The amount of time to wait when initially establishing a connection before giving up and timing out, defaults to * 2 seconds. @@ -133,8 +121,6 @@ public interface Builder { Builder fragmentSize(int fragmentSize); - Builder maxCacheSize(int maxCacheSize); - Builder connectionTimeout(Duration connectionTimeout); Builder readTimeout(Duration connectionTimeout); @@ -152,13 +138,10 @@ default Builder endpointOverride(String endpointOverride) { abstract S3Instructions withReadAheadCount(int readAheadCount); - abstract S3Instructions withMaxCacheSize(int maxCacheSize); - @Lazy S3Instructions singleUse() { final int readAheadCount = Math.min(DEFAULT_READ_AHEAD_COUNT, readAheadCount()); - return withReadAheadCount(readAheadCount) - .withMaxCacheSize(readAheadCount + 1); + return withReadAheadCount(readAheadCount); } @Check @@ -183,14 +166,6 @@ final void boundsCheckMinFragmentSize() { } } - @Check - final void boundsCheckMaxCacheSize() { - if (maxCacheSize() < readAheadCount() + 1) { - throw new IllegalArgumentException("maxCacheSize(=" + maxCacheSize() + ") must be >= 1 + " + - "readAheadCount(=" + readAheadCount() + ")"); - } - } - @Check final void awsSdkV2Credentials() { if (!(credentials() instanceof AwsSdkV2Credentials)) { diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Request.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Request.java index d031b5c8c11..c5c2727f296 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Request.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3Request.java @@ -36,27 +36,13 @@ * * @implNote This class extends from a {@link SoftReference < ByteBuffer >} and implements {@link CleanupReference} to * allow for cancelling the request once all references to the buffer have been released. Users should not - * access the buffer directly, but instead use the {@link #fill(long, ByteBuffer)} method. Also, users should - * hold instances of {@link AcquiredRequest} to keep the requests alive. + * access the buffer directly, but instead use the {@link Acquired#fill} method. Also, users should hold + * instances of {@link Acquired} to keep the requests alive. */ final class S3Request extends SoftReference implements AsyncResponseTransformer, BiConsumer, CleanupReference { - static class AcquiredRequest { - final S3Request request; - /** - * The ownership token keeps the request alive. When the ownership token is GC'd, the request is no longer - * usable and will be cleaned up. - */ - final Object ownershipToken; - - AcquiredRequest(final S3Request request, final Object ownershipToken) { - this.request = request; - this.ownershipToken = ownershipToken; - } - } - /** * A unique identifier for a request, consisting of the URI and fragment index. */ @@ -108,19 +94,19 @@ public boolean equals(Object obj) { /** * Create a new request for the given fragment index using the provided context object. * - * @return A new {@link AcquiredRequest} object containing newly created request and an ownership token. The request - * will stay alive as long as the ownership token is held. + * @return A new {@link Acquired} object containing newly created request and hard reference to the underlying + * buffer. The request will stay alive as long as the buffer is held. * * @implNote This method does not cache the context because contexts are short-lived while a request may be cached. */ @NotNull - static AcquiredRequest createAndAcquire(final long fragmentIndex, @NotNull final S3ChannelContext context) { + static Acquired createAndAcquire(final long fragmentIndex, @NotNull final S3ChannelContext context) { final long from = fragmentIndex * context.instructions.fragmentSize(); final long to = Math.min(from + context.instructions.fragmentSize(), context.size) - 1; final long requestLength = to - from + 1; final ByteBuffer buffer = ByteBuffer.allocate((int) requestLength); final S3Request request = new S3Request(fragmentIndex, context, buffer, from, to); - return new AcquiredRequest(request, buffer); + return request.acquire(buffer); } private S3Request(final long fragmentIndex, @NotNull final S3ChannelContext context, @@ -146,22 +132,28 @@ ID getId() { } /** - * Try to acquire a reference to this request and ownership token. Returns {@code null} if the token is already + * Try to acquire a reference to this request and underlying buffer. Returns {@code null} if the buffer is already * released. */ @Nullable - AcquiredRequest tryAcquire() { - final Object token = get(); - if (token == null) { + Acquired tryAcquire() { + final ByteBuffer acquiredBuffer = get(); + if (acquiredBuffer == null) { return null; } - return new AcquiredRequest(this, token); + return acquire(acquiredBuffer); + } + + private Acquired acquire(final ByteBuffer buffer) { + return new Acquired(buffer); } /** - * Send the request to the S3 service. This method is idempotent and can be called multiple times. + * Send the request to the S3 service. This method is idempotent and can be called multiple times. Note that the + * request must be acquired before calling this method. Therefore, this method should only be called from inside the + * {@link Acquired#send()} method. */ - void sendRequest() { + private void sendImpl() { if (consumerFuture == null) { synchronized (this) { if (consumerFuture == null) { @@ -176,39 +168,82 @@ void sendRequest() { } } - boolean isDone() { - return consumerFuture.isDone(); - } + class Acquired { + /** + * This instance keeps a hard reference to the buffer, which is needed to keep the request alive. When the + * buffer is GC'd, the request is no longer usable and will be available for {@link #cleanup()}. + */ + private final ByteBuffer acquiredBuffer; - /** - * Fill the provided buffer with data from this request, starting at the given local position. Returns the number of - * bytes filled. Note that the request must be acquired before calling this method. - */ - int fill(long localPosition, ByteBuffer dest) throws IOException { - if (get() == null) { - throw new IllegalStateException(String.format("Trying to fill data after release, %s", requestStr())); + private Acquired(final ByteBuffer buffer) { + this.acquiredBuffer = buffer; + } + + /** + * Are we done fetching the data for this request? + */ + boolean isDone() { + return consumerFuture != null && consumerFuture.isDone(); + } + + /** + * Send the request to the S3 service. This method is idempotent and can be called multiple times. + */ + void send() { + sendImpl(); } - final int resultOffset = (int) (localPosition - from); - final int resultLength = Math.min((int) (to - localPosition + 1), dest.remaining()); - final ByteBuffer fullFragment; - try { - fullFragment = getFullFragment().asReadOnlyBuffer(); - } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { - throw S3ChannelContext.handleS3Exception(e, String.format("fetching fragment %s", requestStr()), - instructions); + + final S3Request request() { + return S3Request.this; } - // fullFragment has limit == capacity. This lets us have safety around math and the ability to simply - // clear to reset. - fullFragment.limit(resultOffset + resultLength); - fullFragment.position(resultOffset); - try { - dest.put(fullFragment); - } finally { - fullFragment.clear(); + + /** + * Fill the provided buffer with data from this request, starting at the given local position. Returns the + * number of bytes filled. + */ + int fill(long localPosition, ByteBuffer dest) throws IOException { + final int resultOffset = (int) (localPosition - from); + final int resultLength = Math.min((int) (to - localPosition + 1), dest.remaining()); + final ByteBuffer filledBuffer = getFilledBuffer(); + // filledBuffer has limit == capacity. This lets us have safety around math and the ability to simply + // clear to reset. + filledBuffer.limit(resultOffset + resultLength); + filledBuffer.position(resultOffset); + try { + dest.put(filledBuffer); + } finally { + filledBuffer.clear(); + } + ++fillCount; + fillBytes += resultLength; + return resultLength; + } + + private ByteBuffer getFilledBuffer() throws IOException { + // Giving our own get() a bit of overhead - the clients should already be constructed with appropriate + // apiCallTimeout. + final long readNanos = instructions.readTimeout().plusMillis(100).toNanos(); + final Boolean isComplete; + try { + isComplete = consumerFuture.get(readNanos, TimeUnit.NANOSECONDS); + } catch (final InterruptedException | ExecutionException | TimeoutException | CancellationException e) { + throw S3ChannelContext.handleS3Exception(e, String.format("fetching fragment %s", requestStr()), + instructions); + } + if (!Boolean.TRUE.equals(isComplete)) { + throw new IllegalStateException(String.format("Failed to complete request %s, probably because the " + + "underlying buffer got freed while completing the request", requestStr())); + } + final ByteBuffer filledBuffer = acquiredBuffer.asReadOnlyBuffer(); + if (filledBuffer.position() != 0 || filledBuffer.limit() != filledBuffer.capacity() + || filledBuffer.limit() != requestLength()) { + throw new IllegalStateException(String.format( + "Expected: pos=0, limit=%d, capacity=%d. Actual: pos=%d, limit=%d, capacity=%d", + requestLength(), requestLength(), filledBuffer.position(), filledBuffer.limit(), + filledBuffer.capacity())); + } + return filledBuffer; } - ++fillCount; - fillBytes += resultLength; - return resultLength; } @Override @@ -268,31 +303,6 @@ public void exceptionOccurred(Throwable error) { // -------------------------------------------------------------------------------------------------- - private ByteBuffer getFullFragment() throws ExecutionException, InterruptedException, TimeoutException { - // Giving our own get() a bit of overhead - the clients should already be constructed with appropriate - // apiCallTimeout. - final long readNanos = instructions.readTimeout().plusMillis(100).toNanos(); - final Boolean isComplete = consumerFuture.get(readNanos, TimeUnit.NANOSECONDS); - if (!Boolean.TRUE.equals(isComplete)) { - throw new IllegalStateException(String.format("Failed to complete request %s", requestStr())); - } - final ByteBuffer result = get(); - if (result == null) { - throw new IllegalStateException( - String.format("Tried to read from no-longer-acquired Request, %s", requestStr())); - } - if (result.position() != 0 || result.limit() != result.capacity() || result.limit() != requestLength()) { - throw new IllegalStateException(String.format( - "Expected: pos=0, limit=%d, capacity=%d. Actual: pos=%d, limit=%d, capacity=%d", - requestLength(), requestLength(), result.position(), result.limit(), result.capacity())); - } - return result; - } - - boolean isFragment(final long fragmentIndex) { - return this.fragmentIndex == fragmentIndex; - } - private int requestLength() { return (int) (to - from + 1); } @@ -358,6 +368,7 @@ public void onNext(final ByteBuffer dataBuffer) { final ByteBuffer resultBuffer = S3Request.this.get(); if (resultBuffer == null) { localProducer.complete(false); + subscription.cancel(); return; } final int numBytes = dataBuffer.remaining(); diff --git a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3RequestCache.java b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3RequestCache.java index 9f3fddd33a8..81a757f83f3 100644 --- a/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3RequestCache.java +++ b/extensions/s3/src/main/java/io/deephaven/extensions/s3/S3RequestCache.java @@ -8,6 +8,7 @@ import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import software.amazon.awssdk.services.s3.S3Uri; /** @@ -46,6 +47,27 @@ int getFragmentSize() { return fragmentSize; } + /** + * Acquire a request for the given URI and fragment index if it already exists in the cache. + * + * @param uri the URI + * @param fragmentIndex the fragment index + * @return the request if we could acquire it from the cache, or null + */ + @Nullable + S3Request.Acquired getRequest(@NotNull final S3Uri uri, final long fragmentIndex) { + final S3Request.ID key = new S3Request.ID(uri, fragmentIndex); + final S3Request existingRequest = requests.get(key); + if (existingRequest != null) { + final S3Request.Acquired acquired = existingRequest.tryAcquire(); + if (acquired != null) { + return acquired; + } + remove(existingRequest); + } + return null; + } + /** * Acquire a request for the given URI and fragment index, creating and sending a new request it if it does not * exist in the cache. @@ -56,29 +78,30 @@ int getFragmentSize() { * @return the request */ @NotNull - S3Request.AcquiredRequest getOrCreateRequest(@NotNull final S3Uri uri, final long fragmentIndex, + S3Request.Acquired getOrCreateRequest(@NotNull final S3Uri uri, final long fragmentIndex, @NotNull final S3ChannelContext context) { final S3Request.ID key = new S3Request.ID(uri, fragmentIndex); - S3Request.AcquiredRequest newAcquiredRequest = null; + S3Request.Acquired newAcquired = null; S3Request existingRequest = requests.get(key); while (true) { if (existingRequest != null) { - final S3Request.AcquiredRequest acquired = existingRequest.tryAcquire(); + final S3Request.Acquired acquired = existingRequest.tryAcquire(); if (acquired != null) { return acquired; } else { remove(existingRequest); } } - if (newAcquiredRequest == null) { - newAcquiredRequest = S3Request.createAndAcquire(fragmentIndex, context); + if (newAcquired == null) { + newAcquired = S3Request.createAndAcquire(fragmentIndex, context); } - if ((existingRequest = requests.putIfAbsent(key, newAcquiredRequest.request)) == null) { + if ((existingRequest = requests.putIfAbsent(key, newAcquired.request())) == null) { if (log.isDebugEnabled()) { log.debug().append("Added new request to cache: ").append(String.format("ctx=%d ", - System.identityHashCode(context))).append(newAcquiredRequest.request.requestStr()).endl(); + System.identityHashCode(context))).append(newAcquired.request().requestStr()) + .endl(); } - return newAcquiredRequest; + return newAcquired; } // TODO(deephaven-core#5486): Instead of remove + putIfAbsent pattern, we could have used replace + get // pattern, but KeyedObjectHashMap potentially has a bug in replace method. diff --git a/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java b/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java index ef9e70400a0..521bc02f6be 100644 --- a/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java +++ b/extensions/s3/src/test/java/io/deephaven/extensions/s3/S3InstructionsTest.java @@ -19,7 +19,6 @@ void defaults() { assertThat(instructions.maxConcurrentRequests()).isEqualTo(256); assertThat(instructions.readAheadCount()).isEqualTo(32); assertThat(instructions.fragmentSize()).isEqualTo(65536); - assertThat(instructions.maxCacheSize()).isEqualTo(256); assertThat(instructions.connectionTimeout()).isEqualTo(Duration.ofSeconds(2)); assertThat(instructions.readTimeout()).isEqualTo(Duration.ofSeconds(2)); assertThat(instructions.credentials()).isEqualTo(Credentials.defaultCredentials()); @@ -102,30 +101,6 @@ void tooSmallFragmentSize() { } } - @Test - void minMaxCacheSize() { - assertThat(S3Instructions.builder() - .regionName("some-region") - .readAheadCount(99) - .maxCacheSize(100) - .build() - .maxCacheSize()) - .isEqualTo(100); - } - - @Test - void tooSmallCacheSize() { - try { - S3Instructions.builder() - .regionName("some-region") - .readAheadCount(99) - .maxCacheSize(99) - .build(); - } catch (IllegalArgumentException e) { - assertThat(e).hasMessageContaining("maxCacheSize"); - } - } - @Test void basicCredentials() { assertThat(S3Instructions.builder() diff --git a/py/server/deephaven/experimental/s3.py b/py/server/deephaven/experimental/s3.py index 7065d519f5c..c19a381b4d0 100644 --- a/py/server/deephaven/experimental/s3.py +++ b/py/server/deephaven/experimental/s3.py @@ -38,7 +38,6 @@ def __init__(self, max_concurrent_requests: Optional[int] = None, read_ahead_count: Optional[int] = None, fragment_size: Optional[int] = None, - max_cache_size: Optional[int] = None, connection_timeout: Union[ Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta, None] = None, read_timeout: Union[ @@ -61,8 +60,6 @@ def __init__(self, fragment. Defaults to 32, which means fetch the next 32 fragments in advance when reading the current fragment. fragment_size (int): the maximum size of each fragment to read, defaults to 64 KiB. If there are fewer bytes remaining in the file, the fetched fragment can be smaller. - max_cache_size (int): the maximum number of fragments to cache in memory while reading, defaults to 256. This - caching is done at the Deephaven layer for faster access to recently read fragments. connection_timeout (Union[Duration, int, str, datetime.timedelta, np.timedelta64, pd.Timedelta]): the amount of time to wait when initially establishing a connection before giving up and timing out, can be expressed as an integer in nanoseconds, a time interval string, e.g. "PT00:00:00.001" or "PT1s", or @@ -103,9 +100,6 @@ def __init__(self, if fragment_size is not None: builder.fragmentSize(fragment_size) - if max_cache_size is not None: - builder.maxCacheSize(max_cache_size) - if connection_timeout is not None: builder.connectionTimeout(time.to_j_duration(connection_timeout)) From ffec8b7e03db63773d288553e9ac129c71898cfb Mon Sep 17 00:00:00 2001 From: Jianfeng Mao <4297243+jmao-denver@users.noreply.github.com> Date: Fri, 21 Jun 2024 18:44:52 -0600 Subject: [PATCH 06/18] Validate arg type for sort/partition_by (#5652) --- py/server/deephaven/table.py | 22 +++++++++++++++------- py/server/tests/test_parquet.py | 2 +- py/server/tests/test_table.py | 10 ++++++++++ 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/py/server/deephaven/table.py b/py/server/deephaven/table.py index 2eca7ae3417..df6cc4b3e93 100644 --- a/py/server/deephaven/table.py +++ b/py/server/deephaven/table.py @@ -1102,9 +1102,12 @@ def sort(self, order_by: Union[str, Sequence[str]], order_by = to_sequence(order_by) if not order: order = (SortDirection.ASCENDING,) * len(order_by) - order = to_sequence(order) - if len(order_by) != len(order): - raise DHError(message="The number of sort columns must be the same as the number of sort directions.") + else: + order = to_sequence(order) + if any([o not in (SortDirection.ASCENDING, SortDirection.DESCENDING) for o in order]): + raise DHError(message="The sort direction must be either 'ASCENDING' or 'DESCENDING'.") + if len(order_by) != len(order): + raise DHError(message="The number of sort columns must be the same as the number of sort directions.") sort_columns = [_sort_column(col, dir_) for col, dir_ in zip(order_by, order)] j_sc_list = j_array_list(sort_columns) @@ -2008,6 +2011,9 @@ def partition_by(self, by: Union[str, Sequence[str]], drop_keys: bool = False) - DHError """ try: + if not isinstance(drop_keys, bool): + raise DHError(message="drop_keys must be a boolean value.") + by = to_sequence(by) return PartitionedTable(j_partitioned_table=self.j_table.partitionBy(drop_keys, *by)) except Exception as e: @@ -2737,12 +2743,14 @@ def sort(self, order_by: Union[str, Sequence[str]], DHError """ try: - order_by = to_sequence(order_by) if not order: order = (SortDirection.ASCENDING,) * len(order_by) - order = to_sequence(order) - if len(order_by) != len(order): - raise ValueError("The number of sort columns must be the same as the number of sort directions.") + else: + order = to_sequence(order) + if any([o not in (SortDirection.ASCENDING, SortDirection.DESCENDING) for o in order]): + raise DHError(message="The sort direction must be either 'ASCENDING' or 'DESCENDING'.") + if len(order_by) != len(order): + raise DHError(message="The number of sort columns must be the same as the number of sort directions.") sort_columns = [_sort_column(col, dir_) for col, dir_ in zip(order_by, order)] j_sc_list = j_array_list(sort_columns) diff --git a/py/server/tests/test_parquet.py b/py/server/tests/test_parquet.py index 612d9bbc759..2d49f7c82cd 100644 --- a/py/server/tests/test_parquet.py +++ b/py/server/tests/test_parquet.py @@ -658,7 +658,7 @@ def verify_table_from_disk(table): self.assertTrue(len(table.columns)) self.assertTrue(table.columns[0].name == "X") self.assertTrue(table.columns[0].column_type == ColumnType.PARTITIONING) - self.assert_table_equals(table.select().sort("X", "Y"), source.sort("X", "Y")) + self.assert_table_equals(table.select().sort(["X", "Y"]), source.sort(["X", "Y"])) def verify_file_names(): partition_dir_path = os.path.join(root_dir, 'X=Aa') diff --git a/py/server/tests/test_table.py b/py/server/tests/test_table.py index 5e30966d906..c9abe604a08 100644 --- a/py/server/tests/test_table.py +++ b/py/server/tests/test_table.py @@ -1105,6 +1105,16 @@ def my_fn(vals): t = partitioned_by_formula() self.assertIsNotNone(t) + def test_arg_validation(self): + t = empty_table(1).update(["A=i", "B=i", "C=i"]) + with self.assertRaises(DHError) as cm: + t.sort("A", "B") + self.assertIn("The sort direction must be", str(cm.exception)) + + with self.assertRaises(DHError) as cm: + t.partition_by("A", "B") + self.assertIn("drop_keys must be", str(cm.exception)) + if __name__ == "__main__": unittest.main() From 5d1c5196b373f4ecff50bcf117e3399f2b78e62f Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Fri, 21 Jun 2024 17:55:28 -0700 Subject: [PATCH 07/18] Fix getNextRowSequenceWithLength callsites that close (#5653) --- .../engine/table/impl/util/ChunkUtils.java | 7 +-- .../deephaven/engine/util/TickSuppressor.java | 56 +++++++++---------- 2 files changed, 30 insertions(+), 33 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ChunkUtils.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ChunkUtils.java index ce429aad6ad..412a61dbf47 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ChunkUtils.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ChunkUtils.java @@ -471,7 +471,7 @@ public static void copyData(ChunkSource.WithPrev[] sources, Ro } public static void fillWithNullValue(ChunkSink dest, RowSequence allKeys) { - final int minSize = Math.min(allKeys.intSize(), COPY_DATA_CHUNK_SIZE); + final int minSize = (int) Math.min(allKeys.size(), COPY_DATA_CHUNK_SIZE); if (minSize == 0) { return; } @@ -480,9 +480,8 @@ public static void fillWithNullValue(ChunkSink dest, RowSe final RowSequence.Iterator iter = allKeys.getRowSequenceIterator()) { chunk.fillWithNullValue(0, minSize); while (iter.hasMore()) { - try (final RowSequence nextKeys = iter.getNextRowSequenceWithLength(COPY_DATA_CHUNK_SIZE)) { - dest.fillFromChunk(destContext, chunk, nextKeys); - } + final RowSequence nextKeys = iter.getNextRowSequenceWithLength(COPY_DATA_CHUNK_SIZE); + dest.fillFromChunk(destContext, chunk, nextKeys); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/util/TickSuppressor.java b/engine/table/src/main/java/io/deephaven/engine/util/TickSuppressor.java index ec50df8eb30..156a4b66478 100644 --- a/engine/table/src/main/java/io/deephaven/engine/util/TickSuppressor.java +++ b/engine/table/src/main/java/io/deephaven/engine/util/TickSuppressor.java @@ -172,38 +172,36 @@ public void onUpdate(TableUpdate upstream) { } while (postRsIt.hasMore()) { - try (final RowSequence postChunkOk = postRsIt.getNextRowSequenceWithLength(chunkSize); - final RowSequence preChunkOk = - preRsIt.getNextRowSequenceWithLength(chunkSize)) { - currentSharedContext.reset(); - prevSharedContext.reset(); + final RowSequence postChunkOk = postRsIt.getNextRowSequenceWithLength(chunkSize); + final RowSequence preChunkOk = preRsIt.getNextRowSequenceWithLength(chunkSize); + currentSharedContext.reset(); + prevSharedContext.reset(); + + for (final int cc : changedColumnIndices) { + // noinspection unchecked + final Chunk currentValues = + inputSources[cc].getChunk(getContextArray[cc], postChunkOk); + // noinspection unchecked + final Chunk prevValues = + inputSources[cc].getPrevChunk(prevContextArray[cc], preChunkOk); + + // now we need to compare them + equalityKernel[cc].notEqual(currentValues, prevValues, changedCellsArray[cc]); + } + final MutableInt pos = new MutableInt(0); + postChunkOk.forAllRowKeys((idx) -> { + boolean idxChanged = false; for (final int cc : changedColumnIndices) { - // noinspection unchecked - final Chunk currentValues = - inputSources[cc].getChunk(getContextArray[cc], postChunkOk); - // noinspection unchecked - final Chunk prevValues = - inputSources[cc].getPrevChunk(prevContextArray[cc], preChunkOk); - - // now we need to compare them - equalityKernel[cc].notEqual(currentValues, prevValues, changedCellsArray[cc]); - } - - final MutableInt pos = new MutableInt(0); - postChunkOk.forAllRowKeys((idx) -> { - boolean idxChanged = false; - for (final int cc : changedColumnIndices) { - if (changedCellsArray[cc].get(pos.get())) { - idxChanged = changedColumns[cc] = true; - } - } - if (idxChanged) { - builder.appendKey(idx); + if (changedCellsArray[cc].get(pos.get())) { + idxChanged = changedColumns[cc] = true; } - pos.increment(); - }); - } + } + if (idxChanged) { + builder.appendKey(idx); + } + pos.increment(); + }); } } From 7ec5131ee51ef522f2ebf64a7eda80c2fbe3a5ad Mon Sep 17 00:00:00 2001 From: Chip Kent <5250374+chipkent@users.noreply.github.com> Date: Mon, 24 Jun 2024 13:33:36 -0600 Subject: [PATCH 08/18] feat!: Remove deprecated Daylight Savings Time (DST) methods (#5266) BREAKING CHANGE: Remove deprecated Daylight Savings Time (DST) methods Follow on to #5245 in 0.34.0 --- .../java/io/deephaven/time/DateTimeUtils.java | 193 ------------------ 1 file changed, 193 deletions(-) diff --git a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java index ce572dd30fc..36824bea4b7 100644 --- a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java +++ b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java @@ -2931,199 +2931,6 @@ public static int minuteOfHour(@Nullable final ZonedDateTime dateTime) { return dateTime.getMinute(); } - /////////////////////////////////////////////////////////////////////////////////////////////////// - // ↓↓↓↓↓↓↓ THE METHODS BELOW ARE DEPRECATED AND WILL BE REMOVED SOON ↓↓↓↓↓↓↓ - /////////////////////////////////////////////////////////////////////////////////////////////////// - - /** - * Returns the number of nanoseconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param instant time - * @param timeZone time zone - * @return {@link QueryConstants#NULL_LONG} if either input is {@code null}; otherwise, number of nanoseconds that - * have elapsed since the top of the day - * @deprecated Use {@link #nanosOfDay(Instant, ZoneId, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static long nanosOfDay(@Nullable final Instant instant, @Nullable final ZoneId timeZone) { - return nanosOfDay(instant, timeZone, false); - } - - /** - * Returns the number of nanoseconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param dateTime time - * @return {@link QueryConstants#NULL_LONG} if either input is {@code null}; otherwise, number of nanoseconds that - * have elapsed since the top of the day - * @deprecated Use {@link #nanosOfDay(ZonedDateTime, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static long nanosOfDay(@Nullable final ZonedDateTime dateTime) { - return nanosOfDay(dateTime, false); - } - - /** - * Returns the number of milliseconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param instant time - * @param timeZone time zone - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of milliseconds that - * have elapsed since the top of the day - * @deprecated Use {@link #millisOfDay(Instant, ZoneId, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int millisOfDay(@Nullable final Instant instant, @Nullable final ZoneId timeZone) { - return millisOfDay(instant, timeZone, false); - } - - /** - * Returns the number of milliseconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param dateTime time - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of milliseconds that - * have elapsed since the top of the day - * @deprecated Use {@link #millisOfDay(ZonedDateTime, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int millisOfDay(@Nullable final ZonedDateTime dateTime) { - return millisOfDay(dateTime, false); - } - - /** - * Returns the number of seconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param instant time - * @param timeZone time zone - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of seconds that have - * elapsed since the top of the day - * @deprecated Use {@link #secondOfDay(Instant, ZoneId, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int secondOfDay(@Nullable final Instant instant, @Nullable final ZoneId timeZone) { - return secondOfDay(instant, timeZone, false); - } - - /** - * Returns the number of seconds that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param dateTime time - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of seconds that have - * elapsed since the top of the day - * @deprecated Use {@link #secondOfDay(ZonedDateTime, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int secondOfDay(@Nullable final ZonedDateTime dateTime) { - return secondOfDay(dateTime, false); - } - - /** - * Returns the number of minutes that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param instant time - * @param timeZone time zone - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of minutes that have - * elapsed since the top of the day - * @deprecated Use {@link #minuteOfDay(Instant, ZoneId, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int minuteOfDay(@Nullable final Instant instant, @Nullable final ZoneId timeZone) { - return minuteOfDay(instant, timeZone, false); - } - - /** - * Returns the number of minutes that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param dateTime time - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of minutes that have - * elapsed since the top of the day - * @deprecated Use {@link #minuteOfDay(ZonedDateTime, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int minuteOfDay(@Nullable final ZonedDateTime dateTime) { - return minuteOfDay(dateTime, false); - } - - /** - * Returns the number of hours that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param instant time - * @param timeZone time zone - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of hours that have - * elapsed since the top of the day - * @deprecated Use {@link #hourOfDay(Instant, ZoneId, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int hourOfDay(@Nullable final Instant instant, @Nullable final ZoneId timeZone) { - return hourOfDay(instant, timeZone, false); - } - - /** - * Returns the number of hours that have elapsed since the top of the day. - *

- * On days when daylight savings time events occur, results may be different from what is expected based upon the - * local time. For example, on daylight savings time change days, 9:30AM may be earlier or later in the day based - * upon if the daylight savings time adjustment is forwards or backwards. - * - * @param dateTime time - * @return {@link QueryConstants#NULL_INT} if either input is {@code null}; otherwise, number of hours that have - * elapsed since the top of the day - * @deprecated Use {@link #hourOfDay(ZonedDateTime, boolean)} instead. To be removed soon. - */ - @Deprecated - @ScriptApi - public static int hourOfDay(@Nullable final ZonedDateTime dateTime) { - return hourOfDay(dateTime, false); - } - - /////////////////////////////////////////////////////////////////////////////////////////////////// - // ↑↑↑↑↑↑↑ THE METHODS ABOVE ARE DEPRECATED AND WILL BE REMOVED SOON ↑↑↑↑↑↑↑ - /////////////////////////////////////////////////////////////////////////////////////////////////// - /** * Returns the number of nanoseconds that have elapsed since the start of the day. * From a13a0dc535398a948757c1ccc36d5c5115c9dc7c Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 24 Jun 2024 12:51:21 -0700 Subject: [PATCH 09/18] feat: expose Iceberg features to python users (#5590) * WIP commit, functional but needs docs. * Added AWS Glue functionality. * Update Iceberg and AWS versions, itemize aws dependencies. * Documentation improvements. --- buildSrc/src/main/groovy/Classpaths.groovy | 4 +- extensions/iceberg/s3/build.gradle | 4 +- .../iceberg/util/IcebergToolsS3.java | 48 +++- .../iceberg/util/IcebergToolsTest.java | 61 ++++- .../iceberg/util/IcebergCatalogAdapter.java | 108 ++++++-- .../iceberg/util/IcebergInstructions.java | 7 + py/server/deephaven/experimental/iceberg.py | 252 ++++++++++++++++++ py/server/deephaven/jcompat.py | 31 +++ py/server/deephaven/parquet.py | 22 +- py/server/deephaven/stream/table_publisher.py | 13 +- py/server/tests/test_iceberg.py | 76 ++++++ 11 files changed, 562 insertions(+), 64 deletions(-) create mode 100644 py/server/deephaven/experimental/iceberg.py create mode 100644 py/server/tests/test_iceberg.py diff --git a/buildSrc/src/main/groovy/Classpaths.groovy b/buildSrc/src/main/groovy/Classpaths.groovy index 86c342bb0c7..df2877db0f0 100644 --- a/buildSrc/src/main/groovy/Classpaths.groovy +++ b/buildSrc/src/main/groovy/Classpaths.groovy @@ -126,10 +126,10 @@ class Classpaths { static final String HADOOP_VERSION = '3.4.0' static final String ICEBERG_GROUP = 'org.apache.iceberg' - static final String ICEBERG_VERSION = '1.5.0' + static final String ICEBERG_VERSION = '1.5.2' static final String AWSSDK_GROUP = 'software.amazon.awssdk' - static final String AWSSDK_VERSION = '2.23.19' + static final String AWSSDK_VERSION = '2.24.5' static final String TESTCONTAINER_GROUP = 'org.testcontainers' static final String TESTCONTAINER_VERSION = '1.19.4' diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle index 359651ec7e6..be495b1373d 100644 --- a/extensions/iceberg/s3/build.gradle +++ b/extensions/iceberg/s3/build.gradle @@ -16,8 +16,10 @@ dependencies { implementation project(':extensions-s3') implementation "org.apache.iceberg:iceberg-aws" - runtimeOnly "org.apache.iceberg:iceberg-aws-bundle" + Classpaths.inheritAWSSDK(project) + runtimeOnly "software.amazon.awssdk:sts" + runtimeOnly "software.amazon.awssdk:glue" Classpaths.inheritTestContainers(project) diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java index 6f7845c43eb..166b47e5d28 100644 --- a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java +++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java @@ -7,6 +7,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; +import org.apache.iceberg.aws.glue.GlueCatalog; import org.apache.iceberg.aws.s3.S3FileIOProperties; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.rest.RESTCatalog; @@ -23,6 +24,20 @@ public class IcebergToolsS3 extends IcebergTools { private static final String S3_FILE_IO_CLASS = "org.apache.iceberg.aws.s3.S3FileIO"; + /** + * Create an Iceberg catalog adapter for a REST catalog backed by S3 storage. If {@code null} is provided for a + * value, the system defaults will be used. + * + * @param name the name of the catalog; if omitted, the catalog URI will be used to generate a name + * @param catalogURI the URI of the Iceberg REST catalog + * @param warehouseLocation the location of the S3 datafiles backing the catalog + * @param region the AWS region; if omitted, system defaults will be used + * @param accessKeyId the AWS access key ID; if omitted, system defaults will be used + * @param secretAccessKey the AWS secret access key; if omitted, system defaults will be used + * @param endpointOverride the S3 endpoint override; this is useful for testing with a S3-compatible local service + * such as MinIO or LocalStack + * @return the Iceberg catalog adapter + */ public static IcebergCatalogAdapter createS3Rest( @Nullable final String name, @NotNull final String catalogURI, @@ -53,7 +68,6 @@ public static IcebergCatalogAdapter createS3Rest( properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } - // TODO: create a FileIO interface wrapping the Deephaven S3SeekableByteChannel/Provider final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; @@ -62,4 +76,36 @@ public static IcebergCatalogAdapter createS3Rest( return new IcebergCatalogAdapter(catalog, fileIO); } + /** + * Create an Iceberg catalog adapter for an AWS Glue catalog. System defaults will be used to populate the region + * and credentials. These can be configured by following + * AWS Authentication and + * access credentials guide. + * + * @param name the name of the catalog; if omitted, the catalog URI will be used to generate a name + * @param catalogURI the URI of the AWS Glue catalog + * @param warehouseLocation the location of the S3 datafiles backing the catalog + * @return the Iceberg catalog adapter + */ + public static IcebergCatalogAdapter createGlue( + @Nullable final String name, + @NotNull final String catalogURI, + @NotNull final String warehouseLocation) { + + // Set up the properties map for the Iceberg catalog + final Map properties = new HashMap<>(); + + final GlueCatalog catalog = new GlueCatalog(); + + properties.put(CatalogProperties.CATALOG_IMPL, catalog.getClass().getName()); + properties.put(CatalogProperties.URI, catalogURI); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); + + final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.initialize(catalogName, properties); + + return new IcebergCatalogAdapter(catalog, fileIO); + } } diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 0fd3b3fcf7e..7544976f27b 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -129,19 +129,23 @@ public void testListTables() { final Namespace ns = Namespace.of("sales"); - final Collection tables = adapter.listTables(ns); + Collection tables = adapter.listTables(ns); Assert.eq(tables.size(), "tables.size()", 3, "3 tables in the namespace"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_multi")), "tables.contains(sales_multi)"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_partitioned")), "tables.contains(sales_partitioned)"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)"); - final Table table = adapter.listTablesAsTable(ns); + Table table = adapter.listTablesAsTable(ns); Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); Assert.eqTrue(table.getColumnSource("table_name").getType().equals(String.class), "table_name column type"); Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), "table_identifier_object column type"); + + // Test the string versions of the methods + table = adapter.listTablesAsTable("sales"); + Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); } @Test @@ -160,7 +164,7 @@ public void testListSnapshots() { Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); - final Table table = adapter.listSnapshotsAsTable(tableIdentifier); + Table table = adapter.listSnapshotsAsTable(tableIdentifier); Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); Assert.eqTrue(table.getColumnSource("id").getType().equals(long.class), "id column type"); Assert.eqTrue(table.getColumnSource("timestamp_ms").getType().equals(long.class), "timestamp_ms column type"); @@ -168,6 +172,10 @@ public void testListSnapshots() { Assert.eqTrue(table.getColumnSource("summary").getType().equals(Map.class), "summary column type"); Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), "snapshot_object column type"); + + // Test the string versions of the methods + table = adapter.listSnapshotsAsTable("sales.sales_multi"); + Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); } @Test @@ -180,7 +188,13 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + + // Test the string versions of the methods + table = adapter.readTable("sales.sales_partitioned", instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -196,9 +210,15 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + + // Test the string versions of the methods + table = adapter.readTable("sales.sales_multi", instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); } @Test @@ -211,7 +231,13 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + + // Verify we retrieved all the rows. + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + + // Test the string versions of the methods + table = adapter.readTable("sales.sales_single", instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); @@ -563,16 +589,31 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx final List snapshots = adapter.listSnapshots(tableId); // Verify we retrieved all the rows. - final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); + io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); + Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + + io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); + Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + + io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); + Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + + io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); + Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + + // Test the string versions of the methods + + // Verify we retrieved all the rows. + table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions); Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); - final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); + table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); - final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); + table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); - final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); + table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index c379c715c6d..c54535599c3 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -55,7 +55,7 @@ public class IcebergCatalogAdapter { /** * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into - * account {@link Map column rename instructions} + * account {@link Map<> column rename instructions} * * @param schema The schema of the table. * @param partitionSpec The partition specification of the table. @@ -197,11 +197,11 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { // Create the column source(s) final String[] namespaceArr = new String[(int) size]; - columnSourceMap.put("namespace", + columnSourceMap.put("Namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); final Namespace[] namespaceObjectArr = new Namespace[(int) size]; - columnSourceMap.put("namespace_object", + columnSourceMap.put("NamespaceObject", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceObjectArr, Namespace.class, null)); // Populate the column source arrays @@ -215,6 +215,16 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + /** + * List all {@link Namespace namespaces} in a given namespace as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listNamespaces(Namespace)}. + * + * @return A {@link Table table} of all namespaces. + */ + public Table listNamespacesAsTable(@NotNull final String... namespace) { + return listNamespacesAsTable(Namespace.of(namespace)); + } + /** * List all Iceberg {@link TableIdentifier tables} in a given namespace. * @@ -241,15 +251,15 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) { // Create the column source(s) final String[] namespaceArr = new String[(int) size]; - columnSourceMap.put("namespace", + columnSourceMap.put("Namespace", InMemoryColumnSource.getImmutableMemoryColumnSource(namespaceArr, String.class, null)); final String[] tableNameArr = new String[(int) size]; - columnSourceMap.put("table_name", + columnSourceMap.put("TableName", InMemoryColumnSource.getImmutableMemoryColumnSource(tableNameArr, String.class, null)); final TableIdentifier[] tableIdentifierArr = new TableIdentifier[(int) size]; - columnSourceMap.put("table_identifier_object", + columnSourceMap.put("TableIdentifierObject", InMemoryColumnSource.getImmutableMemoryColumnSource(tableIdentifierArr, TableIdentifier.class, null)); // Populate the column source arrays @@ -264,6 +274,10 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) { return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + public Table listTablesAsTable(@NotNull final String... namespace) { + return listTablesAsTable(Namespace.of(namespace)); + } + /** * List all {@link Snapshot snapshots} of a given Iceberg table. * @@ -292,22 +306,22 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier // Create the column source(s) final long[] idArr = new long[(int) size]; - columnSourceMap.put("id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); + columnSourceMap.put("Id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); final long[] timestampArr = new long[(int) size]; - columnSourceMap.put("timestamp_ms", + columnSourceMap.put("TimestampMs", InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, long.class, null)); final String[] operatorArr = new String[(int) size]; - columnSourceMap.put("operation", + columnSourceMap.put("Operation", InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null)); final Map[] summaryArr = new Map[(int) size]; - columnSourceMap.put("summary", + columnSourceMap.put("Summary", InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null)); final Snapshot[] snapshotArr = new Snapshot[(int) size]; - columnSourceMap.put("snapshot_object", + columnSourceMap.put("SnapshotObject", InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null)); // Populate the column source(s) @@ -324,6 +338,17 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + /** + * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting + * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all tables in the given namespace. + */ + public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) { + return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier)); + } + /** * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. * @@ -334,10 +359,24 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier @SuppressWarnings("unused") public Table readTable( @NotNull final TableIdentifier tableIdentifier, - @NotNull final IcebergInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, null, instructions); } + /** + * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final String tableIdentifier, + @Nullable final IcebergInstructions instructions) { + return readTable(TableIdentifier.parse(tableIdentifier), instructions); + } + /** * Read a static snapshot of an Iceberg table from the Iceberg catalog. * @@ -350,7 +389,7 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, - @NotNull final IcebergInstructions instructions) { + @Nullable final IcebergInstructions instructions) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = listSnapshots(tableIdentifier).stream() @@ -361,6 +400,22 @@ public Table readTable( return readTableInternal(tableIdentifier, tableSnapshot, instructions); } + /** + * Read a static snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableIdentifier The table identifier to load + * @param tableSnapshotId The snapshot id to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + @SuppressWarnings("unused") + public Table readTable( + @NotNull final String tableIdentifier, + final long tableSnapshotId, + @Nullable final IcebergInstructions instructions) { + return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions); + } + /** * Read a static snapshot of an Iceberg table from the Iceberg catalog. * @@ -373,32 +428,35 @@ public Table readTable( public Table readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, - @NotNull final IcebergInstructions instructions) { + @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, tableSnapshot, instructions); } private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, - @NotNull final IcebergInstructions instructions) { + @Nullable final IcebergInstructions instructions) { - // Load the table from the catalog + // Load the table from the catalog. final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); // Do we want the latest or a specific snapshot? final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); final Schema schema = table.schemas().get(snapshot.schemaId()); - // Load the partitioning schema + // Load the partitioning schema. final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); + // Get default instructions if none are provided + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + // Get the user supplied table definition. - final TableDefinition userTableDef = instructions.tableDefinition().orElse(null); + final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); // Get the table definition from the schema (potentially limited by the user supplied table definition and // applying column renames). final TableDefinition icebergTableDef = - fromSchema(schema, partitionSpec, userTableDef, instructions.columnRenames()); + fromSchema(schema, partitionSpec, userTableDef, userInstructions.columnRenames()); // If the user supplied a table definition, make sure it's fully compatible. final TableDefinition tableDef; @@ -433,11 +491,11 @@ private Table readTableInternal( if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, instructions); + keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, userInstructions); } else { // Create the partitioning column location key finder keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, partitionSpec, - instructions); + userInstructions); } refreshService = null; @@ -459,4 +517,12 @@ private Table readTableInternal( return result; } + + /** + * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. + */ + @SuppressWarnings("unused") + public Catalog catalog() { + return catalog; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 4788e0e8714..b595b4cfd14 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -17,6 +17,13 @@ @Immutable @BuildableStyle public abstract class IcebergInstructions { + /** + * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system + * defaults for cloud provider-specific parameters + */ + @SuppressWarnings("unused") + public static final IcebergInstructions DEFAULT = builder().build(); + public static Builder builder() { return ImmutableIcebergInstructions.builder(); } diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py new file mode 100644 index 00000000000..7506bc95a25 --- /dev/null +++ b/py/server/deephaven/experimental/iceberg.py @@ -0,0 +1,252 @@ +# +# Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +# +""" This module adds Iceberg table support into Deephaven. """ +from typing import List, Optional, Union, Dict, Sequence + +import jpy + +from deephaven import DHError +from deephaven._wrapper import JObjectWrapper +from deephaven.column import Column +from deephaven.dtypes import DType +from deephaven.experimental import s3 + +from deephaven.jcompat import j_table_definition + +from deephaven.table import Table + +_JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions") +_JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") + +# IcebergToolsS3 is an optional library +try: + _JIcebergToolsS3 = jpy.get_type("io.deephaven.iceberg.util.IcebergToolsS3") +except Exception: + _JIcebergToolsS3 = None + +_JNamespace = jpy.get_type("org.apache.iceberg.catalog.Namespace") +_JTableIdentifier = jpy.get_type("org.apache.iceberg.catalog.TableIdentifier") +_JSnapshot = jpy.get_type("org.apache.iceberg.Snapshot") + + +class IcebergInstructions(JObjectWrapper): + """ + This class specifies the instructions for reading an Iceberg table into Deephaven. These include column rename + instructions and table definitions, as well as special data instructions for loading data files from the cloud. + """ + + j_object_type = _JIcebergInstructions + + def __init__(self, + table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None, + data_instructions: Optional[s3.S3Instructions] = None, + column_renames: Optional[Dict[str, str]] = None): + """ + Initializes the instructions using the provided parameters. + + Args: + table_definition (Optional[Union[Dict[str, DType], List[Column], None]]): the table definition; if omitted, + the definition is inferred from the Iceberg schema. Setting a definition guarantees the returned table + will have that definition. This is useful for specifying a subset of the Iceberg schema columns. + data_instructions (Optional[s3.S3Instructions]): Special instructions for reading data files, useful when + reading files from a non-local file system, like S3. + column_renames (Optional[Dict[str, str]]): A dictionary of old to new column names that will be renamed in + the output table. + + Raises: + DHError: If unable to build the instructions object. + """ + + try: + builder = self.j_object_type.builder() + + if table_definition is not None: + builder.tableDefinition(j_table_definition(table_definition)) + + if data_instructions is not None: + builder.dataInstructions(data_instructions.j_object) + + if column_renames is not None: + for old_name, new_name in column_renames.items(): + builder.putColumnRenames(old_name, new_name) + + self._j_object = builder.build() + except Exception as e: + raise DHError(e, "Failed to build Iceberg instructions") from e + + @property + def j_object(self) -> jpy.JType: + return self._j_object + + +class IcebergCatalogAdapter(JObjectWrapper): + """ + This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and + snapshots, as well as reading Iceberg tables into Deephaven tables. + """ + j_object_type = _JIcebergCatalogAdapter or type(None) + + def __init__(self, j_object: _JIcebergCatalogAdapter): + self.j_catalog_adapter = j_object + + def namespaces(self, namespace: Optional[str] = None) -> Table: + """ + Returns information on the namespaces in the catalog as a Deephaven table. If a namespace is specified, the + tables in that namespace are listed; otherwise the top-level namespaces are listed. + + Args: + namespace (Optional[str]): the higher-level namespace from which to list namespaces; if omitted, the + top-level namespaces are listed. + + Returns: + a table containing the namespaces. + """ + + if namespace is not None: + return Table(self.j_object.listNamespaces(namespace)) + return Table(self.j_object.listNamespacesAsTable()) + + def tables(self, namespace: str) -> Table: + """ + Returns information on the tables in the specified namespace as a Deephaven table. + + Args: + namespace (str): the namespace from which to list tables. + + Returns: + a table containing the tables in the provided namespace. + """ + + if namespace is not None: + return Table(self.j_object.listTablesAsTable(namespace)) + return Table(self.j_object.listTablesAsTable()) + + def snapshots(self, table_identifier: str) -> Table: + """ + Returns information on the snapshots of the specified table as a Deephaven table. + + Args: + table_identifier (str): the table from which to list snapshots. + + Returns: + a table containing the snapshot information. + """ + + return self.j_object.listSnapshotsAsTable(table_identifier) + + def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table: + """ + Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to + read a specific snapshot of the table. + + Args: + table_identifier (str): the table to read. + instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions + can include column renames, table definition, and specific data instructions for reading the data files + from the provider. If omitted, the table will be read with default instructions. + snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. + + Returns: + Table: the table read from the catalog. + """ + + if instructions is not None: + instructions_object = instructions.j_object + else: + instructions_object = _JIcebergInstructions.DEFAULT + + if snapshot_id is not None: + return Table(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) + return Table(self.j_object.readTable(table_identifier, instructions_object)) + + @property + def j_object(self) -> jpy.JType: + return self.j_catalog_adapter + + +def adapter_s3_rest( + catalog_uri: str, + warehouse_location: str, + name: Optional[str] = None, + region_name: Optional[str] = None, + access_key_id: Optional[str] = None, + secret_access_key: Optional[str] = None, + end_point_override: Optional[str] = None +) -> IcebergCatalogAdapter: + """ + Create a catalog adapter using an S3-compatible provider and a REST catalog. + + Args: + catalog_uri (str): the URI of the REST catalog. + warehouse_location (str): the location of the warehouse. + name (Optional[str]): a descriptive name of the catalog; if omitted the catalog name is inferred from the + catalog URI. + region_name (Optional[str]): the S3 region name to use; If not provided, the default region will be + picked by the AWS SDK from 'aws.region' system property, "AWS_REGION" environment variable, the + {user.home}/.aws/credentials or {user.home}/.aws/config files, or from EC2 metadata service, if running in + EC2. + access_key_id (Optional[str]): the access key for reading files. Both access key and secret access key must be + provided to use static credentials, else default credentials will be used. + secret_access_key (Optional[str]): the secret access key for reading files. Both access key and secret key + must be provided to use static credentials, else default credentials will be used. + end_point_override (Optional[str]): the S3 endpoint to connect to. Callers connecting to AWS do not typically + need to set this; it is most useful when connecting to non-AWS, S3-compatible APIs. + + Returns: + IcebergCatalogAdapter: the catalog adapter for the provided S3 REST catalog. + + Raises: + DHError: If unable to build the catalog adapter. + """ + if not _JIcebergToolsS3: + raise DHError(message="`adapter_s3_rest` requires the Iceberg specific deephaven S3 extensions to be " + "included in the package") + + try: + return IcebergCatalogAdapter( + _JIcebergToolsS3.createS3Rest( + name, + catalog_uri, + warehouse_location, + region_name, + access_key_id, + secret_access_key, + end_point_override)) + except Exception as e: + raise DHError(e, "Failed to build Iceberg Catalog Adapter") from e + + +def adapter_aws_glue( + catalog_uri: str, + warehouse_location: str, + name: Optional[str] = None +) -> IcebergCatalogAdapter: + """ + Create a catalog adapter using an AWS Glue catalog. + + Args: + catalog_uri (str): the URI of the REST catalog. + warehouse_location (str): the location of the warehouse. + name (Optional[str]): a descriptive name of the catalog; if omitted the catalog name is inferred from the + catalog URI. + + Returns: + IcebergCatalogAdapter: the catalog adapter for the provided AWS Glue catalog. + + Raises: + DHError: If unable to build the catalog adapter. + """ + if not _JIcebergToolsS3: + raise DHError(message="`adapter_aws_glue` requires the Iceberg specific deephaven S3 extensions to " + "be included in the package") + + try: + return IcebergCatalogAdapter( + _JIcebergToolsS3.createGlue( + name, + catalog_uri, + warehouse_location)) + except Exception as e: + raise DHError(e, "Failed to build Iceberg Catalog Adapter") from e + diff --git a/py/server/deephaven/jcompat.py b/py/server/deephaven/jcompat.py index e6e921fd8f1..d807cb472f3 100644 --- a/py/server/deephaven/jcompat.py +++ b/py/server/deephaven/jcompat.py @@ -14,9 +14,11 @@ from deephaven import dtypes, DHError from deephaven._wrapper import unwrap, wrap_j_object, JObjectWrapper from deephaven.dtypes import DType, _PRIMITIVE_DTYPE_NULL_MAP +from deephaven.column import Column _NULL_BOOLEAN_AS_BYTE = jpy.get_type("io.deephaven.util.BooleanUtils").NULL_BOOLEAN_AS_BYTE _JPrimitiveArrayConversionUtility = jpy.get_type("io.deephaven.integrations.common.PrimitiveArrayConversionUtility") +_JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") _DH_PANDAS_NULLABLE_TYPE_MAP: Dict[DType, pd.api.extensions.ExtensionDtype] = { dtypes.bool_: pd.BooleanDtype, @@ -325,6 +327,35 @@ def _j_array_to_series(dtype: DType, j_array: jpy.JType, conv_null: bool) -> pd. return s +def j_table_definition(table_definition: Union[Dict[str, DType], List[Column], None]) -> Optional[jpy.JType]: + """Produce a Deephaven TableDefinition from user input. + + Args: + table_definition (Union[Dict[str, DType], List[Column], None]): the table definition as a dictionary of column + names and their corresponding data types or a list of Column objects + + Returns: + a Deephaven TableDefinition object or None if the input is None + + Raises: + DHError + """ + if table_definition is None: + return None + elif isinstance(table_definition, Dict): + return _JTableDefinition.of( + [ + Column(name=name, data_type=dtype).j_column_definition + for name, dtype in table_definition.items() + ] + ) + elif isinstance(table_definition, List): + return _JTableDefinition.of( + [col.j_column_definition for col in table_definition] + ) + else: + raise DHError(f"Unexpected table_definition type: {type(table_definition)}") + class AutoCloseable(JObjectWrapper): """A context manager wrapper to allow Java AutoCloseable to be used in with statements. diff --git a/py/server/deephaven/parquet.py b/py/server/deephaven/parquet.py index 76c9dd86735..dc877660671 100644 --- a/py/server/deephaven/parquet.py +++ b/py/server/deephaven/parquet.py @@ -13,7 +13,7 @@ from deephaven import DHError from deephaven.column import Column from deephaven.dtypes import DType -from deephaven.jcompat import j_array_list +from deephaven.jcompat import j_array_list, j_table_definition from deephaven.table import Table, PartitionedTable from deephaven.experimental import s3 @@ -135,7 +135,7 @@ def _build_parquet_instructions( builder.setFileLayout(_j_file_layout(file_layout)) if table_definition is not None: - builder.setTableDefinition(_j_table_definition(table_definition)) + builder.setTableDefinition(j_table_definition(table_definition)) if index_columns: builder.addAllIndexColumns(_j_list_of_list_of_string(index_columns)) @@ -146,24 +146,6 @@ def _build_parquet_instructions( return builder.build() -def _j_table_definition(table_definition: Union[Dict[str, DType], List[Column], None]) -> Optional[jpy.JType]: - if table_definition is None: - return None - elif isinstance(table_definition, Dict): - return _JTableDefinition.of( - [ - Column(name=name, data_type=dtype).j_column_definition - for name, dtype in table_definition.items() - ] - ) - elif isinstance(table_definition, List): - return _JTableDefinition.of( - [col.j_column_definition for col in table_definition] - ) - else: - raise DHError(f"Unexpected table_definition type: {type(table_definition)}") - - def _j_file_layout(file_layout: Optional[ParquetFileLayout]) -> Optional[jpy.JType]: if file_layout is None: return None diff --git a/py/server/deephaven/stream/table_publisher.py b/py/server/deephaven/stream/table_publisher.py index cb0d1073de8..a6c65f47885 100644 --- a/py/server/deephaven/stream/table_publisher.py +++ b/py/server/deephaven/stream/table_publisher.py @@ -5,13 +5,13 @@ import jpy -from typing import Callable, Dict, Optional, Tuple +from typing import Callable, Dict, Optional, Tuple, Union, List from deephaven._wrapper import JObjectWrapper from deephaven.column import Column from deephaven.dtypes import DType from deephaven.execution_context import get_exec_ctx -from deephaven.jcompat import j_lambda, j_runnable +from deephaven.jcompat import j_lambda, j_runnable, j_table_definition from deephaven.table import Table from deephaven.update_graph import UpdateGraph @@ -75,7 +75,7 @@ def publish_failure(self, failure: Exception) -> None: def table_publisher( name: str, - col_defs: Dict[str, DType], + col_defs: Union[Dict[str, DType], List[Column]], on_flush_callback: Optional[Callable[[TablePublisher], None]] = None, on_shutdown_callback: Optional[Callable[[], None]] = None, update_graph: Optional[UpdateGraph] = None, @@ -107,12 +107,7 @@ def adapt_callback(_table_publisher: jpy.JType): j_table_publisher = _JTablePublisher.of( name, - _JTableDefinition.of( - [ - Column(name=name, data_type=dtype).j_column_definition - for name, dtype in col_defs.items() - ] - ), + j_table_definition(col_defs), j_lambda(adapt_callback, _JConsumer, None) if on_flush_callback else None, j_runnable(on_shutdown_callback) if on_shutdown_callback else None, (update_graph or get_exec_ctx().update_graph).j_update_graph, diff --git a/py/server/tests/test_iceberg.py b/py/server/tests/test_iceberg.py new file mode 100644 index 00000000000..62ba31e6636 --- /dev/null +++ b/py/server/tests/test_iceberg.py @@ -0,0 +1,76 @@ +# +# Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +# +import jpy + +from deephaven import dtypes +from deephaven.column import Column, ColumnType + +from tests.testbase import BaseTestCase +from deephaven.experimental import s3, iceberg + +from deephaven.jcompat import j_map_to_dict, j_list_to_list + +_JTableDefinition = jpy.get_type("io.deephaven.engine.table.TableDefinition") + +class IcebergTestCase(BaseTestCase): + """ Test cases for the deephaven.iceberg module (performed locally) """ + + def setUp(self): + super().setUp() + + def tearDown(self): + super().tearDown() + + def test_instruction_create_empty(self): + iceberg_instructions = iceberg.IcebergInstructions() + + def test_instruction_create_with_s3_instructions(self): + s3_instructions = s3.S3Instructions(region_name="us-east-1", + access_key_id="some_access_key_id", + secret_access_key="som_secret_access_key" + ) + iceberg_instructions = iceberg.IcebergInstructions(data_instructions=s3_instructions) + + def test_instruction_create_with_col_renames(self): + renames = { + "old_name_a": "new_name_a", + "old_name_b": "new_name_b", + "old_name_c": "new_name_c" + } + iceberg_instructions = iceberg.IcebergInstructions(column_renames=renames) + + col_rename_dict = j_map_to_dict(iceberg_instructions.j_object.columnRenames()) + self.assertTrue(col_rename_dict["old_name_a"] == "new_name_a") + self.assertTrue(col_rename_dict["old_name_b"] == "new_name_b") + self.assertTrue(col_rename_dict["old_name_c"] == "new_name_c") + + def test_instruction_create_with_table_definition_dict(self): + table_def={ + "x": dtypes.int32, + "y": dtypes.double, + "z": dtypes.double, + } + + iceberg_instructions = iceberg.IcebergInstructions(table_definition=table_def) + col_names = j_list_to_list(iceberg_instructions.j_object.tableDefinition().get().getColumnNames()) + self.assertTrue(col_names[0] == "x") + self.assertTrue(col_names[1] == "y") + self.assertTrue(col_names[2] == "z") + + def test_instruction_create_with_table_definition_list(self): + table_def=[ + Column( + "Partition", dtypes.int32, column_type=ColumnType.PARTITIONING + ), + Column("x", dtypes.int32), + Column("y", dtypes.double), + Column("z", dtypes.double), + ] + + iceberg_instructions = iceberg.IcebergInstructions(table_definition=table_def) + col_names = j_list_to_list(iceberg_instructions.j_object.tableDefinition().get().getColumnNames()) + self.assertTrue(col_names[0] == "Partition") + self.assertTrue(col_names[1] == "x") + self.assertTrue(col_names[2] == "y") + self.assertTrue(col_names[3] == "z") From 31a98955fcd93650c400e93a479daf54289c3361 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Mon, 24 Jun 2024 12:56:29 -0700 Subject: [PATCH 10/18] Change py-embedded-server from WARN to INFO logging (#5644) The python embedded Deephaven server does not currently show the end-user any INFO logging. It was stated that the default was WARN so that the end-user doesn't get command line spam, but that is already covered in the setup - the slf4j / logback logging does _not_ hit the terminal regardless. This is only applicable for logging that hits the web UI via LogBufferAppender / ConsoleService.SubscribeToLogs. --- .../java-runtime/src/main/resources/logback-minimal.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/py/embedded-server/java-runtime/src/main/resources/logback-minimal.xml b/py/embedded-server/java-runtime/src/main/resources/logback-minimal.xml index 1922705b272..4b1fadb1e55 100644 --- a/py/embedded-server/java-runtime/src/main/resources/logback-minimal.xml +++ b/py/embedded-server/java-runtime/src/main/resources/logback-minimal.xml @@ -15,7 +15,7 @@ - + From 91d5df5bd779caecd62bf643b5dabcb31cd70e75 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 24 Jun 2024 16:29:49 -0700 Subject: [PATCH 11/18] fix: correct weighted summation null handling behavior (#5660) * Corrects null handling issue and potential data truncation. * Another logic correction and renaming misleading variables. --- .../by/LongChunkedWeightedSumOperator.java | 53 ++++++++++--------- .../table/impl/QueryTableAggregationTest.java | 34 ++++++++++++ 2 files changed, 61 insertions(+), 26 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedWeightedSumOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedWeightedSumOperator.java index 30bd2f0fa52..4a7abc3a6ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedWeightedSumOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedWeightedSumOperator.java @@ -16,6 +16,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.*; import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.mutable.MutableLong; import java.util.Collections; import java.util.Map; @@ -46,12 +47,12 @@ public void addChunk(BucketedContext bucketedContext, Chunk va IntChunk startPositions, IntChunk length, WritableBooleanChunk stateModified) { final Context context = (Context) bucketedContext; - final LongChunk doubleValues = context.toLongCast.apply(values); + final LongChunk longValues = context.toLongCast.apply(values); final LongChunk weightValues = weightOperator.getAddedWeights(); Assert.neqNull(weightValues, "weightValues"); for (int ii = 0; ii < startPositions.size(); ++ii) { final int startPosition = startPositions.get(ii); - stateModified.set(ii, addChunk(doubleValues, weightValues, startPosition, length.get(ii), + stateModified.set(ii, addChunk(longValues, weightValues, startPosition, length.get(ii), destinations.get(startPosition))); } } @@ -62,12 +63,12 @@ public void removeChunk(BucketedContext bucketedContext, Chunk IntChunk startPositions, IntChunk length, WritableBooleanChunk stateModified) { final Context context = (Context) bucketedContext; - final LongChunk doubleValues = context.prevToLongCast.apply(values); + final LongChunk longValues = context.prevToLongCast.apply(values); final LongChunk weightValues = weightOperator.getRemovedWeights(); Assert.neqNull(weightValues, "weightValues"); for (int ii = 0; ii < startPositions.size(); ++ii) { final int startPosition = startPositions.get(ii); - stateModified.set(ii, removeChunk(doubleValues, weightValues, startPosition, length.get(ii), + stateModified.set(ii, removeChunk(longValues, weightValues, startPosition, length.get(ii), destinations.get(startPosition))); } } @@ -93,18 +94,18 @@ public void modifyChunk(BucketedContext bucketedContext, Chunk public boolean addChunk(SingletonContext singletonContext, int chunkSize, Chunk values, LongChunk inputRowKeys, long destination) { final Context context = (Context) singletonContext; - final LongChunk doubleValues = context.toLongCast.apply(values); + final LongChunk longValues = context.toLongCast.apply(values); final LongChunk weightValues = weightOperator.getAddedWeights(); - return addChunk(doubleValues, weightValues, 0, values.size(), destination); + return addChunk(longValues, weightValues, 0, values.size(), destination); } @Override public boolean removeChunk(SingletonContext singletonContext, int chunkSize, Chunk values, LongChunk inputRowKeys, long destination) { final Context context = (Context) singletonContext; - final LongChunk doubleValues = context.prevToLongCast.apply(values); + final LongChunk longValues = context.prevToLongCast.apply(values); final LongChunk weightValues = weightOperator.getRemovedWeights(); - return removeChunk(doubleValues, weightValues, 0, values.size(), destination); + return removeChunk(longValues, weightValues, 0, values.size(), destination); } @Override @@ -121,19 +122,19 @@ public boolean modifyChunk(SingletonContext singletonContext, int chunkSize, Chu newDoubleValues.size(), destination); } - private static void sumChunks(LongChunk doubleValues, LongChunk weightValues, + private static void sumChunks(LongChunk longValues, LongChunk weightValues, int start, int length, MutableInt normalOut, - MutableInt weightedSumOut) { + MutableLong weightedSumOut) { int normal = 0; - int weightedSum = 0; + long weightedSum = 0; for (int ii = 0; ii < length; ++ii) { - final double weight = weightValues.get(start + ii); - final double component = doubleValues.get(start + ii); + final long weight = weightValues.get(start + ii); + final long component = longValues.get(start + ii); - if (weight == QueryConstants.NULL_DOUBLE || component == QueryConstants.NULL_DOUBLE) { + if (weight == QueryConstants.NULL_LONG || component == QueryConstants.NULL_LONG) { continue; } @@ -148,12 +149,12 @@ private static void sumChunks(LongChunk doubleValues, LongChun private boolean addChunk(LongChunk longValues, LongChunk weightValues, int start, int length, long destination) { final MutableInt normalOut = new MutableInt(); - final MutableInt weightedSumOut = new MutableInt(); + final MutableLong weightedSumOut = new MutableLong(); sumChunks(longValues, weightValues, start, length, normalOut, weightedSumOut); final int newNormal = normalOut.get(); - final int newWeightedSum = weightedSumOut.get(); + final long newWeightedSum = weightedSumOut.get(); final long totalNormal; final long existingNormal = normalCount.getUnsafe(destination); @@ -171,21 +172,21 @@ private boolean addChunk(LongChunk longValues, LongChunk doubleValues, LongChunk weightValues, + private boolean removeChunk(LongChunk longValues, LongChunk weightValues, int start, int length, long destination) { final MutableInt normalOut = new MutableInt(); - final MutableInt weightedSumOut = new MutableInt(); + final MutableLong weightedSumOut = new MutableLong(); - sumChunks(doubleValues, weightValues, start, length, normalOut, weightedSumOut); + sumChunks(longValues, weightValues, start, length, normalOut, weightedSumOut); final int newNormal = normalOut.get(); - final int newWeightedSum = weightedSumOut.get(); + final long newWeightedSum = weightedSumOut.get(); final long totalNormal; final long existingNormal = normalCount.getUnsafe(destination); @@ -226,17 +227,17 @@ private boolean modifyChunk(LongChunk prevDoubleValues, LongChunk prevWeightValues, LongChunk newDoubleValues, LongChunk newWeightValues, int start, int length, long destination) { final MutableInt normalOut = new MutableInt(); - final MutableInt weightedSumOut = new MutableInt(); + final MutableLong weightedSumOut = new MutableLong(); sumChunks(prevDoubleValues, prevWeightValues, start, length, normalOut, weightedSumOut); final int prevNormal = normalOut.get(); - final int prevWeightedSum = weightedSumOut.get(); + final long prevWeightedSum = weightedSumOut.get(); sumChunks(newDoubleValues, newWeightValues, start, length, normalOut, weightedSumOut); final int newNormal = normalOut.get(); - final int newWeightedSum = weightedSumOut.get(); + final long newWeightedSum = weightedSumOut.get(); final long totalNormal; final long existingNormal = normalCount.getUnsafe(destination); @@ -255,12 +256,12 @@ private boolean modifyChunk(LongChunk prevDoubleValues, weightedSum.set(destination, totalWeightedSum); } - final double existingResult = resultColumn.getAndSetUnsafe(destination, totalWeightedSum); + final long existingResult = resultColumn.getAndSetUnsafe(destination, totalWeightedSum); return totalWeightedSum != existingResult; } else { if (prevNormal > 0) { weightedSum.set(destination, 0L); - resultColumn.set(destination, QueryConstants.NULL_DOUBLE); + resultColumn.set(destination, QueryConstants.NULL_LONG); return true; } return false; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 138cc91b59c..6194ddc85a5 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -2248,6 +2248,40 @@ private void testWeightedAvgByIncremental(int size, int seed) { } + @Test + public void testWeightedSumByLong() { + final QueryTable table = testRefreshingTable(i(2, 4, 6).toTracking(), + col("Long1", 2L, 4L, 6L), col("Long2", 1L, 2L, 3L)); + final Table result = table.wsumBy("Long2"); + TableTools.show(result); + TestCase.assertEquals(1, result.size()); + long result_wsum = result.getColumnSource("Long1", long.class).getLong(result.getRowSet().firstRowKey()); + long wsum = 2 + 8 + 18; + TestCase.assertEquals(wsum, result_wsum); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(table, i(8), col("Long1", (long) Integer.MAX_VALUE), col("Long2", 7L)); + table.notifyListeners(i(8), i(), i()); + }); + show(result); + result_wsum = result.getColumnSource("Long1", long.class).getLong(result.getRowSet().firstRowKey()); + wsum = wsum + (7L * (long) Integer.MAX_VALUE); + TestCase.assertEquals(wsum, result_wsum); + } + + @Test + public void testId5522() { + final QueryTable table = testRefreshingTable(i(2, 4, 6).toTracking(), + col("Long1", 10L, 20L, 30L), col("Long2", 1L, NULL_LONG, 1L)); + final Table result = table.wsumBy("Long2"); + TableTools.show(result); + TestCase.assertEquals(1, result.size()); + long result_wsum = result.getColumnSource("Long1", long.class).getLong(result.getRowSet().firstRowKey()); + long wsum = 10 + 30; + TestCase.assertEquals(wsum, result_wsum); + } + @Test public void testWeightedSumByIncremental() { final int[] sizes = {10, 50, 200}; From 8f312d143a418c0d3a214427cc9487cdba218b77 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 24 Jun 2024 17:49:28 -0700 Subject: [PATCH 12/18] Corrected OOB test failures due to Iceberg API table column renames. (#5662) --- .../iceberg/util/IcebergToolsTest.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 7544976f27b..a2db0131879 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -118,8 +118,8 @@ public void testListNamespaces() { final Table table = adapter.listNamespacesAsTable(); Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); - Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); - Assert.eqTrue(table.getColumnSource("namespace_object").getType().equals(Namespace.class), + Assert.eqTrue(table.getColumnSource("Namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("NamespaceObject").getType().equals(Namespace.class), "namespace_object column type"); } @@ -138,9 +138,9 @@ public void testListTables() { Table table = adapter.listTablesAsTable(ns); Assert.eq(table.size(), "table.size()", 3, "3 tables in the namespace"); - Assert.eqTrue(table.getColumnSource("namespace").getType().equals(String.class), "namespace column type"); - Assert.eqTrue(table.getColumnSource("table_name").getType().equals(String.class), "table_name column type"); - Assert.eqTrue(table.getColumnSource("table_identifier_object").getType().equals(TableIdentifier.class), + Assert.eqTrue(table.getColumnSource("Namespace").getType().equals(String.class), "namespace column type"); + Assert.eqTrue(table.getColumnSource("TableName").getType().equals(String.class), "table_name column type"); + Assert.eqTrue(table.getColumnSource("TableIdentifierObject").getType().equals(TableIdentifier.class), "table_identifier_object column type"); // Test the string versions of the methods @@ -166,11 +166,11 @@ public void testListSnapshots() { Table table = adapter.listSnapshotsAsTable(tableIdentifier); Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); - Assert.eqTrue(table.getColumnSource("id").getType().equals(long.class), "id column type"); - Assert.eqTrue(table.getColumnSource("timestamp_ms").getType().equals(long.class), "timestamp_ms column type"); - Assert.eqTrue(table.getColumnSource("operation").getType().equals(String.class), "operation column type"); - Assert.eqTrue(table.getColumnSource("summary").getType().equals(Map.class), "summary column type"); - Assert.eqTrue(table.getColumnSource("snapshot_object").getType().equals(Snapshot.class), + Assert.eqTrue(table.getColumnSource("Id").getType().equals(long.class), "id column type"); + Assert.eqTrue(table.getColumnSource("TimestampMs").getType().equals(long.class), "timestamp_ms column type"); + Assert.eqTrue(table.getColumnSource("Operation").getType().equals(String.class), "operation column type"); + Assert.eqTrue(table.getColumnSource("Summary").getType().equals(Map.class), "summary column type"); + Assert.eqTrue(table.getColumnSource("SnapshotObject").getType().equals(Snapshot.class), "snapshot_object column type"); // Test the string versions of the methods From 697b1382dc85481be0adaf908cb7455a60374258 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 25 Jun 2024 06:15:59 -0700 Subject: [PATCH 13/18] Remove server dependency on extensions-iceberg (#5664) --- server/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/server/build.gradle b/server/build.gradle index d3ccceabb13..27f6f18aa0a 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -9,7 +9,6 @@ dependencies { implementation project(':engine-table') implementation project(':extensions-csv') implementation project(':extensions-arrow') - implementation project(':extensions-iceberg') implementation project(':extensions-parquet-table') implementation project(':extensions-performance') implementation project(':extensions-jdbc') From f98c78db4ab462f15c34176f208d379731d275d4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 25 Jun 2024 09:35:38 -0700 Subject: [PATCH 14/18] renamed iceberg s3 project (#5663) --- py/embedded-server/java-runtime/build.gradle | 2 +- server/jetty-app-custom/build.gradle | 2 +- server/jetty-app/build.gradle | 2 +- server/netty-app/build.gradle | 2 +- settings.gradle | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/py/embedded-server/java-runtime/build.gradle b/py/embedded-server/java-runtime/build.gradle index b64ddaf2958..b3e43690825 100644 --- a/py/embedded-server/java-runtime/build.gradle +++ b/py/embedded-server/java-runtime/build.gradle @@ -32,7 +32,7 @@ dependencies { if (!hasProperty('excludeS3')) { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg:s3') + runtimeOnly project(':extensions-iceberg-s3') } } diff --git a/server/jetty-app-custom/build.gradle b/server/jetty-app-custom/build.gradle index 11917d41326..2224c4e3f90 100644 --- a/server/jetty-app-custom/build.gradle +++ b/server/jetty-app-custom/build.gradle @@ -55,7 +55,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg:s3') + runtimeOnly project(':extensions-iceberg-s3') } } diff --git a/server/jetty-app/build.gradle b/server/jetty-app/build.gradle index 670c7cfb982..b2024d8b209 100644 --- a/server/jetty-app/build.gradle +++ b/server/jetty-app/build.gradle @@ -54,7 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg:s3') + runtimeOnly project(':extensions-iceberg-s3') } } diff --git a/server/netty-app/build.gradle b/server/netty-app/build.gradle index cd5b23ea1bb..95d9eeea483 100644 --- a/server/netty-app/build.gradle +++ b/server/netty-app/build.gradle @@ -54,7 +54,7 @@ if (!hasProperty('excludeSql')) { if (!hasProperty('excludeS3')) { dependencies { runtimeOnly project(':extensions-s3') - runtimeOnly project(':extensions-iceberg:s3') + runtimeOnly project(':extensions-iceberg-s3') } } diff --git a/settings.gradle b/settings.gradle index d381956ca88..deef79359aa 100644 --- a/settings.gradle +++ b/settings.gradle @@ -270,8 +270,8 @@ project(':extensions-s3').projectDir = file('extensions/s3') include(':extensions-iceberg') project(':extensions-iceberg').projectDir = file('extensions/iceberg') -include(':extensions-iceberg:s3') -project(':extensions-iceberg:s3').projectDir = file('extensions/iceberg/s3') +include(':extensions-iceberg-s3') +project(':extensions-iceberg-s3').projectDir = file('extensions/iceberg/s3') include(':extensions-json') project(':extensions-json').projectDir = file('extensions/json') From f647f1afd43e3dd58cf70dde40886bb7dfdb2417 Mon Sep 17 00:00:00 2001 From: Nate Bauernfeind Date: Tue, 25 Jun 2024 11:03:30 -0600 Subject: [PATCH 15/18] Fix ExactMatch Filter for Non-Convertible Types; Use RangeFilter on QueryScope Vars (#5587) Co-authored-by: Ryan Caudy --- .../util/datastructures/CachingSupplier.java | 10 + .../engine/table/impl/DeferredViewTable.java | 4 +- .../table/impl/select/ByteRangeFilter.java | 12 +- .../table/impl/select/CharRangeFilter.java | 12 +- .../table/impl/select/DoubleRangeFilter.java | 12 +- .../table/impl/select/FloatRangeFilter.java | 12 +- .../table/impl/select/IntRangeFilter.java | 12 +- .../table/impl/select/LongRangeFilter.java | 12 +- .../engine/table/impl/select/MatchFilter.java | 448 +++++++++++++++--- ...eConditionFilter.java => RangeFilter.java} | 208 ++++---- .../table/impl/select/ShortRangeFilter.java | 12 +- .../engine/table/impl/select/WhereFilter.java | 4 + .../table/impl/select/WhereFilterAdapter.java | 24 +- .../table/impl/select/WhereFilterFactory.java | 167 ++++--- .../gui/table/filters/Condition.java | 46 +- .../engine/table/impl/QueryFactory.java | 2 +- .../table/impl/QueryTableWhereTest.java | 304 +++++++++++- .../table/impl/select/WhereFilterTest.java | 164 +++---- .../java/io/deephaven/time/DateTimeUtils.java | 66 +++ .../io/deephaven/time/TestDateTimeUtils.java | 132 ++++++ .../table/ops/filter/FilterFactory.java | 4 +- 21 files changed, 1283 insertions(+), 384 deletions(-) rename engine/table/src/main/java/io/deephaven/engine/table/impl/select/{RangeConditionFilter.java => RangeFilter.java} (53%) diff --git a/Util/src/main/java/io/deephaven/util/datastructures/CachingSupplier.java b/Util/src/main/java/io/deephaven/util/datastructures/CachingSupplier.java index 0a0880f7896..d36719b1ad0 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/CachingSupplier.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/CachingSupplier.java @@ -49,4 +49,14 @@ public OUTPUT_TYPE get() { } return cachedResult; } + + public OUTPUT_TYPE getIfCached() { + if (hasCachedResult) { // force a volatile read + if (errorResult != null) { + throw errorResult; + } + return cachedResult; + } + return null; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/DeferredViewTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/DeferredViewTable.java index 37816ccd279..c6aaed349db 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/DeferredViewTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/DeferredViewTable.java @@ -212,9 +212,7 @@ private PreAndPostFilters applyFilterRenamings(WhereFilter[] filters) { } else if (filter instanceof MatchFilter) { final MatchFilter matchFilter = (MatchFilter) filter; Assert.assertion(myRenames.size() == 1, "Match Filters should only use one column!"); - String newName = myRenames.get(matchFilter.getColumnName()); - Assert.neqNull(newName, "newName"); - final MatchFilter newFilter = matchFilter.renameFilter(newName); + final WhereFilter newFilter = matchFilter.renameFilter(myRenames); newFilter.init(tableReference.getDefinition(), compilationProcessor); preViewFilters.add(newFilter); } else if (filter instanceof ConditionFilter) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ByteRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ByteRangeFilter.java index d506eee8bc5..49bd613eba2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ByteRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ByteRangeFilter.java @@ -51,18 +51,18 @@ public ByteRangeFilter(String columnName, byte val1, byte val2, boolean lowerInc } } - static WhereFilter makeByteRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeByteRangeFilter(String columnName, Condition condition, byte value) { switch (condition) { case LESS_THAN: - return lt(columnName, RangeConditionFilter.parseByteFilter(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, RangeConditionFilter.parseByteFilter(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, RangeConditionFilter.parseByteFilter(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, RangeConditionFilter.parseByteFilter(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/CharRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/CharRangeFilter.java index f5d640d13f5..466f58de85a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/CharRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/CharRangeFilter.java @@ -47,18 +47,18 @@ public CharRangeFilter(String columnName, char val1, char val2, boolean lowerInc } } - static WhereFilter makeCharRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeCharRangeFilter(String columnName, Condition condition, char value) { switch (condition) { case LESS_THAN: - return lt(columnName, RangeConditionFilter.parseCharFilter(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, RangeConditionFilter.parseCharFilter(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, RangeConditionFilter.parseCharFilter(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, RangeConditionFilter.parseCharFilter(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DoubleRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DoubleRangeFilter.java index 1b195e01b0b..edaed1132a9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DoubleRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DoubleRangeFilter.java @@ -63,18 +63,18 @@ public static WhereFilter makeRange(String columnName, String val) { (double) (positiveOrZero ? parsed + offset : parsed - offset), positiveOrZero, !positiveOrZero); } - static WhereFilter makeDoubleRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeDoubleRangeFilter(String columnName, Condition condition, double value) { switch (condition) { case LESS_THAN: - return lt(columnName, Double.parseDouble(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, Double.parseDouble(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, Double.parseDouble(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, Double.parseDouble(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FloatRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FloatRangeFilter.java index 2cdebf0bde7..d06d81f1857 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FloatRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FloatRangeFilter.java @@ -59,18 +59,18 @@ public static WhereFilter makeRange(String columnName, String val) { (float) (positiveOrZero ? parsed + offset : parsed - offset), positiveOrZero, !positiveOrZero); } - static WhereFilter makeFloatRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeFloatRangeFilter(String columnName, Condition condition, float value) { switch (condition) { case LESS_THAN: - return lt(columnName, Float.parseFloat(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, Float.parseFloat(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, Float.parseFloat(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, Float.parseFloat(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/IntRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/IntRangeFilter.java index ee4a345e2a2..cbe4daafef5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/IntRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/IntRangeFilter.java @@ -51,18 +51,18 @@ public IntRangeFilter(String columnName, int val1, int val2, boolean lowerInclus } } - static WhereFilter makeIntRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeIntRangeFilter(String columnName, Condition condition, int value) { switch (condition) { case LESS_THAN: - return lt(columnName, RangeConditionFilter.parseIntFilter(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, RangeConditionFilter.parseIntFilter(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, RangeConditionFilter.parseIntFilter(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, RangeConditionFilter.parseIntFilter(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/LongRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/LongRangeFilter.java index d0718eab953..353cb457e4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/LongRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/LongRangeFilter.java @@ -51,18 +51,18 @@ public LongRangeFilter(String columnName, long val1, long val2, boolean lowerInc } } - static WhereFilter makeLongRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeLongRangeFilter(String columnName, Condition condition, long value) { switch (condition) { case LESS_THAN: - return lt(columnName, RangeConditionFilter.parseLongFilter(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, RangeConditionFilter.parseLongFilter(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, RangeConditionFilter.parseLongFilter(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, RangeConditionFilter.parseLongFilter(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java index 2144b522bb1..d4026f087aa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java @@ -5,6 +5,7 @@ import io.deephaven.api.literal.Literal; import io.deephaven.base.string.cache.CompressedString; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; @@ -14,13 +15,17 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.QueryCompilerRequestProcessor; +import io.deephaven.engine.table.impl.lang.QueryLanguageFunctionUtils; import io.deephaven.engine.table.impl.preview.DisplayWrapper; import io.deephaven.engine.table.impl.DependencyStreamProvider; import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.updategraph.NotificationQueue; import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.QueryConstants; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.datastructures.CachingSupplier; import io.deephaven.util.type.ArrayTypeUtils; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.jpy.PyObject; @@ -28,7 +33,12 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZonedDateTime; import java.util.*; +import java.util.function.Consumer; import java.util.stream.Stream; public class MatchFilter extends WhereFilterImpl implements DependencyStreamProvider { @@ -45,8 +55,11 @@ static MatchFilter ofLiterals( literals.stream().map(AsObject::of).toArray()); } + /** A fail-over WhereFilter supplier should the match filter initialization fail. */ + private final CachingSupplier failoverFilter; + @NotNull - private final String columnName; + private String columnName; private Object[] values; private final String[] strValues; private final boolean invertMatch; @@ -78,7 +91,7 @@ public MatchFilter( @NotNull final MatchType matchType, @NotNull final String columnName, @NotNull final Object... values) { - this(CaseSensitivity.MatchCase, matchType, columnName, null, values); + this(null, CaseSensitivity.MatchCase, matchType, columnName, null, values); } /** @@ -89,30 +102,34 @@ public MatchFilter( public MatchFilter( @NotNull final String columnName, @NotNull final Object... values) { - this(CaseSensitivity.IgnoreCase, MatchType.Regular, columnName, null, values); + this(null, CaseSensitivity.IgnoreCase, MatchType.Regular, columnName, null, values); } public MatchFilter( @NotNull final CaseSensitivity sensitivity, + @NotNull final MatchType matchType, @NotNull final String columnName, @NotNull final String... strValues) { - this(sensitivity, MatchType.Regular, columnName, strValues, null); + this(null, sensitivity, matchType, columnName, strValues, null); } public MatchFilter( + @Nullable final CachingSupplier failoverFilter, @NotNull final CaseSensitivity sensitivity, @NotNull final MatchType matchType, @NotNull final String columnName, @NotNull final String... strValues) { - this(sensitivity, matchType, columnName, strValues, null); + this(failoverFilter, sensitivity, matchType, columnName, strValues, null); } private MatchFilter( + @Nullable final CachingSupplier failoverFilter, @NotNull final CaseSensitivity sensitivity, @NotNull final MatchType matchType, @NotNull final String columnName, @Nullable String[] strValues, @Nullable final Object[] values) { + this.failoverFilter = failoverFilter; this.caseInsensitive = sensitivity == CaseSensitivity.IgnoreCase; this.invertMatch = (matchType == MatchType.Inverted); this.columnName = columnName; @@ -120,20 +137,25 @@ private MatchFilter( this.values = values; } - public MatchFilter renameFilter(String newName) { + private ConditionFilter getFailoverFilterIfCached() { + return failoverFilter != null ? failoverFilter.getIfCached() : null; + } + + public WhereFilter renameFilter(Map renames) { + final String newName = renames.get(columnName); + Assert.neqNull(newName, "newName"); if (strValues == null) { + // when we're constructed with values then there is no failover filter return new MatchFilter(getMatchType(), newName, values); } else { return new MatchFilter( + failoverFilter != null ? new CachingSupplier<>( + () -> failoverFilter.get().renameFilter(renames)) : null, caseInsensitive ? CaseSensitivity.IgnoreCase : CaseSensitivity.MatchCase, - getMatchType(), newName, strValues); + getMatchType(), newName, strValues, null); } } - public String getColumnName() { - return columnName; - } - public Object[] getValues() { return values; } @@ -148,11 +170,25 @@ public MatchType getMatchType() { @Override public List getColumns() { + if (!initialized) { + throw new IllegalStateException("Filter must be initialized to invoke getColumnName"); + } + final WhereFilter failover = getFailoverFilterIfCached(); + if (failover != null) { + return failover.getColumns(); + } return Collections.singletonList(columnName); } @Override public List getColumnArrays() { + if (!initialized) { + throw new IllegalStateException("Filter must be initialized to invoke getColumnArrays"); + } + final WhereFilter failover = getFailoverFilterIfCached(); + if (failover != null) { + return failover.getColumnArrays(); + } return Collections.emptyList(); } @@ -168,47 +204,41 @@ public synchronized void init( if (initialized) { return; } - ColumnDefinition column = tableDefinition.getColumn(columnName); - if (column == null) { - throw new RuntimeException("Column \"" + columnName - + "\" doesn't exist in this table, available columns: " + tableDefinition.getColumnNames()); - } - if (strValues == null) { - initialized = true; - return; - } - final List valueList = new ArrayList<>(); - final Map queryScopeVariables = compilationProcessor.getQueryScopeVariables(); - final ColumnTypeConvertor convertor = - ColumnTypeConvertorFactory.getConvertor(column.getDataType(), column.getName()); - for (String strValue : strValues) { - if (queryScopeVariables.containsKey(strValue)) { - Object paramValue = queryScopeVariables.get(strValue); - if (paramValue != null && paramValue.getClass().isArray()) { - ArrayTypeUtils.ArrayAccessor accessor = ArrayTypeUtils.getArrayAccessor(paramValue); - for (int ai = 0; ai < accessor.length(); ++ai) { - valueList.add(convertor.convertParamValue(accessor.get(ai))); - } - } else if (paramValue != null && Collection.class.isAssignableFrom(paramValue.getClass())) { - for (final Object paramValueMember : (Collection) paramValue) { - valueList.add(convertor.convertParamValue(paramValueMember)); - } + try { + ColumnDefinition column = tableDefinition.getColumn(columnName); + if (column == null) { + if (strValues != null && strValues.length == 1 + && (column = tableDefinition.getColumn(strValues[0])) != null) { + // fix up for the case where column name and variable name were swapped + String tmp = columnName; + columnName = strValues[0]; + strValues[0] = tmp; } else { - valueList.add(convertor.convertParamValue(paramValue)); + throw new RuntimeException("Column \"" + columnName + + "\" doesn't exist in this table, available columns: " + tableDefinition.getColumnNames()); } - } else { - Object convertedValue; - try { - convertedValue = convertor.convertStringLiteral(strValue); - } catch (Throwable t) { - throw new IllegalArgumentException("Failed to convert literal value <" + strValue + - "> for column \"" + columnName + "\" of type " + column.getDataType().getName(), t); - } - valueList.add(convertedValue); + } + if (strValues == null) { + initialized = true; + return; + } + final List valueList = new ArrayList<>(); + final Map queryScopeVariables = compilationProcessor.getQueryScopeVariables(); + final ColumnTypeConvertor convertor = ColumnTypeConvertorFactory.getConvertor(column.getDataType()); + for (String strValue : strValues) { + convertor.convertValue(column, tableDefinition, strValue, queryScopeVariables, valueList::add); + } + values = valueList.toArray(); + } catch (final RuntimeException err) { + if (failoverFilter == null) { + throw err; + } + try { + failoverFilter.get().init(tableDefinition, compilationProcessor); + } catch (final RuntimeException ignored) { + throw err; } } - // values = (Object[])ArrayTypeUtils.toArray(valueList, TypeUtils.getBoxedType(theColumn.getDataType())); - values = valueList.toArray(); initialized = true; } @@ -250,6 +280,11 @@ public Stream getDependencyStream() { @Override public WritableRowSet filter( @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev) { + final WhereFilter failover = getFailoverFilterIfCached(); + if (failover != null) { + return failover.filter(selection, fullSet, table, usePrev); + } + final ColumnSource columnSource = table.getColumnSource(columnName); return columnSource.match(invertMatch, usePrev, caseInsensitive, dataIndex, selection, values); } @@ -258,12 +293,22 @@ public WritableRowSet filter( @Override public WritableRowSet filterInverse( @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev) { + final WhereFilter failover = getFailoverFilterIfCached(); + if (failover != null) { + return failover.filterInverse(selection, fullSet, table, usePrev); + } + final ColumnSource columnSource = table.getColumnSource(columnName); return columnSource.match(!invertMatch, usePrev, caseInsensitive, dataIndex, selection, values); } @Override public boolean isSimpleFilter() { + final WhereFilter failover = getFailoverFilterIfCached(); + if (failover != null) { + return failover.isSimpleFilter(); + } + return true; } @@ -282,56 +327,205 @@ Object convertParamValue(Object paramValue) { } return paramValue; } + + /** + * Convert the string value to the appropriate type for the column. + * + * @param column the column definition + * @param strValue the string value to convert + * @param queryScopeVariables the query scope variables + * @param valueConsumer the consumer for the converted value + * @return whether the value was an array or collection + */ + final boolean convertValue( + @NotNull final ColumnDefinition column, + @NotNull final TableDefinition tableDefinition, + @NotNull final String strValue, + @NotNull final Map queryScopeVariables, + @NotNull final Consumer valueConsumer) { + if (tableDefinition.getColumn(strValue) != null) { + // this is also a column name which needs to take precedence, and we can't convert it + throw new IllegalArgumentException(String.format( + "Failed to convert value <%s> for column \"%s\" of type %s; it is a column name", + strValue, column.getName(), column.getDataType().getName())); + } + if (strValue.endsWith("_") + && tableDefinition.getColumn(strValue.substring(0, strValue.length() - 1)) != null) { + // this also a column array name which needs to take precedence, and we can't convert it + throw new IllegalArgumentException(String.format( + "Failed to convert value <%s> for column \"%s\" of type %s; it is a column array access name", + strValue, column.getName(), column.getDataType().getName())); + } + + if (queryScopeVariables.containsKey(strValue)) { + Object paramValue = queryScopeVariables.get(strValue); + if (paramValue != null && paramValue.getClass().isArray()) { + ArrayTypeUtils.ArrayAccessor accessor = ArrayTypeUtils.getArrayAccessor(paramValue); + for (int ai = 0; ai < accessor.length(); ++ai) { + valueConsumer.accept(convertParamValue(accessor.get(ai))); + } + return true; + } + if (paramValue != null && Collection.class.isAssignableFrom(paramValue.getClass())) { + for (final Object paramValueMember : (Collection) paramValue) { + valueConsumer.accept(convertParamValue(paramValueMember)); + } + return true; + } + valueConsumer.accept(convertParamValue(paramValue)); + return false; + } + + try { + valueConsumer.accept(convertStringLiteral(strValue)); + } catch (Throwable t) { + throw new IllegalArgumentException(String.format( + "Failed to convert literal value <%s> for column \"%s\" of type %s", + strValue, column.getName(), column.getDataType().getName()), t); + } + + return false; + } } public static class ColumnTypeConvertorFactory { - public static ColumnTypeConvertor getConvertor(final Class cls, final String name) { + public static ColumnTypeConvertor getConvertor(final Class cls) { if (cls == byte.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_BYTE".equals(str)) { + return QueryConstants.NULL_BYTE_BOXED; + } return Byte.parseByte(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Byte || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.byteCast(boxer.get(paramValue)); + } }; } if (cls == short.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_SHORT".equals(str)) { + return QueryConstants.NULL_SHORT_BOXED; + } return Short.parseShort(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Short || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.shortCast(boxer.get(paramValue)); + } }; } if (cls == int.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_INT".equals(str)) { + return QueryConstants.NULL_INT_BOXED; + } return Integer.parseInt(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Integer || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.intCast(boxer.get(paramValue)); + } }; } if (cls == long.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_LONG".equals(str)) { + return QueryConstants.NULL_LONG_BOXED; + } return Long.parseLong(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Long || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.longCast(boxer.get(paramValue)); + } }; } if (cls == float.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_FLOAT".equals(str)) { + return QueryConstants.NULL_FLOAT_BOXED; + } return Float.parseFloat(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Float || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.floatCast(boxer.get(paramValue)); + } }; } if (cls == double.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_DOUBLE".equals(str)) { + return QueryConstants.NULL_DOUBLE_BOXED; + } return Double.parseDouble(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Double || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.doubleCast(boxer.get(paramValue)); + } }; } if (cls == Boolean.class) { @@ -339,6 +533,9 @@ Object convertStringLiteral(String str) { @Override Object convertStringLiteral(String str) { // NB: Boolean.parseBoolean(str) doesn't do what we want here - anything not true is false. + if ("null".equals(str) || "NULL_BOOLEAN".equals(str)) { + return QueryConstants.NULL_BOOLEAN; + } if (str.equalsIgnoreCase("true")) { return Boolean.TRUE; } @@ -354,6 +551,9 @@ Object convertStringLiteral(String str) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str) || "NULL_CHAR".equals(str)) { + return QueryConstants.NULL_CHAR_BOXED; + } if (str.length() > 1) { // TODO: #1517 Allow escaping of chars if (str.length() == 3 && ((str.charAt(0) == '\'' && str.charAt(2) == '\'') @@ -366,22 +566,84 @@ Object convertStringLiteral(String str) { } return str.charAt(0); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof Character || paramValue == null) { + return paramValue; + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + return QueryLanguageFunctionUtils.charCast(boxer.get(paramValue)); + } }; } if (cls == BigDecimal.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } return new BigDecimal(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof BigDecimal || paramValue == null) { + return paramValue; + } + if (paramValue instanceof BigInteger) { + return new BigDecimal((BigInteger) paramValue); + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + final Object boxedValue = boxer.get(paramValue); + if (boxedValue == null) { + return null; + } + if (boxedValue instanceof Number) { + return BigDecimal.valueOf(((Number) boxedValue).doubleValue()); + } + return paramValue; + } }; } if (cls == BigInteger.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } return new BigInteger(str); } + + @Override + Object convertParamValue(Object paramValue) { + paramValue = super.convertParamValue(paramValue); + if (paramValue instanceof BigInteger || paramValue == null) { + return paramValue; + } + if (paramValue instanceof BigDecimal) { + return ((BigDecimal) paramValue).toBigInteger(); + } + // noinspection unchecked + final TypeUtils.TypeBoxer boxer = + (TypeUtils.TypeBoxer) TypeUtils.getTypeBoxer(paramValue.getClass()); + final Object boxedValue = boxer.get(paramValue); + if (boxedValue == null) { + return null; + } + if (boxedValue instanceof Number) { + return BigInteger.valueOf(((Number) boxedValue).longValue()); + } + return paramValue; + } }; } if (cls == String.class) { @@ -451,6 +713,9 @@ Object convertParamValue(Object paramValue) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } if (str.charAt(0) != '\'' || str.charAt(str.length() - 1) != '\'') { throw new IllegalArgumentException( "Instant literal not enclosed in single-quotes (\"" + str + "\")"); @@ -459,10 +724,73 @@ Object convertStringLiteral(String str) { } }; } + if (cls == LocalDate.class) { + return new ColumnTypeConvertor() { + @Override + Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } + if (str.charAt(0) != '\'' || str.charAt(str.length() - 1) != '\'') { + throw new IllegalArgumentException( + "LocalDate literal not enclosed in single-quotes (\"" + str + "\")"); + } + return DateTimeUtils.parseLocalDate(str.substring(1, str.length() - 1)); + } + }; + } + if (cls == LocalTime.class) { + return new ColumnTypeConvertor() { + @Override + Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } + if (str.charAt(0) != '\'' || str.charAt(str.length() - 1) != '\'') { + throw new IllegalArgumentException( + "LocalTime literal not enclosed in single-quotes (\"" + str + "\")"); + } + return DateTimeUtils.parseLocalTime(str.substring(1, str.length() - 1)); + } + }; + } + if (cls == LocalDateTime.class) { + return new ColumnTypeConvertor() { + @Override + Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } + if (str.charAt(0) != '\'' || str.charAt(str.length() - 1) != '\'') { + throw new IllegalArgumentException( + "LocalDateTime literal not enclosed in single-quotes (\"" + str + "\")"); + } + return DateTimeUtils.parseLocalDateTime(str.substring(1, str.length() - 1)); + } + }; + } + if (cls == ZonedDateTime.class) { + return new ColumnTypeConvertor() { + @Override + Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } + if (str.charAt(0) != '\'' || str.charAt(str.length() - 1) != '\'') { + throw new IllegalArgumentException( + "ZoneDateTime literal not enclosed in single-quotes (\"" + str + "\")"); + } + return DateTimeUtils.parseZonedDateTime(str.substring(1, str.length() - 1)); + } + }; + } if (cls == Object.class) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } if (str.startsWith("\"") || str.startsWith("`")) { return str.substring(1, str.length() - 1); } else if (str.contains(".")) { @@ -489,6 +817,9 @@ Object convertStringLiteral(String str) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } if (str.startsWith("\"") || str.startsWith("`")) { return DisplayWrapper.make(str.substring(1, str.length() - 1)); } else { @@ -497,8 +828,16 @@ Object convertStringLiteral(String str) { } }; } - throw new IllegalArgumentException( - "Unknown type " + cls.getName() + " for MatchFilter value auto-conversion"); + return new ColumnTypeConvertor() { + @Override + Object convertStringLiteral(String str) { + if ("null".equals(str)) { + return null; + } + throw new IllegalArgumentException( + "Can't create " + cls.getName() + " from String Literal for value auto-conversion"); + } + }; } } @@ -544,9 +883,12 @@ public boolean canMemoize() { public WhereFilter copy() { final MatchFilter copy; if (strValues != null) { - copy = new MatchFilter(caseInsensitive ? CaseSensitivity.IgnoreCase : CaseSensitivity.MatchCase, - getMatchType(), columnName, strValues); + copy = new MatchFilter( + failoverFilter == null ? null : new CachingSupplier<>(() -> failoverFilter.get().copy()), + caseInsensitive ? CaseSensitivity.IgnoreCase : CaseSensitivity.MatchCase, + getMatchType(), columnName, strValues, null); } else { + // when we're constructed with values then there is no failover filter copy = new MatchFilter(getMatchType(), columnName, values); } if (initialized) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeConditionFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java similarity index 53% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeConditionFilter.java rename to engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java index f7cdfaa570a..65d65b75e03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeConditionFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java @@ -12,12 +12,18 @@ import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.gui.table.filters.Condition; +import io.deephaven.util.annotations.VisibleForTesting; import io.deephaven.util.type.TypeUtils; +import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; import java.math.BigDecimal; import java.math.BigInteger; -import java.util.Collections; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZonedDateTime; import java.util.List; /** @@ -29,11 +35,11 @@ *
  • GREATER_THAN_OR_EQUAL
  • * */ -public class RangeConditionFilter extends WhereFilterImpl { +public class RangeFilter extends WhereFilterImpl { - private final String columnName; - private final Condition condition; - private final String value; + private String columnName; + private String value; + private Condition condition; // The expression prior to being parsed private final String expression; @@ -42,48 +48,48 @@ public class RangeConditionFilter extends WhereFilterImpl { private final FormulaParserConfiguration parserConfiguration; /** - * Creates a RangeConditionFilter. + * Creates a RangeFilter. * * @param columnName the column to filter * @param condition the condition for filtering * @param value a String representation of the numeric filter value */ - public RangeConditionFilter(String columnName, Condition condition, String value) { + public RangeFilter(String columnName, Condition condition, String value) { this(columnName, condition, value, null, null, null); } /** - * Creates a RangeConditionFilter. + * Creates a RangeFilter. * * @param columnName the column to filter * @param condition the condition for filtering * @param value a String representation of the numeric filter value * @param expression the original expression prior to being parsed - * @param parserConfiguration + * @param parserConfiguration the parser configuration to use */ - public RangeConditionFilter(String columnName, Condition condition, String value, String expression, + public RangeFilter(String columnName, Condition condition, String value, String expression, FormulaParserConfiguration parserConfiguration) { this(columnName, condition, value, expression, null, parserConfiguration); } /** - * Creates a RangeConditionFilter. + * Creates a RangeFilter. * * @param columnName the column to filter * @param conditionString the String representation of a condition for filtering * @param value a String representation of the numeric filter value * @param expression the original expression prior to being parsed - * @param parserConfiguration + * @param parserConfiguration the parser configuration to use */ - public RangeConditionFilter(String columnName, String conditionString, String value, String expression, + public RangeFilter(String columnName, String conditionString, String value, String expression, FormulaParserConfiguration parserConfiguration) { this(columnName, conditionFromString(conditionString), value, expression, parserConfiguration); } // Used for copy method - private RangeConditionFilter(String columnName, Condition condition, String value, String expression, + private RangeFilter(String columnName, Condition condition, String value, String expression, WhereFilter filter, FormulaParserConfiguration parserConfiguration) { - Assert.eqTrue(conditionSupported(condition), condition + " is not supported by RangeConditionFilter"); + Assert.eqTrue(conditionSupported(condition), condition + " is not supported by RangeFilter"); this.columnName = columnName; this.condition = condition; this.value = value; @@ -115,18 +121,29 @@ private static Condition conditionFromString(String conditionString) { case ">=": return Condition.GREATER_THAN_OR_EQUAL; default: - throw new IllegalArgumentException(conditionString + " is not supported by RangeConditionFilter"); + throw new IllegalArgumentException(conditionString + " is not supported by RangeFilter"); } } @Override public List getColumns() { - return Collections.singletonList(columnName); + if (filter == null) { + throw new IllegalStateException("Filter must be initialized to invoke getColumnName"); + } + return filter.getColumns(); } @Override public List getColumnArrays() { - return Collections.emptyList(); + if (filter == null) { + throw new IllegalStateException("Filter must be initialized to invoke getColumnArrays"); + } + return filter.getColumnArrays(); + } + + @VisibleForTesting + public WhereFilter getRealFilter() { + return filter; } @Override @@ -142,47 +159,104 @@ public void init( return; } - final ColumnDefinition def = tableDefinition.getColumn(columnName); + RuntimeException conversionError = null; + ColumnDefinition def = tableDefinition.getColumn(columnName); if (def == null) { - throw new RuntimeException("Column \"" + columnName + "\" doesn't exist in this table, available columns: " - + tableDefinition.getColumnNames()); + if ((def = tableDefinition.getColumn(value)) != null) { + // fix up for the case where column name and variable name were swapped + String tmp = columnName; + columnName = value; + value = tmp; + condition = condition.mirror(); + } else { + conversionError = new RuntimeException("Column \"" + columnName + + "\" doesn't exist in this table, available columns: " + tableDefinition.getColumnNames()); + } } - final Class colClass = def.getDataType(); + final Class colClass = def == null ? null : def.getDataType(); + final MutableObject realValue = new MutableObject<>(); + + if (def != null) { + final MatchFilter.ColumnTypeConvertor convertor = + MatchFilter.ColumnTypeConvertorFactory.getConvertor(def.getDataType()); + + try { + boolean wasAnArrayType = convertor.convertValue( + def, tableDefinition, value, compilationProcessor.getQueryScopeVariables(), + realValue::setValue); + if (wasAnArrayType) { + conversionError = + new IllegalArgumentException("RangeFilter does not support array types for column " + + columnName + " with value <" + value + ">"); + } + } catch (final RuntimeException err) { + conversionError = err; + } + } - if (colClass == double.class || colClass == Double.class) { - filter = DoubleRangeFilter.makeDoubleRangeFilter(columnName, condition, value); + if (conversionError != null) { + if (expression != null) { + try { + filter = ConditionFilter.createConditionFilter(expression, parserConfiguration); + } catch (final RuntimeException ignored) { + throw conversionError; + } + } else { + throw conversionError; + } + } else if (colClass == double.class || colClass == Double.class) { + filter = DoubleRangeFilter.makeDoubleRangeFilter(columnName, condition, + TypeUtils.unbox((Double) realValue.getValue())); } else if (colClass == float.class || colClass == Float.class) { - filter = FloatRangeFilter.makeFloatRangeFilter(columnName, condition, value); + filter = FloatRangeFilter.makeFloatRangeFilter(columnName, condition, + TypeUtils.unbox((Float) realValue.getValue())); } else if (colClass == char.class || colClass == Character.class) { - filter = CharRangeFilter.makeCharRangeFilter(columnName, condition, value); + filter = CharRangeFilter.makeCharRangeFilter(columnName, condition, + TypeUtils.unbox((Character) realValue.getValue())); } else if (colClass == byte.class || colClass == Byte.class) { - filter = ByteRangeFilter.makeByteRangeFilter(columnName, condition, value); + filter = ByteRangeFilter.makeByteRangeFilter(columnName, condition, + TypeUtils.unbox((Byte) realValue.getValue())); } else if (colClass == short.class || colClass == Short.class) { - filter = ShortRangeFilter.makeShortRangeFilter(columnName, condition, value); + filter = ShortRangeFilter.makeShortRangeFilter(columnName, condition, + TypeUtils.unbox((Short) realValue.getValue())); } else if (colClass == int.class || colClass == Integer.class) { - filter = IntRangeFilter.makeIntRangeFilter(columnName, condition, value); + filter = IntRangeFilter.makeIntRangeFilter(columnName, condition, + TypeUtils.unbox((Integer) realValue.getValue())); } else if (colClass == long.class || colClass == Long.class) { - filter = LongRangeFilter.makeLongRangeFilter(columnName, condition, value); - } else if (io.deephaven.util.type.TypeUtils.isDateTime(colClass)) { - filter = makeDateTimeRangeFilter(columnName, condition, value); + filter = LongRangeFilter.makeLongRangeFilter(columnName, condition, + TypeUtils.unbox((Long) realValue.getValue())); + } else if (colClass == Instant.class) { + filter = makeInstantRangeFilter(columnName, condition, + DateTimeUtils.epochNanos((Instant) realValue.getValue())); + } else if (colClass == LocalDate.class) { + filter = makeComparableRangeFilter(columnName, condition, (LocalDate) realValue.getValue()); + } else if (colClass == LocalTime.class) { + filter = makeComparableRangeFilter(columnName, condition, (LocalTime) realValue.getValue()); + } else if (colClass == LocalDateTime.class) { + filter = makeComparableRangeFilter(columnName, condition, (LocalDateTime) realValue.getValue()); + } else if (colClass == ZonedDateTime.class) { + filter = makeComparableRangeFilter(columnName, condition, (ZonedDateTime) realValue.getValue()); } else if (BigDecimal.class.isAssignableFrom(colClass)) { - filter = makeComparableRangeFilter(columnName, condition, new BigDecimal(value)); + filter = makeComparableRangeFilter(columnName, condition, (BigDecimal) realValue.getValue()); } else if (BigInteger.class.isAssignableFrom(colClass)) { - filter = makeComparableRangeFilter(columnName, condition, new BigInteger(value)); + filter = makeComparableRangeFilter(columnName, condition, (BigInteger) realValue.getValue()); } else if (io.deephaven.util.type.TypeUtils.isString(colClass)) { - final String stringValue = MatchFilter.ColumnTypeConvertorFactory.getConvertor(String.class, columnName) - .convertStringLiteral(value).toString(); - filter = makeComparableRangeFilter(columnName, condition, stringValue); + filter = makeComparableRangeFilter(columnName, condition, (String) realValue.getValue()); } else if (TypeUtils.isBoxedBoolean(colClass) || colClass == boolean.class) { - filter = makeComparableRangeFilter(columnName, condition, Boolean.valueOf(value)); + filter = makeComparableRangeFilter(columnName, condition, (Boolean) realValue.getValue()); } else { // The expression looks like a comparison of number, string, or boolean // but the type does not match (or the column type is misconfigured) if (expression != null) { - filter = ConditionFilter.createConditionFilter(expression, parserConfiguration); + try { + filter = ConditionFilter.createConditionFilter(expression, parserConfiguration); + } catch (final RuntimeException ignored) { + throw new IllegalArgumentException("RangeFilter does not support type " + + colClass.getSimpleName() + " for column " + columnName); + } } else { - throw new IllegalArgumentException("RangeConditionFilter does not support type " + throw new IllegalArgumentException("RangeFilter does not support type " + colClass.getSimpleName() + " for column " + columnName); } } @@ -190,52 +264,19 @@ public void init( filter.init(tableDefinition, compilationProcessor); } - public static char parseCharFilter(String value) { - if (value.startsWith("'") && value.endsWith("'") && value.length() == 3) { - return value.charAt(1); - } - if (value.startsWith("\"") && value.endsWith("\"") && value.length() == 3) { - return value.charAt(1); - } - return (char) Long.parseLong(value); - } - - public static byte parseByteFilter(String value) { - return Byte.parseByte(value); - } - - public static short parseShortFilter(String value) { - return Short.parseShort(value); - } - - public static int parseIntFilter(String value) { - return Integer.parseInt(value); - } - - public static long parseLongFilter(String value) { - return Long.parseLong(value); - } - - private static LongRangeFilter makeDateTimeRangeFilter(String columnName, Condition condition, String value) { + private static LongRangeFilter makeInstantRangeFilter(String columnName, Condition condition, long value) { switch (condition) { case LESS_THAN: - return new InstantRangeFilter(columnName, parseInstantNanos(value), Long.MIN_VALUE, true, false); + return new InstantRangeFilter(columnName, value, Long.MIN_VALUE, true, false); case LESS_THAN_OR_EQUAL: - return new InstantRangeFilter(columnName, parseInstantNanos(value), Long.MIN_VALUE, true, true); + return new InstantRangeFilter(columnName, value, Long.MIN_VALUE, true, true); case GREATER_THAN: - return new InstantRangeFilter(columnName, parseInstantNanos(value), Long.MAX_VALUE, false, true); + return new InstantRangeFilter(columnName, value, Long.MAX_VALUE, false, true); case GREATER_THAN_OR_EQUAL: - return new InstantRangeFilter(columnName, parseInstantNanos(value), Long.MAX_VALUE, true, true); + return new InstantRangeFilter(columnName, value, Long.MAX_VALUE, true, true); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); - } - } - - private static long parseInstantNanos(String value) { - if (value.startsWith("'") && value.endsWith("'")) { - return DateTimeUtils.epochNanos(DateTimeUtils.parseInstant(value.substring(1, value.length() - 1))); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } - return Long.parseLong(value); } private static SingleSidedComparableRangeFilter makeComparableRangeFilter(String columnName, Condition condition, @@ -250,7 +291,7 @@ private static SingleSidedComparableRangeFilter makeComparableRangeFilter(String case GREATER_THAN_OR_EQUAL: return new SingleSidedComparableRangeFilter(columnName, comparable, true, true); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } @@ -270,7 +311,7 @@ public WritableRowSet filterInverse( @Override public boolean isSimpleFilter() { - return true; + return filter.isSimpleFilter(); } @Override @@ -278,11 +319,12 @@ public void setRecomputeListener(RecomputeListener listener) {} @Override public WhereFilter copy() { - return new RangeConditionFilter(columnName, condition, value, expression, filter, parserConfiguration); + final WhereFilter innerCopy = filter == null ? null : filter.copy(); + return new RangeFilter(columnName, condition, value, expression, innerCopy, parserConfiguration); } @Override public String toString() { - return "RangeConditionFilter(" + columnName + " " + condition.description + " " + value + ")"; + return "RangeFilter(" + columnName + " " + condition.description + " " + value + ")"; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ShortRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ShortRangeFilter.java index 461707e7b7f..6257a2275b7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ShortRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ShortRangeFilter.java @@ -51,18 +51,18 @@ public ShortRangeFilter(String columnName, short val1, short val2, boolean lower } } - static WhereFilter makeShortRangeFilter(String columnName, Condition condition, String value) { + static WhereFilter makeShortRangeFilter(String columnName, Condition condition, short value) { switch (condition) { case LESS_THAN: - return lt(columnName, RangeConditionFilter.parseShortFilter(value)); + return lt(columnName, value); case LESS_THAN_OR_EQUAL: - return leq(columnName, RangeConditionFilter.parseShortFilter(value)); + return leq(columnName, value); case GREATER_THAN: - return gt(columnName, RangeConditionFilter.parseShortFilter(value)); + return gt(columnName, value); case GREATER_THAN_OR_EQUAL: - return geq(columnName, RangeConditionFilter.parseShortFilter(value)); + return geq(columnName, value); default: - throw new IllegalArgumentException("RangeConditionFilter does not support condition " + condition); + throw new IllegalArgumentException("RangeFilter does not support condition " + condition); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java index dbe6b627903..5c17955ccde 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java @@ -87,6 +87,8 @@ interface RecomputeListener { /** * Get the columns required by this select filter. + *

    + * This filter must already be initialized before calling this method. * * @return the columns used as input by this select filter. */ @@ -94,6 +96,8 @@ interface RecomputeListener { /** * Get the array columns required by this select filter. + *

    + * This filter must already be initialized before calling this method. * * @return the columns used as array input by this select filter. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterAdapter.java index 70c6c040974..8ec35409e0a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterAdapter.java @@ -241,8 +241,8 @@ public PreferredLhsColumnRhsVisitor(ColumnName lhs) { this.lhs = Objects.requireNonNull(lhs); } - // The String vs non-String cases are separated out, as it's necessary in the RangeConditionFilter case to - // wrap String literals with quotes (as that's what RangeConditionFilter expects wrt parsing). MatchFilter + // The String vs non-String cases are separated out, as it's necessary in the RangeFilter case to + // wrap String literals with quotes (as that's what RangeFilter expects wrt parsing). MatchFilter // allows us to pass in the already parsed Object (otherwise, if we were passing strValues we would need to // wrap them) @@ -368,34 +368,34 @@ public WhereFilter visit(RawString rawString) { return original(); } - private RangeConditionFilter range(Object rhsLiteral) { + private RangeFilter range(Object rhsLiteral) { // TODO(deephaven-core#3730): More efficient io.deephaven.api.filter.FilterComparison to RangeFilter final String rhsLiteralAsStr = rhsLiteral.toString(); switch (preferred.operator()) { case LESS_THAN: - return new RangeConditionFilter(lhs.name(), Condition.LESS_THAN, rhsLiteralAsStr); + return new RangeFilter(lhs.name(), Condition.LESS_THAN, rhsLiteralAsStr); case LESS_THAN_OR_EQUAL: - return new RangeConditionFilter(lhs.name(), Condition.LESS_THAN_OR_EQUAL, rhsLiteralAsStr); + return new RangeFilter(lhs.name(), Condition.LESS_THAN_OR_EQUAL, rhsLiteralAsStr); case GREATER_THAN: - return new RangeConditionFilter(lhs.name(), Condition.GREATER_THAN, rhsLiteralAsStr); + return new RangeFilter(lhs.name(), Condition.GREATER_THAN, rhsLiteralAsStr); case GREATER_THAN_OR_EQUAL: - return new RangeConditionFilter(lhs.name(), Condition.GREATER_THAN_OR_EQUAL, rhsLiteralAsStr); + return new RangeFilter(lhs.name(), Condition.GREATER_THAN_OR_EQUAL, rhsLiteralAsStr); } throw new IllegalStateException("Unexpected"); } - private RangeConditionFilter range(String rhsLiteral) { + private RangeFilter range(String rhsLiteral) { // TODO(deephaven-core#3730): More efficient io.deephaven.api.filter.FilterComparison to RangeFilter final String quotedRhsLiteral = '"' + rhsLiteral + '"'; switch (preferred.operator()) { case LESS_THAN: - return new RangeConditionFilter(lhs.name(), Condition.LESS_THAN, quotedRhsLiteral); + return new RangeFilter(lhs.name(), Condition.LESS_THAN, quotedRhsLiteral); case LESS_THAN_OR_EQUAL: - return new RangeConditionFilter(lhs.name(), Condition.LESS_THAN_OR_EQUAL, quotedRhsLiteral); + return new RangeFilter(lhs.name(), Condition.LESS_THAN_OR_EQUAL, quotedRhsLiteral); case GREATER_THAN: - return new RangeConditionFilter(lhs.name(), Condition.GREATER_THAN, quotedRhsLiteral); + return new RangeFilter(lhs.name(), Condition.GREATER_THAN, quotedRhsLiteral); case GREATER_THAN_OR_EQUAL: - return new RangeConditionFilter(lhs.name(), Condition.GREATER_THAN_OR_EQUAL, quotedRhsLiteral); + return new RangeFilter(lhs.name(), Condition.GREATER_THAN_OR_EQUAL, quotedRhsLiteral); } throw new IllegalStateException("Unexpected"); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterFactory.java index c021504f702..11ff3114cdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterFactory.java @@ -7,7 +7,6 @@ import io.deephaven.api.filter.FilterPattern; import io.deephaven.api.filter.FilterPattern.Mode; import io.deephaven.base.Pair; -import io.deephaven.engine.context.QueryScope; import io.deephaven.api.expression.AbstractExpressionFactory; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; @@ -21,6 +20,7 @@ import io.deephaven.io.logger.Logger; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.VisibleForTesting; +import io.deephaven.util.datastructures.CachingSupplier; import io.deephaven.util.text.SplitIgnoreQuotes; import org.jetbrains.annotations.NotNull; @@ -48,93 +48,28 @@ public class WhereFilterFactory { private static final ExpressionParser parser = new ExpressionParser<>(); static { - // == - // = - // != + // Each side may fit: (||) + // Supported ops: ==, =, !=, <, <=, >, >= parser.registerFactory(new AbstractExpressionFactory<>( - START_PTRN + "(" + ID_PTRN + ")\\s*(?:(?:={1,2})|(!=))\\s*(" + LITERAL_PTRN + ")" + END_PTRN) { + START_PTRN + "(?:(" + ID_PTRN + ")|(" + LITERAL_PTRN + "))\\s*((?:=|!|<|>)=?)\\s*(?:(" + ID_PTRN + ")|(" + + LITERAL_PTRN + "))" + END_PTRN) { @Override public WhereFilter getExpression(String expression, Matcher matcher, Object... args) { - final String columnName = matcher.group(1); - final boolean inverted = matcher.group(2) != null; - final String value = matcher.group(3); - - final FormulaParserConfiguration parserConfiguration = (FormulaParserConfiguration) args[0]; - if (isRowVariable(columnName)) { - log.debug().append("WhereFilterFactory creating ConditionFilter for expression: ") - .append(expression).endl(); - return ConditionFilter.createConditionFilter(expression, parserConfiguration); - } - log.debug().append("WhereFilterFactory creating MatchFilter for expression: ").append(expression) - .endl(); - return new MatchFilter( - MatchFilter.CaseSensitivity.MatchCase, - inverted ? MatchFilter.MatchType.Inverted : MatchFilter.MatchType.Regular, - columnName, - value); - } - }); - // == - // = - // != - parser.registerFactory(new AbstractExpressionFactory<>( - START_PTRN + "(" + ID_PTRN + ")\\s*(?:(?:={1,2})|(!=))\\s*(" + ID_PTRN + ")" + END_PTRN) { - @Override - public WhereFilter getExpression(String expression, Matcher matcher, Object... args) { - final String columnName = matcher.group(1); - final boolean inverted = matcher.group(2) != null; - final String paramName = matcher.group(3); - - final FormulaParserConfiguration parserConfiguration = (FormulaParserConfiguration) args[0]; + // LITERAL_PTRN has 5 groups; mostly non-capturing + final boolean leftIsId = matcher.group(1) != null; + final boolean rightIsId = matcher.group(8) != null; - if (isRowVariable(columnName)) { - log.debug().append("WhereFilterFactory creating ConditionFilter for expression: ") - .append(expression).endl(); - return ConditionFilter.createConditionFilter(expression, parserConfiguration); + if (!leftIsId && !rightIsId) { + return ConditionFilter.createConditionFilter(expression, (FormulaParserConfiguration) args[0]); } - try { - QueryScope.getParamValue(paramName); - } catch (QueryScope.MissingVariableException e) { - return ConditionFilter.createConditionFilter(expression, parserConfiguration); - } - log.debug().append("WhereFilterFactory creating MatchFilter for expression: ").append(expression) - .endl(); - return new MatchFilter( - MatchFilter.CaseSensitivity.MatchCase, - inverted ? MatchFilter.MatchType.Inverted : MatchFilter.MatchType.Regular, - columnName, - paramName); - } - }); + final boolean mirrored = !leftIsId; - // < - // <= - // > - // >= - parser.registerFactory(new AbstractExpressionFactory<>( - START_PTRN + "(" + ID_PTRN + ")\\s*([<>]=?)\\s*(" + LITERAL_PTRN + ")" + END_PTRN) { - @Override - public WhereFilter getExpression(String expression, Matcher matcher, Object... args) { - final FormulaParserConfiguration parserConfiguration = (FormulaParserConfiguration) args[0]; - final String columnName = matcher.group(1); - final String conditionString = matcher.group(2); - final String value = matcher.group(3); - if (isRowVariable(columnName)) { - log.debug().append("WhereFilterFactory creating ConditionFilter for expression: ") - .append(expression).endl(); - return ConditionFilter.createConditionFilter(expression, parserConfiguration); - } - try { - log.debug().append("WhereFilterFactory creating RangeConditionFilter for expression: ") - .append(expression).endl(); - return new RangeConditionFilter(columnName, conditionString, value, expression, - parserConfiguration); - } catch (Exception e) { - log.warn().append("WhereFilterFactory could not make RangeFilter for expression: ") - .append(expression).append(" due to ").append(e) - .append(" Creating ConditionFilter instead.").endl(); - return ConditionFilter.createConditionFilter(expression, parserConfiguration); - } + final String columnName = leftIsId ? matcher.group(1) : matcher.group(8); + final String op = matcher.group(7); + final String value = leftIsId ? rightIsId ? matcher.group(8) : matcher.group(9) : matcher.group(2); + + return getWhereFilterOneSideColumn( + expression, (FormulaParserConfiguration) args[0], columnName, op, value, mirrored); } }); @@ -202,6 +137,67 @@ public WhereFilter getExpression(String expression, Matcher matcher, Object... a }); } + private static @NotNull WhereFilter getWhereFilterOneSideColumn( + final String expression, + final FormulaParserConfiguration parserConfiguration, + final String columnName, + String op, + final String value, + boolean mirrored) { + + if (isRowVariable(columnName)) { + log.debug().append("WhereFilterFactory creating ConditionFilter for expression: ") + .append(expression).endl(); + return ConditionFilter.createConditionFilter(expression, parserConfiguration); + } + + boolean inverted = false; + switch (op) { + case "!=": + inverted = true; + case "=": + case "==": + log.debug().append("WhereFilterFactory creating MatchFilter for expression: ").append(expression) + .endl(); + return new MatchFilter( + new CachingSupplier<>(() -> (ConditionFilter) ConditionFilter.createConditionFilter(expression, + parserConfiguration)), + CaseSensitivity.MatchCase, + inverted ? MatchType.Inverted : MatchType.Regular, + columnName, + value); + + case "<": + case ">": + case "<=": + case ">=": + if (mirrored) { + switch (op) { + case "<": + op = ">"; + break; + case "<=": + op = ">="; + break; + case ">": + op = "<"; + break; + case ">=": + op = "<="; + break; + default: + throw new IllegalStateException("Unexpected operator: " + op); + } + } + log.debug().append("WhereFilterFactory creating RangeFilter for expression: ") + .append(expression).endl(); + return new RangeFilter(columnName, op, value, expression, parserConfiguration); + + default: + throw new IllegalStateException("Unexpected operator: " + op); + } + } + private static boolean isRowVariable(String columnName) { return columnName.equals("i") || columnName.equals("ii") || columnName.equals("k"); } @@ -396,7 +392,7 @@ public static WhereFilter stringContainsFilter( boolean removeQuotes, String... values) { final String value = - constructStringContainsRegex(values, matchType, internalDisjunctive, removeQuotes, columnName); + constructStringContainsRegex(values, matchType, internalDisjunctive, removeQuotes); return WhereFilterAdapter.of(FilterPattern.of( ColumnName.of(columnName), Pattern.compile(value, sensitivity == CaseSensitivity.IgnoreCase ? Pattern.CASE_INSENSITIVE : 0), @@ -408,14 +404,13 @@ private static String constructStringContainsRegex( String[] values, MatchType matchType, boolean internalDisjunctive, - boolean removeQuotes, - String columnName) { + boolean removeQuotes) { if (values == null || values.length == 0) { throw new IllegalArgumentException( "constructStringContainsRegex must be called with at least one value parameter"); } final MatchFilter.ColumnTypeConvertor converter = removeQuotes - ? MatchFilter.ColumnTypeConvertorFactory.getConvertor(String.class, columnName) + ? MatchFilter.ColumnTypeConvertorFactory.getConvertor(String.class) : null; final String regex; final Stream valueStream = Arrays.stream(values) diff --git a/engine/table/src/main/java/io/deephaven/gui/table/filters/Condition.java b/engine/table/src/main/java/io/deephaven/gui/table/filters/Condition.java index 0c29d40e598..8c7a3c6ab51 100644 --- a/engine/table/src/main/java/io/deephaven/gui/table/filters/Condition.java +++ b/engine/table/src/main/java/io/deephaven/gui/table/filters/Condition.java @@ -20,16 +20,46 @@ public enum Condition { NOT_INCLUDES_MATCH_CASE("not includes (casesen)", false), // Numbers and Dates - LESS_THAN("less than", false), - GREATER_THAN("greater than", false), - LESS_THAN_OR_EQUAL("less than or equal to", false), - GREATER_THAN_OR_EQUAL("greater than or equal to", false), + LESS_THAN("less than", false) { + @Override + public Condition mirror() { + return Condition.GREATER_THAN; + } + }, + GREATER_THAN("greater than", false) { + @Override + public Condition mirror() { + return Condition.LESS_THAN; + } + }, + LESS_THAN_OR_EQUAL("less than or equal to", false) { + @Override + public Condition mirror() { + return Condition.GREATER_THAN_OR_EQUAL; + } + }, + GREATER_THAN_OR_EQUAL("greater than or equal to", false) { + @Override + public Condition mirror() { + return Condition.LESS_THAN_OR_EQUAL; + } + }, // Numbers EQUALS_ABS("equals (abs)", true), NOT_EQUALS_ABS("not equals (abs)", false), - LESS_THAN_ABS("less than (abs)", false), - GREATER_THAN_ABS("greater than (abs)", false), + LESS_THAN_ABS("less than (abs)", false) { + @Override + public Condition mirror() { + return Condition.GREATER_THAN_ABS; + } + }, + GREATER_THAN_ABS("greater than (abs)", false) { + @Override + public Condition mirror() { + return Condition.LESS_THAN_ABS; + } + }, // Lists INCLUDED_IN("included in list", true), @@ -45,4 +75,8 @@ public enum Condition { this.description = description; this.defaultOr = defaultOr; } + + public Condition mirror() { + return this; + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java index 62c25a31b03..1e40d00084e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java @@ -664,7 +664,7 @@ private String createWhereFilter(Random random) { switch (columnTypes[colNum].getSimpleName()) { case "Instant": - filter.append(colName).append(" > ").append(random.nextInt(1000) * 1_000_000_000L); + filter.append(colName).append(" > '").append(random.nextInt(1000) * 1_000_000_000L).append("'"); break; case "String": diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java index dafc65cfe97..acfa9686b6f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java @@ -32,20 +32,25 @@ import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.engine.util.TableTools; +import io.deephaven.gui.table.filters.Condition; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import io.deephaven.time.DateTimeUtils; import io.deephaven.util.QueryConstants; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.ReflexiveUse; +import io.deephaven.util.datastructures.CachingSupplier; import junit.framework.TestCase; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableObject; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import java.math.BigDecimal; import java.math.BigInteger; import java.time.Instant; +import java.time.ZonedDateTime; import java.util.Collections; import java.util.Random; import java.util.concurrent.CountDownLatch; @@ -1152,6 +1157,19 @@ public void testComparableBinarySearch() { QueryScope.addParam("nine", null); } + @Test + public void testZonedDateRangeFilter() { + final ZonedDateTime startTime = DateTimeUtils.parseZonedDateTime("2021-04-23T09:30 NY"); + final ZonedDateTime[] array = new ZonedDateTime[10]; + for (int ii = 0; ii < array.length; ++ii) { + array[ii] = DateTimeUtils.plus(startTime, 60_000_000_000L * ii); + } + final Table table = TableTools.newTable(col("ZDT", array)); + showWithRowSet(table); + + testRangeFilterHelper(table, "ZDT", array[5]); + } + @Test public void testInstantRangeFilter() { final Instant startTime = DateTimeUtils.parseInstant("2021-04-23T09:30 NY"); @@ -1162,11 +1180,7 @@ public void testInstantRangeFilter() { final Table table = TableTools.newTable(col("DT", array)); showWithRowSet(table); - final Table sorted = table.sort("DT"); - final Table backwards = table.sort("DT"); - - assertTableEquals(sorted.where("DT < '" + array[5] + "'"), sorted.where("ii < 5")); - assertTableEquals(backwards.where("DT < '" + array[5] + "'"), backwards.where("ii < 5")); + testRangeFilterHelper(table, "DT", array[5]); } @Test @@ -1184,22 +1198,26 @@ public void testCharRangeFilter() { final Table table = TableTools.newTable(charCol("CH", array)); showWithRowSet(table); - final Table sorted = table.sort("CH"); - final Table backwards = table.sort("CH"); + testRangeFilterHelper(table, "CH", array[5]); + } + + private void testRangeFilterHelper(Table table, String name, T mid) { + final Table sorted = table.sort(name); + final Table backwards = table.sort(name); showWithRowSet(sorted); - log.debug().append("Pivot: " + array[5]).endl(); + log.debug().append("Pivot: " + mid).endl(); - final Table rangeFiltered = sorted.where("CH < '" + array[5] + "'"); - final Table standardFiltered = sorted.where("'" + array[5] + "' > CH"); + final Table rangeFiltered = sorted.where(name + " < '" + mid + "'"); + final Table standardFiltered = sorted.where("'" + mid + "' > " + name); showWithRowSet(rangeFiltered); showWithRowSet(standardFiltered); assertTableEquals(rangeFiltered, standardFiltered); - assertTableEquals(backwards.where("CH < '" + array[5] + "'"), backwards.where("'" + array[5] + "' > CH")); - assertTableEquals(backwards.where("CH <= '" + array[5] + "'"), backwards.where("'" + array[5] + "' >= CH")); - assertTableEquals(backwards.where("CH > '" + array[5] + "'"), backwards.where("'" + array[5] + "' < CH")); - assertTableEquals(backwards.where("CH >= '" + array[5] + "'"), backwards.where("'" + array[5] + "' <= CH")); + assertTableEquals(backwards.where(name + " < '" + mid + "'"), backwards.where("'" + mid + "' > " + name)); + assertTableEquals(backwards.where(name + " <= '" + mid + "'"), backwards.where("'" + mid + "' >= " + name)); + assertTableEquals(backwards.where(name + " > '" + mid + "'"), backwards.where("'" + mid + "' < " + name)); + assertTableEquals(backwards.where(name + " >= '" + mid + "'"), backwards.where("'" + mid + "' <= " + name)); } @Test @@ -1351,4 +1369,262 @@ public void testFilterErrorUpdate() { // The where result should have failed, because the filter expression is invalid for the new data. Assert.eqTrue(whereResult.isFailed(), "whereResult.isFailed()"); } + + @Test + public void testMatchFilterFallback() { + final Table table = emptyTable(10).update("X=i"); + ExecutionContext.getContext().getQueryScope().putParam("var1", 10); + ExecutionContext.getContext().getQueryScope().putParam("var2", 20); + + final MutableBoolean called = new MutableBoolean(false); + final MatchFilter filter = new MatchFilter( + new CachingSupplier<>(() -> { + called.setValue(true); + return (ConditionFilter) ConditionFilter.createConditionFilter("var1 != var2"); + }), + MatchFilter.CaseSensitivity.IgnoreCase, MatchFilter.MatchType.Inverted, "var1", "var2"); + + final Table result = table.where(filter); + assertTableEquals(table, result); + + Assert.eqTrue(called.booleanValue(), "called.booleanValue()"); + } + + @Test + public void testRangeFilterFallback() { + final Table table = emptyTable(10).update("X=i"); + ExecutionContext.getContext().getQueryScope().putParam("var1", 10); + ExecutionContext.getContext().getQueryScope().putParam("var2", 20); + + final RangeFilter filter = new RangeFilter( + "0", Condition.LESS_THAN, "var2", "0 < var2", FormulaParserConfiguration.parser); + + final Table result = table.where(filter); + assertTableEquals(table, result); + + final WhereFilter realFilter = filter.getRealFilter(); + Assert.eqTrue(realFilter instanceof ConditionFilter, "realFilter instanceof ConditionFilter"); + } + + @Test + public void testEnsureColumnsTakePrecedence() { + final Table table = emptyTable(10).update("X=i", "Y=i%2"); + ExecutionContext.getContext().getQueryScope().putParam("Y", 5); + + { + final Table r1 = table.where("X == Y"); + final Table r2 = table.where("Y == X"); + Assert.equals(r1.getRowSet(), "r1.getRowSet()", RowSetFactory.flat(2)); + assertTableEquals(r1, r2); + } + + { + final Table r1 = table.where("X >= Y"); + final Table r2 = table.where("Y <= X"); + Assert.equals(r1.getRowSet(), "r1.getRowSet()", RowSetFactory.flat(10)); + assertTableEquals(r1, r2); + } + + { + final Table r1 = table.where("X > Y"); + final Table r2 = table.where("Y < X"); + Assert.equals(r1.getRowSet(), "r1.getRowSet()", RowSetFactory.fromRange(2, 9)); + assertTableEquals(r1, r2); + } + + { + final Table r1 = table.where("X < Y"); + final Table r2 = table.where("Y > X"); + Assert.equals(r1.getRowSet(), "r1.getRowSet()", RowSetFactory.empty()); + assertTableEquals(r1, r2); + } + + { + final Table r1 = table.where("X <= Y"); + final Table r2 = table.where("Y >= X"); + Assert.equals(r1.getRowSet(), "r1.getRowSet()", RowSetFactory.flat(2)); + assertTableEquals(r1, r2); + } + } + + @Test + @Ignore + public void testEnsureColumnArraysTakePrecedence() { + // TODO: column arrays aren't well supported in match arrays and this example's where filter fails to compile + final Table table = emptyTable(10).update("X=i", "Y=new int[]{1, 5, 9}"); + ExecutionContext.getContext().getQueryScope().putParam("Y_", new int[] {0, 4, 8}); + + final Table result = table.where("X == Y_[1]"); + Assert.equals(result.getRowSet(), "result.getRowSet()", RowSetFactory.fromKeys(5)); + + // check that the mirror matches the expected result + final Table mResult = table.where("Y_[1] == X"); + assertTableEquals(result, mResult); + } + + @Test + public void testIntToByteCoercion() { + final Table table = emptyTable(11).update("X = ii % 2 == 0 ? (byte) ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", byte.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testIntToShortCoercion() { + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? (short) ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", short.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testLongToIntCoercion() { + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? (int) ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", int.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_LONG); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5L); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testIntToLongCoercion() { + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", long.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testIntToFloatCoercion() { + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? (float) ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", float.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testIntToDoubleCoercion() { + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? (double) ii : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", double.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + } + + @Test + public void testBigIntegerCoercion() { + ExecutionContext.getContext().getQueryLibrary().importClass(BigInteger.class); + + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? BigInteger.valueOf(ii) : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", BigInteger.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + + // let's also test BigDecimal -> BigInteger conversion; note that conversion does not round + ExecutionContext.getContext().getQueryScope().putParam("bd_5", BigDecimal.valueOf(5.8)); + final Table bd_result = table.where("X >= bd_5"); + assertTableEquals(range_result, bd_result); + } + + @Test + public void testBigDecimalCoercion() { + ExecutionContext.getContext().getQueryLibrary().importClass(BigDecimal.class); + + final Table table = emptyTable(11).update("X= ii % 2 == 0 ? BigDecimal.valueOf(ii) : null"); + final Class colType = table.getDefinition().getColumn("X").getDataType(); + Assert.eq(colType, "colType", BigDecimal.class); + + ExecutionContext.getContext().getQueryScope().putParam("real_null", null); + ExecutionContext.getContext().getQueryScope().putParam("val_null", QueryConstants.NULL_INT); + ExecutionContext.getContext().getQueryScope().putParam("val_5", 5); + + final Table real_null_result = table.where("X == real_null"); + final Table null_result = table.where("X == val_null"); + Assert.eq(null_result.size(), "null_result.size()", 5); + assertTableEquals(real_null_result, null_result); + + final Table range_result = table.where("X >= val_5"); + Assert.eq(range_result.size(), "range_result.size()", 3); + + // let's also test BigInteger -> BigDecimal conversion + ExecutionContext.getContext().getQueryScope().putParam("bi_5", BigInteger.valueOf(5)); + final Table bi_result = table.where("X >= bi_5"); + assertTableEquals(range_result, bi_result); + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterTest.java index 4ce85050b27..f1900291d51 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterTest.java @@ -73,13 +73,13 @@ public void testEq() { regular(FilterComparison.eq(V42, FOO), MatchFilter.class, "Foo in [42]"); regular(FilterComparison.eq(FOO, HELLO), MatchFilter.class, "Foo in [Hello]"); regular(FilterComparison.eq(HELLO, FOO), MatchFilter.class, "Foo in [Hello]"); - regular(FilterComparison.eq(FOO, BAR), ConditionFilter.class, "Foo == Bar"); + regular(FilterComparison.eq(FOO, BAR), MatchFilter.class, "Foo in [Bar]"); inverse(FilterComparison.eq(FOO, V42), MatchFilter.class, "Foo not in [42]"); inverse(FilterComparison.eq(V42, FOO), MatchFilter.class, "Foo not in [42]"); inverse(FilterComparison.eq(FOO, HELLO), MatchFilter.class, "Foo not in [Hello]"); inverse(FilterComparison.eq(HELLO, FOO), MatchFilter.class, "Foo not in [Hello]"); - inverse(FilterComparison.eq(FOO, BAR), ConditionFilter.class, "Foo != Bar"); + inverse(FilterComparison.eq(FOO, BAR), MatchFilter.class, "Foo not in [Bar]"); } public void testNeq() { @@ -87,101 +87,101 @@ public void testNeq() { regular(FilterComparison.neq(V42, FOO), MatchFilter.class, "Foo not in [42]"); regular(FilterComparison.neq(FOO, HELLO), MatchFilter.class, "Foo not in [Hello]"); regular(FilterComparison.neq(HELLO, FOO), MatchFilter.class, "Foo not in [Hello]"); - regular(FilterComparison.neq(FOO, BAR), ConditionFilter.class, "Foo != Bar"); + regular(FilterComparison.neq(FOO, BAR), MatchFilter.class, "Foo not in [Bar]"); inverse(FilterComparison.neq(FOO, V42), MatchFilter.class, "Foo in [42]"); inverse(FilterComparison.neq(V42, FOO), MatchFilter.class, "Foo in [42]"); inverse(FilterComparison.neq(FOO, HELLO), MatchFilter.class, "Foo in [Hello]"); inverse(FilterComparison.neq(HELLO, FOO), MatchFilter.class, "Foo in [Hello]"); - inverse(FilterComparison.neq(FOO, BAR), ConditionFilter.class, "Foo == Bar"); + inverse(FilterComparison.neq(FOO, BAR), MatchFilter.class, "Foo in [Bar]"); } public void testGt() { - regular(FilterComparison.gt(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than 42)"); - regular(FilterComparison.gt(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than 42)"); - regular(FilterComparison.gt(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than \"Hello\")"); - regular(FilterComparison.gt(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than \"Hello\")"); - regular(FilterComparison.gt(FOO, BAR), ConditionFilter.class, "Foo > Bar"); - - inverse(FilterComparison.gt(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to 42)"); - inverse(FilterComparison.gt(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to 42)"); - inverse(FilterComparison.gt(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to \"Hello\")"); - inverse(FilterComparison.gt(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to \"Hello\")"); - inverse(FilterComparison.gt(FOO, BAR), ConditionFilter.class, "Foo <= Bar"); + regular(FilterComparison.gt(FOO, V42), RangeFilter.class, + "RangeFilter(Foo greater than 42)"); + regular(FilterComparison.gt(V42, FOO), RangeFilter.class, + "RangeFilter(Foo less than 42)"); + regular(FilterComparison.gt(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo greater than \"Hello\")"); + regular(FilterComparison.gt(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo less than \"Hello\")"); + regular(FilterComparison.gt(FOO, BAR), RangeFilter.class, "RangeFilter(Foo greater than Bar)"); + + inverse(FilterComparison.gt(FOO, V42), RangeFilter.class, + "RangeFilter(Foo less than or equal to 42)"); + inverse(FilterComparison.gt(V42, FOO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to 42)"); + inverse(FilterComparison.gt(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo less than or equal to \"Hello\")"); + inverse(FilterComparison.gt(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to \"Hello\")"); + inverse(FilterComparison.gt(FOO, BAR), RangeFilter.class, "RangeFilter(Foo less than or equal to Bar)"); } public void testGte() { - regular(FilterComparison.geq(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to 42)"); - regular(FilterComparison.geq(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to 42)"); - regular(FilterComparison.geq(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to \"Hello\")"); - regular(FilterComparison.geq(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to \"Hello\")"); - regular(FilterComparison.geq(FOO, BAR), ConditionFilter.class, "Foo >= Bar"); - - inverse(FilterComparison.geq(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than 42)"); - inverse(FilterComparison.geq(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than 42)"); - inverse(FilterComparison.geq(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than \"Hello\")"); - inverse(FilterComparison.geq(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than \"Hello\")"); - inverse(FilterComparison.geq(FOO, BAR), ConditionFilter.class, "Foo < Bar"); + regular(FilterComparison.geq(FOO, V42), RangeFilter.class, + "RangeFilter(Foo greater than or equal to 42)"); + regular(FilterComparison.geq(V42, FOO), RangeFilter.class, + "RangeFilter(Foo less than or equal to 42)"); + regular(FilterComparison.geq(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to \"Hello\")"); + regular(FilterComparison.geq(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo less than or equal to \"Hello\")"); + regular(FilterComparison.geq(FOO, BAR), RangeFilter.class, "RangeFilter(Foo greater than or equal to Bar)"); + + inverse(FilterComparison.geq(FOO, V42), RangeFilter.class, + "RangeFilter(Foo less than 42)"); + inverse(FilterComparison.geq(V42, FOO), RangeFilter.class, + "RangeFilter(Foo greater than 42)"); + inverse(FilterComparison.geq(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo less than \"Hello\")"); + inverse(FilterComparison.geq(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo greater than \"Hello\")"); + inverse(FilterComparison.geq(FOO, BAR), RangeFilter.class, "RangeFilter(Foo less than Bar)"); } public void testLt() { - regular(FilterComparison.lt(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than 42)"); - regular(FilterComparison.lt(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than 42)"); - regular(FilterComparison.lt(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than \"Hello\")"); - regular(FilterComparison.lt(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than \"Hello\")"); - regular(FilterComparison.lt(FOO, BAR), ConditionFilter.class, "Foo < Bar"); - - inverse(FilterComparison.lt(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to 42)"); - inverse(FilterComparison.lt(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to 42)"); - inverse(FilterComparison.lt(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to \"Hello\")"); - inverse(FilterComparison.lt(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to \"Hello\")"); - inverse(FilterComparison.lt(FOO, BAR), ConditionFilter.class, "Foo >= Bar"); + regular(FilterComparison.lt(FOO, V42), RangeFilter.class, + "RangeFilter(Foo less than 42)"); + regular(FilterComparison.lt(V42, FOO), RangeFilter.class, + "RangeFilter(Foo greater than 42)"); + regular(FilterComparison.lt(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo less than \"Hello\")"); + regular(FilterComparison.lt(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo greater than \"Hello\")"); + regular(FilterComparison.lt(FOO, BAR), RangeFilter.class, "RangeFilter(Foo less than Bar)"); + + inverse(FilterComparison.lt(FOO, V42), RangeFilter.class, + "RangeFilter(Foo greater than or equal to 42)"); + inverse(FilterComparison.lt(V42, FOO), RangeFilter.class, + "RangeFilter(Foo less than or equal to 42)"); + inverse(FilterComparison.lt(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to \"Hello\")"); + inverse(FilterComparison.lt(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo less than or equal to \"Hello\")"); + inverse(FilterComparison.lt(FOO, BAR), RangeFilter.class, "RangeFilter(Foo greater than or equal to Bar)"); } public void testLte() { - regular(FilterComparison.leq(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to 42)"); - regular(FilterComparison.leq(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to 42)"); - regular(FilterComparison.leq(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than or equal to \"Hello\")"); - regular(FilterComparison.leq(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than or equal to \"Hello\")"); - regular(FilterComparison.leq(FOO, BAR), ConditionFilter.class, "Foo <= Bar"); - - inverse(FilterComparison.leq(FOO, V42), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than 42)"); - inverse(FilterComparison.leq(V42, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than 42)"); - inverse(FilterComparison.leq(FOO, HELLO), RangeConditionFilter.class, - "RangeConditionFilter(Foo greater than \"Hello\")"); - inverse(FilterComparison.leq(HELLO, FOO), RangeConditionFilter.class, - "RangeConditionFilter(Foo less than \"Hello\")"); - inverse(FilterComparison.leq(FOO, BAR), ConditionFilter.class, "Foo > Bar"); + regular(FilterComparison.leq(FOO, V42), RangeFilter.class, + "RangeFilter(Foo less than or equal to 42)"); + regular(FilterComparison.leq(V42, FOO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to 42)"); + regular(FilterComparison.leq(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo less than or equal to \"Hello\")"); + regular(FilterComparison.leq(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo greater than or equal to \"Hello\")"); + regular(FilterComparison.leq(FOO, BAR), RangeFilter.class, "RangeFilter(Foo less than or equal to Bar)"); + + inverse(FilterComparison.leq(FOO, V42), RangeFilter.class, + "RangeFilter(Foo greater than 42)"); + inverse(FilterComparison.leq(V42, FOO), RangeFilter.class, + "RangeFilter(Foo less than 42)"); + inverse(FilterComparison.leq(FOO, HELLO), RangeFilter.class, + "RangeFilter(Foo greater than \"Hello\")"); + inverse(FilterComparison.leq(HELLO, FOO), RangeFilter.class, + "RangeFilter(Foo less than \"Hello\")"); + inverse(FilterComparison.leq(FOO, BAR), RangeFilter.class, "RangeFilter(Foo greater than Bar)"); } public void testFunction() { @@ -303,8 +303,8 @@ public void testInLiteralsDifferentTypes() { public void testInSingleNotLiteral() { final FilterIn in = FilterIn.of(FOO, BAR); - regular(in, ConditionFilter.class, "Foo == Bar"); - inverse(in, ConditionFilter.class, "Foo != Bar"); + regular(in, MatchFilter.class, "Foo in [Bar]"); + inverse(in, MatchFilter.class, "Foo not in [Bar]"); } diff --git a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java index 36824bea4b7..367bddda671 100644 --- a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java +++ b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java @@ -81,6 +81,13 @@ public class DateTimeUtils { private static final Pattern DATE_TZ_PATTERN = Pattern.compile( "(?[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9])(?[tT]?) (?[a-zA-Z_/]+)"); + /** + * Matches dates without time zones. + */ + private static final Pattern LOCAL_DATE_PATTERN = Pattern.compile( + "(?[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9])(?[tT]?)"); + + /** * Matches time durations. */ @@ -4810,6 +4817,65 @@ public static Instant parseInstantQuiet(@Nullable final String s) { } } + /** + * Parses the string argument as a {@link LocalDateTime}. + *

    + * Date time strings are formatted according to the ISO 8601 date time format + * {@code yyyy-MM-ddThh:mm:ss[.SSSSSSSSS]} and others. + * + * @param s date time string + * @return a {@link LocalDateTime} represented by the input string + * @throws DateTimeParseException if the string cannot be parsed + */ + @ScriptApi + @NotNull + public static LocalDateTime parseLocalDateTime(@NotNull final String s) { + // noinspection ConstantConditions + if (s == null) { + throw new DateTimeParseException("Cannot parse local date time (null): " + s); + } + + try { + return LocalDateTime.parse(s); + } catch (java.time.format.DateTimeParseException e) { + // ignore + } + + try { + final Matcher dtMatcher = LOCAL_DATE_PATTERN.matcher(s); + if (dtMatcher.matches()) { + final String dateString = dtMatcher.group("date"); + return LocalDate.parse(dateString, FORMATTER_ISO_LOCAL_DATE).atTime(LocalTime.of(0, 0)); + } + return LocalDateTime.parse(s, FORMATTER_ISO_LOCAL_DATE_TIME); + } catch (Exception ex) { + throw new DateTimeParseException("Cannot parse local date time: " + s, ex); + } + } + + /** + * Parses the string argument as a {@link LocalDateTime}. + *

    + * Date time strings are formatted according to the ISO 8601 date time format + * {@code yyyy-MM-ddThh:mm:ss[.SSSSSSSSS]} and others. + * + * @param s date time string + * @return a {@link LocalDateTime} represented by the input string, or {@code null} if the string can not be parsed + */ + @ScriptApi + @Nullable + public static LocalDateTime parseLocalDateTimeQuiet(@Nullable final String s) { + if (s == null || s.length() <= 1) { + return null; + } + + try { + return parseLocalDateTime(s); + } catch (Exception e) { + return null; + } + } + /** * Parses the string argument as a {@link ZonedDateTime}. *

    diff --git a/engine/time/src/test/java/io/deephaven/time/TestDateTimeUtils.java b/engine/time/src/test/java/io/deephaven/time/TestDateTimeUtils.java index 8515f0f94bb..7dcad0ad771 100644 --- a/engine/time/src/test/java/io/deephaven/time/TestDateTimeUtils.java +++ b/engine/time/src/test/java/io/deephaven/time/TestDateTimeUtils.java @@ -566,6 +566,138 @@ public void testParseInstantQuiet() { TestCase.assertEquals(dt1s, DateTimeUtils.parseInstantQuiet(Long.toString(seconds))); } + public void testParseLocalDateTime() { + final String[] roots = { + "2010-01-01T12:11", + "2010-01-01T12:00:02", + "2010-01-01T12:00:00.1", + "2010-01-01T12:00:00.123", + "2010-01-01T12:00:00.123", + "2010-01-01T12:00:00.123456789", + }; + + for (String root : roots) { + final LocalDateTime ldt = LocalDateTime.parse(root); + TestCase.assertEquals("LocalDateTime string: " + root, ldt, DateTimeUtils.parseLocalDateTime(root)); + } + + final String[] uglyRoots = { + "2023-04-30", + "2023-04-30T", + "2023-04-30t", + "2023-04-30T9:30:00", + "2023-4-3T9:3:6", + "2023-4-3T9:3", + "2023-4-3T9:3:6.1", + "2023-4-3T9:3:6.123", + "2023-4-3T9:3:6.123456789", + }; + + final LocalDateTime[] uglyLDTs = { + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 9, 30, 0), + LocalDateTime.of(2023, 4, 3, 9, 3, 6), + LocalDateTime.of(2023, 4, 3, 9, 3, 0), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 100_000_000), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 123_000_000), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 123456789), + }; + + for (int i = 0; i < uglyRoots.length; i++) { + final String root = uglyRoots[i]; + final LocalDateTime ldt = uglyLDTs[i]; + TestCase.assertEquals("LocalDateTime string: " + root, ldt, DateTimeUtils.parseLocalDateTime(root)); + } + + try { + DateTimeUtils.parseLocalDateTime("JUNK"); + TestCase.fail("Should throw an exception"); + } catch (Exception ex) { + // pass + } + + try { + DateTimeUtils.parseLocalDateTime("2010-01-01JUNK12:11"); + TestCase.fail("Should throw an exception"); + } catch (Exception ex) { + // pass + } + + try { + DateTimeUtils.parseLocalDateTime("2010-01-01T12:11 JUNK"); + TestCase.fail("Should throw an exception"); + } catch (Exception ex) { + // pass + } + + try { + // noinspection ConstantConditions + DateTimeUtils.parseLocalDateTime(null); + TestCase.fail("Should throw an exception"); + } catch (Exception ex) { + // pass + } + + final String iso8601 = "2022-04-26T00:30:31.087360"; + assertEquals(LocalDateTime.parse(iso8601), DateTimeUtils.parseLocalDateTime(iso8601)); + } + + public void testParseLocalDateTimeQuiet() { + final String[] roots = { + "2010-01-01T12:11", + "2010-01-01T12:00:02", + "2010-01-01T12:00:00.1", + "2010-01-01T12:00:00.123", + "2010-01-01T12:00:00.123", + "2010-01-01T12:00:00.123456789", + }; + + for (String root : roots) { + final LocalDateTime ldt = LocalDateTime.parse(root); + TestCase.assertEquals("LocalDateTime string: " + root, ldt, DateTimeUtils.parseLocalDateTime(root)); + } + + final String[] uglyRoots = { + "2023-04-30", + "2023-04-30T", + "2023-04-30t", + "2023-04-30T9:30:00", + "2023-4-3T9:3:6", + "2023-4-3T9:3", + "2023-4-3T9:3:6.1", + "2023-4-3T9:3:6.123", + "2023-4-3T9:3:6.123456789", + }; + + final LocalDateTime[] uglyLDTs = { + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 0, 0), + LocalDateTime.of(2023, 4, 30, 9, 30, 0), + LocalDateTime.of(2023, 4, 3, 9, 3, 6), + LocalDateTime.of(2023, 4, 3, 9, 3, 0), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 100_000_000), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 123_000_000), + LocalDateTime.of(2023, 4, 3, 9, 3, 6, 123456789), + }; + + for (int i = 0; i < uglyRoots.length; i++) { + final String root = uglyRoots[i]; + final LocalDateTime ldt = uglyLDTs[i]; + TestCase.assertEquals("LocalDateTime string: " + root, ldt, DateTimeUtils.parseLocalDateTime(root)); + } + + TestCase.assertNull(DateTimeUtils.parseLocalDateTimeQuiet("JUNK")); + TestCase.assertNull(DateTimeUtils.parseLocalDateTimeQuiet("2010-01-01JUNK12:11")); + TestCase.assertNull(DateTimeUtils.parseLocalDateTimeQuiet("2010-01-01T12:11 JUNK")); + TestCase.assertNull(DateTimeUtils.parseLocalDateTimeQuiet(null)); + + final String iso8601 = "2022-04-26T00:30:31.087360"; + assertEquals(LocalDateTime.parse(iso8601), DateTimeUtils.parseLocalDateTime(iso8601)); + } + public void testParseZonedDateTime() { final String[] tzs = { "NY", diff --git a/server/src/main/java/io/deephaven/server/table/ops/filter/FilterFactory.java b/server/src/main/java/io/deephaven/server/table/ops/filter/FilterFactory.java index 7b5edc29c9a..4ad44c629ca 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/filter/FilterFactory.java +++ b/server/src/main/java/io/deephaven/server/table/ops/filter/FilterFactory.java @@ -11,7 +11,7 @@ import io.deephaven.engine.table.impl.select.DisjunctiveFilter; import io.deephaven.engine.table.impl.select.FormulaParserConfiguration; import io.deephaven.engine.table.impl.select.MatchFilter; -import io.deephaven.engine.table.impl.select.RangeConditionFilter; +import io.deephaven.engine.table.impl.select.RangeFilter; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.select.WhereFilterFactory; import io.deephaven.engine.table.impl.select.WhereNoneFilter; @@ -145,7 +145,7 @@ private WhereFilter generateNumericConditionFilter(CompareCondition.CompareOpera default: throw new IllegalStateException("Range filter can't handle literal type " + value.getValueCase()); } - return new RangeConditionFilter(columName, rangeCondition(operation, invert), valueString, null, + return new RangeFilter(columName, rangeCondition(operation, invert), valueString, null, FormulaParserConfiguration.parser); } From c2e2424a84ddfc9c191d00dc5b435bf0315667b7 Mon Sep 17 00:00:00 2001 From: Colin Alworth Date: Tue, 25 Jun 2024 12:49:45 -0500 Subject: [PATCH 16/18] fix: Only update the redir index on the first column (#5659) JS API full table subscriptions currently spend more time than they should updating redirection indexes for large snapshots. This patch is a small step to reduce the work done when receiving these large payloads. Fixes #5658 --- .../web/client/api/subscription/SubscriptionTableData.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/SubscriptionTableData.java b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/SubscriptionTableData.java index 84e4dc1c758..7c08d330e91 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/SubscriptionTableData.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/SubscriptionTableData.java @@ -68,6 +68,7 @@ public TableData handleSnapshot(TableSnapshot snapshot) { long includedRowCount = snapshot.getIncludedRows().size(); RangeSet destination = freeRows(includedRowCount); + boolean indexUpdated = false; for (int index = 0; index < dataColumns.length; index++) { ColumnData dataColumn = dataColumns[index]; @@ -89,10 +90,14 @@ public TableData handleSnapshot(TableSnapshot snapshot) { while (indexIter.hasNext()) { assert destIter.hasNext(); long dest = destIter.nextLong(); - redirectedIndexes.put(indexIter.nextLong(), dest); + long nextIndex = indexIter.nextLong(); + if (!indexUpdated) { + redirectedIndexes.put(nextIndex, dest); + } arrayCopy.copyTo(localCopy, dest, dataColumn.getData(), j++); } assert !destIter.hasNext(); + indexUpdated = true; } return notifyUpdates(index, RangeSet.empty(), RangeSet.empty()); From a7d4ea885ddf208f29c932cdba90e9a4fc98c2f1 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Tue, 25 Jun 2024 19:02:51 +0000 Subject: [PATCH 17/18] Update web version 0.83.0 (#5667) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Release notes https://github.com/deephaven/web-client-ui/releases/tag/v0.83.0 ## [0.83.0](https://github.com/deephaven/web-client-ui/compare/v0.82.0...v0.83.0) (2024-06-25) ### ⚠ BREAKING CHANGES * ComboBox component has been replaced. To migrate to new version: - Passing children is used instead of `options` prop to define dropdown items. For cases where option value and display are the same, passing an array of values as `children` will work. For cases where value and display differ, `Item` elements must be passed as children. e.g. `{display}` e.g. ```typescript // values will be used for display + value const items = useMemo( () => ['Aaa', 'Bbb', 'Ccc'], [] ) {items} ``` ```typescript Aaa Bbb Ccc ``` - The `spellcheck=false` prop is no longer supported or needed - `searchPlaceholder` and `inputPlaceholder` props are no longer supported and should be omitted. There is an optional `description` prop for cases where a descriptive label is desired. There is also a `label` prop for the primary component label. ### Features * ComboBox - @deephaven/components ([#2067](https://github.com/deephaven/web-client-ui/issues/2067)) ([640e002](https://github.com/deephaven/web-client-ui/commit/640e002f85ea86961a22695c9c7659ca5d1de1ee)) * ComboBoxNormalized - windowed data component ([#2072](https://github.com/deephaven/web-client-ui/issues/2072)) ([a30341a](https://github.com/deephaven/web-client-ui/commit/a30341a728625dc7fdc2b0a54b88dfc737977b7a)) * Embed widget loading workspace settings ([#2068](https://github.com/deephaven/web-client-ui/issues/2068)) ([b090f20](https://github.com/deephaven/web-client-ui/commit/b090f200b38a7ecab1056b17f445c2af3ae09a41)) * Export iris-grid mouse handlers ([#2083](https://github.com/deephaven/web-client-ui/issues/2083)) ([336c078](https://github.com/deephaven/web-client-ui/commit/336c07872af4f750c8b3d38638a8893670e0881a)) ### Bug Fixes * Console scroll bar following dynamic output ([#2076](https://github.com/deephaven/web-client-ui/issues/2076)) ([a91e4f3](https://github.com/deephaven/web-client-ui/commit/a91e4f348fc23618f10ac1d8c3a87bf237eb7bbd)) * Dashboard plugin crashing UI on throw ([#2080](https://github.com/deephaven/web-client-ui/issues/2080)) ([e6b55cf](https://github.com/deephaven/web-client-ui/commit/e6b55cf78561a1508d49109e9003813b9cc27262)) * DH-17199: Filter by value in the tree table context menu always shows null ([#2078](https://github.com/deephaven/web-client-ui/issues/2078)) ([4eb38dd](https://github.com/deephaven/web-client-ui/commit/4eb38dd2c47071516269662f8a975044e6bb0a9a)) * Reconnect Auth Fail Fix - embed-widget ([#2023](https://github.com/deephaven/web-client-ui/issues/2023)) ([3e52242](https://github.com/deephaven/web-client-ui/commit/3e522428b88ed59cb9f8c38612a80236fd219e5d)) * view border styling ([#2063](https://github.com/deephaven/web-client-ui/issues/2063)) ([6f99e6b](https://github.com/deephaven/web-client-ui/commit/6f99e6b764a63e31aec36d435ec62926d109955e)) Co-authored-by: deephaven-internal <66694643+deephaven-internal@users.noreply.github.com> --- web/client-ui/Dockerfile | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/web/client-ui/Dockerfile b/web/client-ui/Dockerfile index 9ff23c93094..1ddd91ec0a7 100644 --- a/web/client-ui/Dockerfile +++ b/web/client-ui/Dockerfile @@ -2,10 +2,10 @@ FROM deephaven/node:local-build WORKDIR /usr/src/app # Most of the time, these versions are the same, except in cases where a patch only affects one of the packages -ARG WEB_VERSION=0.82.0 -ARG GRID_VERSION=0.82.0 -ARG CHART_VERSION=0.82.0 -ARG WIDGET_VERSION=0.82.0 +ARG WEB_VERSION=0.83.0 +ARG GRID_VERSION=0.83.0 +ARG CHART_VERSION=0.83.0 +ARG WIDGET_VERSION=0.83.0 # Pull in the published code-studio package from npmjs and extract is RUN set -eux; \ From 77b6c1c1007f5460b6e672736dd5c1099e24ca51 Mon Sep 17 00:00:00 2001 From: Akshat Jawne <69530774+AkshatJawne@users.noreply.github.com> Date: Wed, 26 Jun 2024 07:37:07 -0600 Subject: [PATCH 18/18] fix: Modify key table viewport to only include key columns (#5669) * modify key table viewport to only include key columns * change fetchSummary --- .../web/client/api/JsPartitionedTable.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/JsPartitionedTable.java b/web/client-api/src/main/java/io/deephaven/web/client/api/JsPartitionedTable.java index 38a184716a6..3ae370fccfb 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/JsPartitionedTable.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/JsPartitionedTable.java @@ -13,7 +13,7 @@ import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.partitionedtable_pb.GetTableRequest; import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.partitionedtable_pb.MergeRequest; import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.partitionedtable_pb.PartitionedTableDescriptor; -import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.table_pb.DropColumnsRequest; +import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.table_pb.SelectOrUpdateRequest; import io.deephaven.javascript.proto.dhinternal.io.deephaven.proto.ticket_pb.TypedTicket; import io.deephaven.web.client.api.barrage.WebBarrageUtils; import io.deephaven.web.client.api.barrage.def.ColumnDefinition; @@ -82,15 +82,15 @@ public Promise refetch() { return w.getExportedObjects()[0].fetch(); }).then(result -> connection.newState((c, state, metadata) -> { JsTable keyTable = (JsTable) result; - DropColumnsRequest drop = new DropColumnsRequest(); - drop.setColumnNamesList(new String[] {descriptor.getConstituentColumnName()}); - drop.setSourceId(keyTable.state().getHandle().makeTableReference()); - drop.setResultId(state.getHandle().makeTicket()); - connection.tableServiceClient().dropColumns(drop, metadata, (fail, success) -> { + SelectOrUpdateRequest view = new SelectOrUpdateRequest(); + view.setSourceId(keyTable.state().getHandle().makeTableReference()); + view.setResultId(state.getHandle().makeTicket()); + view.setColumnSpecsList(descriptor.getKeyColumnNamesList()); + connection.tableServiceClient().view(view, metadata, (fail, success) -> { keyTable.close(); c.apply(fail, success); }); - }, "drop constituent column") + }, "view only keys columns") .refetch(this, connection.metadata()) .then(state -> Promise.resolve(new JsTable(connection, state)))).then(result -> { keys = result;