From d5770fe8dd1486d457c87c17a7df8dba276e9bcd Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Wed, 19 Jul 2017 14:34:37 -0700 Subject: [PATCH] [FLINK-6429] [table] Bump Calcite version to 1.13. This closes #4373. --- flink-libraries/flink-table/pom.xml | 2 +- .../calcite/avatica/util/DateTimeUtils.java | 1044 ++++ .../calcite/rel/rules/PushProjector.java | 868 +++ .../calcite/sql/fun/SqlGroupFunction.java | 103 - .../calcite/sql/fun/SqlStdOperatorTable.java | 2133 ------- .../calcite/sql/validate/AggChecker.java | 225 - .../validate/SqlUserDefinedAggFunction.java | 82 - .../calcite/sql2rel/SqlToRelConverter.java | 5356 ----------------- .../table/calcite/FlinkTypeFactory.scala | 2 +- .../flink/table/calcite/FlinkTypeSystem.scala | 2 +- .../table/catalog/ExternalCatalogSchema.scala | 2 + .../flink/table/codegen/CodeGenerator.scala | 3 + .../apache/flink/table/expressions/call.scala | 1 + .../apache/flink/table/expressions/time.scala | 44 +- .../functions/utils/AggSqlFunction.scala | 1 + .../table/plan/rules/FlinkRuleSets.scala | 4 +- .../table/plan/stats/FlinkStatistic.scala | 5 +- .../table/plan/util/RexProgramExtractor.scala | 3 + .../table/api/batch/sql/CorrelateTest.scala | 24 +- .../table/api/batch/table/CorrelateTest.scala | 6 +- .../table/api/stream/sql/CorrelateTest.scala | 24 +- .../api/stream/table/CorrelateTest.scala | 24 +- .../expressions/ScalarFunctionsTest.scala | 5 +- .../plan/TimeIndicatorConversionTest.scala | 2 +- 24 files changed, 2002 insertions(+), 7963 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlGroupFunction.java delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/AggChecker.java delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml index 8a7e3ac3f8bc9..0e943adf7679d 100644 --- a/flink-libraries/flink-table/pom.xml +++ b/flink-libraries/flink-table/pom.xml @@ -52,7 +52,7 @@ under the License. org.apache.calcite calcite-core - 1.12.0 + 1.13.0 org.apache.calcite.avatica diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java new file mode 100644 index 0000000000000..d1a87a7d50446 --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/calcite/avatica/util/DateTimeUtils.java @@ -0,0 +1,1044 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.calcite.avatica.util; + +import java.text.DateFormat; +import java.text.NumberFormat; +import java.text.ParsePosition; +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Date; +import java.util.Locale; +import java.util.TimeZone; + +/* + * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-1884 IS FIXED. + */ + +/** + * Utility functions for datetime types: date, time, timestamp. + * + *

Used by the JDBC driver. + * + *

TODO: review methods for performance. Due to allocations required, it may + * be preferable to introduce a "formatter" with the required state. + */ +public class DateTimeUtils { + /** The julian date of the epoch, 1970-01-01. */ + public static final int EPOCH_JULIAN = 2440588; + + private DateTimeUtils() {} + + //~ Static fields/initializers --------------------------------------------- + + /** The SimpleDateFormat string for ISO dates, "yyyy-MM-dd". */ + public static final String DATE_FORMAT_STRING = "yyyy-MM-dd"; + + /** The SimpleDateFormat string for ISO times, "HH:mm:ss". */ + public static final String TIME_FORMAT_STRING = "HH:mm:ss"; + + /** The SimpleDateFormat string for ISO timestamps, "yyyy-MM-dd HH:mm:ss". */ + public static final String TIMESTAMP_FORMAT_STRING = + DATE_FORMAT_STRING + " " + TIME_FORMAT_STRING; + + /** The GMT time zone. + * + * @deprecated Use {@link #UTC_ZONE} */ + @Deprecated // to be removed before 2.0 + public static final TimeZone GMT_ZONE = TimeZone.getTimeZone("GMT"); + + /** The UTC time zone. */ + public static final TimeZone UTC_ZONE = TimeZone.getTimeZone("UTC"); + + /** The Java default time zone. */ + public static final TimeZone DEFAULT_ZONE = TimeZone.getDefault(); + + /** + * The number of milliseconds in a second. + */ + public static final long MILLIS_PER_SECOND = 1000L; + + /** + * The number of milliseconds in a minute. + */ + public static final long MILLIS_PER_MINUTE = 60000L; + + /** + * The number of milliseconds in an hour. + */ + public static final long MILLIS_PER_HOUR = 3600000L; // = 60 * 60 * 1000 + + /** + * The number of milliseconds in a day. + * + *

This is the modulo 'mask' used when converting + * TIMESTAMP values to DATE and TIME values. + */ + public static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + + /** + * Calendar set to the epoch (1970-01-01 00:00:00 UTC). Useful for + * initializing other values. Calendars are not immutable, so be careful not + * to screw up this object for everyone else. + */ + public static final Calendar ZERO_CALENDAR; + + static { + ZERO_CALENDAR = Calendar.getInstance(DateTimeUtils.UTC_ZONE, Locale.ROOT); + ZERO_CALENDAR.setTimeInMillis(0); + } + + //~ Methods ---------------------------------------------------------------- + + /** + * Parses a string using {@link SimpleDateFormat} and a given pattern. This + * method parses a string at the specified parse position and if successful, + * updates the parse position to the index after the last character used. + * The parsing is strict and requires months to be less than 12, days to be + * less than 31, etc. + * + * @param s string to be parsed + * @param dateFormat Date format + * @param tz time zone in which to interpret string. Defaults to the Java + * default time zone + * @param pp position to start parsing from + * @return a Calendar initialized with the parsed value, or null if parsing + * failed. If returned, the Calendar is configured to the GMT time zone. + */ + private static Calendar parseDateFormat(String s, DateFormat dateFormat, + TimeZone tz, ParsePosition pp) { + if (tz == null) { + tz = DEFAULT_ZONE; + } + Calendar ret = Calendar.getInstance(tz, Locale.ROOT); + dateFormat.setCalendar(ret); + dateFormat.setLenient(false); + + final Date d = dateFormat.parse(s, pp); + if (null == d) { + return null; + } + ret.setTime(d); + ret.setTimeZone(UTC_ZONE); + return ret; + } + + @Deprecated // to be removed before 2.0 + public static Calendar parseDateFormat(String s, String pattern, + TimeZone tz) { + return parseDateFormat(s, new SimpleDateFormat(pattern, Locale.ROOT), tz); + } + + /** + * Parses a string using {@link SimpleDateFormat} and a given pattern. The + * entire string must match the pattern specified. + * + * @param s string to be parsed + * @param dateFormat Date format + * @param tz time zone in which to interpret string. Defaults to the Java + * default time zone + * @return a Calendar initialized with the parsed value, or null if parsing + * failed. If returned, the Calendar is configured to the UTC time zone. + */ + public static Calendar parseDateFormat(String s, DateFormat dateFormat, + TimeZone tz) { + ParsePosition pp = new ParsePosition(0); + Calendar ret = parseDateFormat(s, dateFormat, tz, pp); + if (pp.getIndex() != s.length()) { + // Didn't consume entire string - not good + return null; + } + return ret; + } + + @Deprecated // to be removed before 2.0 + public static PrecisionTime parsePrecisionDateTimeLiteral( + String s, + String pattern, + TimeZone tz) { + assert pattern != null; + return parsePrecisionDateTimeLiteral(s, + new SimpleDateFormat(pattern, Locale.ROOT), tz, 3); + } + + /** + * Parses a string using {@link SimpleDateFormat} and a given pattern, and + * if present, parses a fractional seconds component. The fractional seconds + * component must begin with a decimal point ('.') followed by numeric + * digits. The precision is rounded to a maximum of 3 digits of fractional + * seconds precision (to obtain milliseconds). + * + * @param s string to be parsed + * @param dateFormat Date format + * @param tz time zone in which to interpret string. Defaults to the + * local time zone + * @return a {@link DateTimeUtils.PrecisionTime PrecisionTime} initialized + * with the parsed value, or null if parsing failed. The PrecisionTime + * contains a GMT Calendar and a precision. + */ + public static PrecisionTime parsePrecisionDateTimeLiteral(String s, + DateFormat dateFormat, TimeZone tz, int maxPrecision) { + final ParsePosition pp = new ParsePosition(0); + final Calendar cal = parseDateFormat(s, dateFormat, tz, pp); + if (cal == null) { + return null; // Invalid date/time format + } + + // Note: the Java SimpleDateFormat 'S' treats any number after + // the decimal as milliseconds. That means 12:00:00.9 has 9 + // milliseconds and 12:00:00.9999 has 9999 milliseconds. + int p = 0; + String secFraction = ""; + if (pp.getIndex() < s.length()) { + // Check to see if rest is decimal portion + if (s.charAt(pp.getIndex()) != '.') { + return null; + } + + // Skip decimal sign + pp.setIndex(pp.getIndex() + 1); + + // Parse decimal portion + if (pp.getIndex() < s.length()) { + secFraction = s.substring(pp.getIndex()); + if (!secFraction.matches("\\d+")) { + return null; + } + NumberFormat nf = NumberFormat.getIntegerInstance(Locale.ROOT); + Number num = nf.parse(s, pp); + if ((num == null) || (pp.getIndex() != s.length())) { + // Invalid decimal portion + return null; + } + + // Determine precision - only support prec 3 or lower + // (milliseconds) Higher precisions are quietly rounded away + p = secFraction.length(); + if (maxPrecision >= 0) { + // If there is a maximum precision, ignore subsequent digits + p = Math.min(maxPrecision, p); + secFraction = secFraction.substring(0, p); + } + + // Calculate milliseconds + String millis = secFraction; + if (millis.length() > 3) { + millis = secFraction.substring(0, 3); + } + while (millis.length() < 3) { + millis = millis + "0"; + } + + int ms = Integer.valueOf(millis); + cal.add(Calendar.MILLISECOND, ms); + } + } + + assert pp.getIndex() == s.length(); + return new PrecisionTime(cal, secFraction, p); + } + + /** + * Gets the active time zone based on a Calendar argument + */ + public static TimeZone getTimeZone(Calendar cal) { + if (cal == null) { + return DEFAULT_ZONE; + } + return cal.getTimeZone(); + } + + /** + * Checks if the date/time format is valid + * + * @param pattern {@link SimpleDateFormat} pattern + * @throws IllegalArgumentException if the given pattern is invalid + */ + public static void checkDateFormat(String pattern) { + new SimpleDateFormat(pattern, Locale.ROOT); + } + + /** + * Creates a new date formatter with Farrago specific options. Farrago + * parsing is strict and does not allow values such as day 0, month 13, etc. + * + * @param format {@link SimpleDateFormat} pattern + */ + public static SimpleDateFormat newDateFormat(String format) { + SimpleDateFormat sdf = new SimpleDateFormat(format, Locale.ROOT); + sdf.setLenient(false); + return sdf; + } + + /** Helper for CAST({timestamp} AS VARCHAR(n)). */ + public static String unixTimestampToString(long timestamp) { + return unixTimestampToString(timestamp, 0); + } + + public static String unixTimestampToString(long timestamp, int precision) { + final StringBuilder buf = new StringBuilder(17); + int date = (int) (timestamp / MILLIS_PER_DAY); + int time = (int) (timestamp % MILLIS_PER_DAY); + if (time < 0) { + --date; + time += MILLIS_PER_DAY; + } + unixDateToString(buf, date); + buf.append(' '); + unixTimeToString(buf, time, precision); + return buf.toString(); + } + + /** Helper for CAST({timestamp} AS VARCHAR(n)). */ + public static String unixTimeToString(int time) { + return unixTimeToString(time, 0); + } + + public static String unixTimeToString(int time, int precision) { + final StringBuilder buf = new StringBuilder(8); + unixTimeToString(buf, time, precision); + return buf.toString(); + } + + private static void unixTimeToString(StringBuilder buf, int time, + int precision) { + int h = time / 3600000; + int time2 = time % 3600000; + int m = time2 / 60000; + int time3 = time2 % 60000; + int s = time3 / 1000; + int ms = time3 % 1000; + int2(buf, h); + buf.append(':'); + int2(buf, m); + buf.append(':'); + int2(buf, s); + if (precision > 0) { + buf.append('.'); + while (precision > 0) { + buf.append((char) ('0' + (ms / 100))); + ms = ms % 100; + ms = ms * 10; + --precision; + } + } + } + + private static void int2(StringBuilder buf, int i) { + buf.append((char) ('0' + (i / 10) % 10)); + buf.append((char) ('0' + i % 10)); + } + + private static void int4(StringBuilder buf, int i) { + buf.append((char) ('0' + (i / 1000) % 10)); + buf.append((char) ('0' + (i / 100) % 10)); + buf.append((char) ('0' + (i / 10) % 10)); + buf.append((char) ('0' + i % 10)); + } + + /** Helper for CAST({date} AS VARCHAR(n)). */ + public static String unixDateToString(int date) { + final StringBuilder buf = new StringBuilder(10); + unixDateToString(buf, date); + return buf.toString(); + } + + private static void unixDateToString(StringBuilder buf, int date) { + julianToString(buf, date + EPOCH_JULIAN); + } + + private static void julianToString(StringBuilder buf, int julian) { + // Algorithm the book "Astronomical Algorithms" by Jean Meeus, 1998 + int b, c; + if (julian > 2299160) { + int a = julian + 32044; + b = (4 * a + 3) / 146097; + c = a - b *146097 / 4; + } else { + b = 0; + c = julian + 32082; + } + int d = (4 * c + 3) / 1461; + int e = c - (1461 * d) / 4; + int m = (5 * e + 2) / 153; + int day = e - (153 * m + 2) / 5 + 1; + int month = m + 3 - 12 * (m / 10); + int year = b * 100 + d - 4800 + (m / 10); + + int4(buf, year); + buf.append('-'); + int2(buf, month); + buf.append('-'); + int2(buf, day); + } + + public static String intervalYearMonthToString(int v, TimeUnitRange range) { + final StringBuilder buf = new StringBuilder(); + if (v >= 0) { + buf.append('+'); + } else { + buf.append('-'); + v = -v; + } + final int y; + final int m; + switch (range) { + case YEAR: + v = roundUp(v, 12); + y = v / 12; + buf.append(y); + break; + case YEAR_TO_MONTH: + y = v / 12; + buf.append(y); + buf.append('-'); + m = v % 12; + number(buf, m, 2); + break; + case MONTH: + m = v; + buf.append(m); + break; + default: + throw new AssertionError(range); + } + return buf.toString(); + } + + public static StringBuilder number(StringBuilder buf, int v, int n) { + for (int k = digitCount(v); k < n; k++) { + buf.append('0'); + } + return buf.append(v); + } + + public static int digitCount(int v) { + for (int n = 1;; n++) { + v /= 10; + if (v == 0) { + return n; + } + } + } + + private static int roundUp(int dividend, int divisor) { + int remainder = dividend % divisor; + dividend -= remainder; + if (remainder * 2 > divisor) { + dividend += divisor; + } + return dividend; + } + + /** Cheap, unsafe, long power. power(2, 3) returns 8. */ + public static long powerX(long a, long b) { + long x = 1; + while (b > 0) { + x *= a; + --b; + } + return x; + } + + public static String intervalDayTimeToString(long v, TimeUnitRange range, + int scale) { + final StringBuilder buf = new StringBuilder(); + if (v >= 0) { + buf.append('+'); + } else { + buf.append('-'); + v = -v; + } + final long ms; + final long s; + final long m; + final long h; + final long d; + switch (range) { + case DAY_TO_SECOND: + v = roundUp(v, powerX(10, 3 - scale)); + ms = v % 1000; + v /= 1000; + s = v % 60; + v /= 60; + m = v % 60; + v /= 60; + h = v % 24; + v /= 24; + d = v; + buf.append((int) d); + buf.append(' '); + number(buf, (int) h, 2); + buf.append(':'); + number(buf, (int) m, 2); + buf.append(':'); + number(buf, (int) s, 2); + fraction(buf, scale, ms); + break; + case DAY_TO_MINUTE: + v = roundUp(v, 1000 * 60); + v /= 1000; + v /= 60; + m = v % 60; + v /= 60; + h = v % 24; + v /= 24; + d = v; + buf.append((int) d); + buf.append(' '); + number(buf, (int) h, 2); + buf.append(':'); + number(buf, (int) m, 2); + break; + case DAY_TO_HOUR: + v = roundUp(v, 1000 * 60 * 60); + v /= 1000; + v /= 60; + v /= 60; + h = v % 24; + v /= 24; + d = v; + buf.append((int) d); + buf.append(' '); + number(buf, (int) h, 2); + break; + case DAY: + v = roundUp(v, 1000 * 60 * 60 * 24); + d = v / (1000 * 60 * 60 * 24); + buf.append((int) d); + break; + case HOUR: + v = roundUp(v, 1000 * 60 * 60); + v /= 1000; + v /= 60; + v /= 60; + h = v; + buf.append((int) h); + break; + case HOUR_TO_MINUTE: + v = roundUp(v, 1000 * 60); + v /= 1000; + v /= 60; + m = v % 60; + v /= 60; + h = v; + buf.append((int) h); + buf.append(':'); + number(buf, (int) m, 2); + break; + case HOUR_TO_SECOND: + v = roundUp(v, powerX(10, 3 - scale)); + ms = v % 1000; + v /= 1000; + s = v % 60; + v /= 60; + m = v % 60; + v /= 60; + h = v; + buf.append((int) h); + buf.append(':'); + number(buf, (int) m, 2); + buf.append(':'); + number(buf, (int) s, 2); + fraction(buf, scale, ms); + break; + case MINUTE_TO_SECOND: + v = roundUp(v, powerX(10, 3 - scale)); + ms = v % 1000; + v /= 1000; + s = v % 60; + v /= 60; + m = v; + buf.append((int) m); + buf.append(':'); + number(buf, (int) s, 2); + fraction(buf, scale, ms); + break; + case MINUTE: + v = roundUp(v, 1000 * 60); + v /= 1000; + v /= 60; + m = v; + buf.append((int) m); + break; + case SECOND: + v = roundUp(v, powerX(10, 3 - scale)); + ms = v % 1000; + v /= 1000; + s = v; + buf.append((int) s); + fraction(buf, scale, ms); + break; + default: + throw new AssertionError(range); + } + return buf.toString(); + } + + /** + * Rounds a dividend to the nearest divisor. + * For example roundUp(31, 10) yields 30; roundUp(37, 10) yields 40. + * @param dividend Number to be divided + * @param divisor Number to divide by + * @return Rounded dividend + */ + private static long roundUp(long dividend, long divisor) { + long remainder = dividend % divisor; + dividend -= remainder; + if (remainder * 2 > divisor) { + dividend += divisor; + } + return dividend; + } + + private static void fraction(StringBuilder buf, int scale, long ms) { + if (scale > 0) { + buf.append('.'); + long v1 = scale == 3 ? ms + : scale == 2 ? ms / 10 + : scale == 1 ? ms / 100 + : 0; + number(buf, (int) v1, scale); + } + } + + public static int dateStringToUnixDate(String s) { + int hyphen1 = s.indexOf('-'); + int y; + int m; + int d; + if (hyphen1 < 0) { + y = Integer.parseInt(s.trim()); + m = 1; + d = 1; + } else { + y = Integer.parseInt(s.substring(0, hyphen1).trim()); + final int hyphen2 = s.indexOf('-', hyphen1 + 1); + if (hyphen2 < 0) { + m = Integer.parseInt(s.substring(hyphen1 + 1).trim()); + d = 1; + } else { + m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim()); + d = Integer.parseInt(s.substring(hyphen2 + 1).trim()); + } + } + return ymdToUnixDate(y, m, d); + } + + public static int timeStringToUnixDate(String v) { + return timeStringToUnixDate(v, 0); + } + + public static int timeStringToUnixDate(String v, int start) { + final int colon1 = v.indexOf(':', start); + int hour; + int minute; + int second; + int milli; + if (colon1 < 0) { + hour = Integer.parseInt(v.trim()); + minute = 1; + second = 1; + milli = 0; + } else { + hour = Integer.parseInt(v.substring(start, colon1).trim()); + final int colon2 = v.indexOf(':', colon1 + 1); + if (colon2 < 0) { + minute = Integer.parseInt(v.substring(colon1 + 1).trim()); + second = 1; + milli = 0; + } else { + minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim()); + int dot = v.indexOf('.', colon2); + if (dot < 0) { + second = Integer.parseInt(v.substring(colon2 + 1).trim()); + milli = 0; + } else { + second = Integer.parseInt(v.substring(colon2 + 1, dot).trim()); + milli = parseFraction(v.substring(dot + 1).trim(), 100); + } + } + } + return hour * (int) MILLIS_PER_HOUR + + minute * (int) MILLIS_PER_MINUTE + + second * (int) MILLIS_PER_SECOND + + milli; + } + + /** Parses a fraction, multiplying the first character by {@code multiplier}, + * the second character by {@code multiplier / 10}, + * the third character by {@code multiplier / 100}, and so forth. + * + *

For example, {@code parseFraction("1234", 100)} yields {@code 123}. */ + private static int parseFraction(String v, int multiplier) { + int r = 0; + for (int i = 0; i < v.length(); i++) { + char c = v.charAt(i); + int x = c < '0' || c > '9' ? 0 : (c - '0'); + r += multiplier * x; + if (multiplier < 10) { + // We're at the last digit. Check for rounding. + if (i + 1 < v.length() + && v.charAt(i + 1) >= '5') { + ++r; + } + break; + } + multiplier /= 10; + } + return r; + } + + public static long timestampStringToUnixDate(String s) { + final long d; + final long t; + s = s.trim(); + int space = s.indexOf(' '); + if (space >= 0) { + d = dateStringToUnixDate(s.substring(0, space)); + t = timeStringToUnixDate(s, space + 1); + } else { + d = dateStringToUnixDate(s); + t = 0; + } + return d * MILLIS_PER_DAY + t; + } + + public static long unixDateExtract(TimeUnitRange range, long date) { + return julianExtract(range, (int) date + EPOCH_JULIAN); + } + + private static int julianExtract(TimeUnitRange range, int julian) { + // Algorithm the book "Astronomical Algorithms" by Jean Meeus, 1998 + int b, c; + if (julian > 2299160) { + int a = julian + 32044; + b = (4 * a + 3) / 146097; + c = a - b *146097 / 4; + } else { + b = 0; + c = julian + 32082; + } + int d = (4 * c + 3) / 1461; + int e = c - (1461 * d) / 4; + int m = (5 * e + 2) / 153; + int day = e - (153 * m + 2) / 5 + 1; + int month = m + 3 - 12 * (m / 10); + int year = b * 100 + d - 4800 + (m / 10); + + switch (range) { + case YEAR: + return year; + case QUARTER: + return (month + 2) / 3; + case MONTH: + return month; + case DAY: + return day; + case DOW: + return (int) floorMod(julian + 1, 7) + 1; // sun=1, sat=7 + case WEEK: + long fmofw = firstMondayOfFirstWeek(year); + if (julian < fmofw) { + fmofw = firstMondayOfFirstWeek(year - 1); + } + return (int) (julian - fmofw) / 7 + 1; + case DOY: + final long janFirst = ymdToJulian(year, 1, 1); + return (int) (julian - janFirst) + 1; + case CENTURY: + return year > 0 + ? (year + 99) / 100 + : (year - 99) / 100; + case MILLENNIUM: + return year > 0 + ? (year + 999) / 1000 + : (year - 999) / 1000; + default: + throw new AssertionError(range); + } + } + + /** Returns the first day of the first week of a year. + * Per ISO-8601 it is the Monday of the week that contains Jan 4, + * or equivalently, it is a Monday between Dec 29 and Jan 4. + * Sometimes it is in the year before the given year. */ + private static long firstMondayOfFirstWeek(int year) { + final long janFirst = ymdToJulian(year, 1, 1); + final long janFirstDow = floorMod(janFirst + 1, 7); // sun=0, sat=6 + return janFirst + (11 - janFirstDow) % 7 - 3; + } + + /** Extracts a time unit from a UNIX date (milliseconds since epoch). */ + public static int unixTimestampExtract(TimeUnitRange range, + long timestamp) { + return unixTimeExtract(range, (int) floorMod(timestamp, MILLIS_PER_DAY)); + } + + /** Extracts a time unit from a time value (milliseconds since midnight). */ + public static int unixTimeExtract(TimeUnitRange range, int time) { + assert time >= 0; + assert time < MILLIS_PER_DAY; + switch (range) { + case HOUR: + return time / (int) MILLIS_PER_HOUR; + case MINUTE: + final int minutes = time / (int) MILLIS_PER_MINUTE; + return minutes % 60; + case SECOND: + final int seconds = time / (int) MILLIS_PER_SECOND; + return seconds % 60; + default: + throw new AssertionError(range); + } + } + + /** Resets to zero the "time" part of a timestamp. */ + public static long resetTime(long timestamp) { + int date = (int) (timestamp / MILLIS_PER_DAY); + return (long) date * MILLIS_PER_DAY; + } + + /** Resets to epoch (1970-01-01) the "date" part of a timestamp. */ + public static long resetDate(long timestamp) { + return floorMod(timestamp, MILLIS_PER_DAY); + } + + public static long unixTimestampFloor(TimeUnitRange range, long timestamp) { + int date = (int) (timestamp / MILLIS_PER_DAY); + final int f = julianDateFloor(range, date + EPOCH_JULIAN, true); + return (long) f * MILLIS_PER_DAY; + } + + public static long unixDateFloor(TimeUnitRange range, long date) { + return julianDateFloor(range, (int) date + EPOCH_JULIAN, true); + } + + public static long unixTimestampCeil(TimeUnitRange range, long timestamp) { + int date = (int) (timestamp / MILLIS_PER_DAY); + final int f = julianDateFloor(range, date + EPOCH_JULIAN, false); + return (long) f * MILLIS_PER_DAY; + } + + public static long unixDateCeil(TimeUnitRange range, long date) { + return julianDateFloor(range, (int) date + EPOCH_JULIAN, true); + } + + private static int julianDateFloor(TimeUnitRange range, int julian, + boolean floor) { + // Algorithm the book "Astronomical Algorithms" by Jean Meeus, 1998 + int b, c; + if (julian > 2299160) { + int a = julian + 32044; + b = (4 * a + 3) / 146097; + c = a - b *146097 / 4; + } else { + b = 0; + c = julian + 32082; + } + int d = (4 * c + 3) / 1461; + int e = c - (1461 * d) / 4; + int m = (5 * e + 2) / 153; + int day = e - (153 * m + 2) / 5 + 1; + int month = m + 3 - 12 * (m / 10); + int year = b * 100 + d - 4800 + (m / 10); + + switch (range) { + case YEAR: + if (!floor && (month > 1 || day > 1)) { + ++year; + } + return ymdToUnixDate(year, 1, 1); + case MONTH: + if (!floor && day > 1) { + ++month; + } + return ymdToUnixDate(year, month, 1); + default: + throw new AssertionError(range); + } + } + + public static int ymdToUnixDate(int year, int month, int day) { + final int julian = ymdToJulian(year, month, day); + return julian - EPOCH_JULIAN; + } + + public static int ymdToJulian(int year, int month, int day) { + int a = (14 - month) / 12; + int y = year + 4800 - a; + int m = month + 12 * a - 3; + int j = day + (153 * m + 2) / 5 + + 365 * y + + y / 4 + - y / 100 + + y / 400 + - 32045; + if (j < 2299161) { + j = day + (153 * m + 2) / 5 + 365 * y + y / 4 - 32083; + } + return j; + } + + public static long unixTimestamp(int year, int month, int day, int hour, + int minute, int second) { + final int date = ymdToUnixDate(year, month, day); + return (long) date * MILLIS_PER_DAY + + (long) hour * MILLIS_PER_HOUR + + (long) minute * MILLIS_PER_MINUTE + + (long) second * MILLIS_PER_SECOND; + } + + /** Adds a given number of months to a timestamp, represented as the number + * of milliseconds since the epoch. */ + public static long addMonths(long timestamp, int m) { + final long millis = + DateTimeUtils.floorMod(timestamp, DateTimeUtils.MILLIS_PER_DAY); + timestamp -= millis; + final long x = + addMonths((int) (timestamp / DateTimeUtils.MILLIS_PER_DAY), m); + return x * DateTimeUtils.MILLIS_PER_DAY + millis; + } + + /** Adds a given number of months to a date, represented as the number of + * days since the epoch. */ + public static int addMonths(int date, int m) { + int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date); + int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date); + int d0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.DAY, date); + int y = m / 12; + y0 += y; + m0 += m - y * 12; + int last = lastDay(y0, m0); + if (d0 > last) { + d0 = 1; + if (++m0 > 12) { + m0 = 1; + ++y0; + } + } + return DateTimeUtils.ymdToUnixDate(y0, m0, d0); + } + + private static int lastDay(int y, int m) { + switch (m) { + case 2: + return y % 4 == 0 + && (y % 100 != 0 + || y % 400 == 0) + ? 29 : 28; + case 4: + case 6: + case 9: + case 11: + return 30; + default: + return 31; + } + } + + /** Finds the number of months between two dates, each represented as the + * number of days since the epoch. */ + public static int subtractMonths(int date0, int date1) { + if (date0 < date1) { + return -subtractMonths(date1, date0); + } + // Start with an estimate. + // Since no month has more than 31 days, the estimate is <= the true value. + int m = (date0 - date1) / 31; + for (;;) { + int date2 = addMonths(date1, m); + if (date2 >= date0) { + return m; + } + int date3 = addMonths(date1, m + 1); + if (date3 > date0) { + return m; + } + ++m; + } + } + + public static int subtractMonths(long t0, long t1) { + final long millis0 = + DateTimeUtils.floorMod(t0, DateTimeUtils.MILLIS_PER_DAY); + final int d0 = (int) DateTimeUtils.floorDiv(t0 - millis0, + DateTimeUtils.MILLIS_PER_DAY); + final long millis1 = + DateTimeUtils.floorMod(t1, DateTimeUtils.MILLIS_PER_DAY); + final int d1 = (int) DateTimeUtils.floorDiv(t1 - millis1, + DateTimeUtils.MILLIS_PER_DAY); + int x = subtractMonths(d0, d1); + final long d2 = addMonths(d1, x); + if (d2 == d0 && millis0 < millis1) { + --x; + } + return x; + } + + /** Divide, rounding towards negative infinity. */ + public static long floorDiv(long x, long y) { + long r = x / y; + // if the signs are different and modulo not zero, round down + if ((x ^ y) < 0 && (r * y != x)) { + r--; + } + return r; + } + + /** Modulo, always returning a non-negative result. */ + public static long floorMod(long x, long y) { + return x - floorDiv(x, y) * y; + } + + /** Creates an instance of {@link Calendar} in the root locale and UTC time + * zone. */ + public static Calendar calendar() { + return Calendar.getInstance(UTC_ZONE, Locale.ROOT); + } + + //~ Inner Classes ---------------------------------------------------------- + + /** + * Helper class for {@link DateTimeUtils#parsePrecisionDateTimeLiteral} + */ + public static class PrecisionTime { + private final Calendar cal; + private final String fraction; + private final int precision; + + public PrecisionTime(Calendar cal, String fraction, int precision) { + this.cal = cal; + this.fraction = fraction; + this.precision = precision; + } + + public Calendar getCalendar() { + return cal; + } + + public int getPrecision() { + return precision; + } + + public String getFraction() { + return fraction; + } + } +} + +// End DateTimeUtils.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java new file mode 100644 index 0000000000000..0955aebed4017 --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/PushProjector.java @@ -0,0 +1,868 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.calcite.rel.rules; + +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.plan.Strong; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SemiJoin; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.runtime.PredicateImpl; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.util.BitSets; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Pair; + +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; +import java.util.Set; + +// This class is copied from Apache Calcite except that it does not +// automatically name the field using the name of the operators +// as the Table API rejects special characters like '-' in the field names. + +/** + * PushProjector is a utility class used to perform operations used in push + * projection rules. + * + *

Pushing is particularly interesting in the case of join, because there + * are multiple inputs. Generally an expression can be pushed down to a + * particular input if it depends upon no other inputs. If it can be pushed + * down to both sides, it is pushed down to the left. + * + *

Sometimes an expression needs to be split before it can be pushed down. + * To flag that an expression cannot be split, specify a rule that it must be + * preserved. Such an expression will be pushed down intact to one + * of the inputs, or not pushed down at all.

+ */ +public class PushProjector { + //~ Instance fields -------------------------------------------------------- + + private final Project origProj; + private final RexNode origFilter; + private final RelNode childRel; + private final ExprCondition preserveExprCondition; + private final RelBuilder relBuilder; + + /** + * Original projection expressions + */ + final List origProjExprs; + + /** + * Fields from the RelNode that the projection is being pushed past + */ + final List childFields; + + /** + * Number of fields in the RelNode that the projection is being pushed past + */ + final int nChildFields; + + /** + * Bitmap containing the references in the original projection + */ + final BitSet projRefs; + + /** + * Bitmap containing the fields in the RelNode that the projection is being + * pushed past, if the RelNode is not a join. If the RelNode is a join, then + * the fields correspond to the left hand side of the join. + */ + final ImmutableBitSet childBitmap; + + /** + * Bitmap containing the fields in the right hand side of a join, in the + * case where the projection is being pushed past a join. Not used + * otherwise. + */ + final ImmutableBitSet rightBitmap; + + /** + * Bitmap containing the fields that should be strong, i.e. when preserving expressions + * we can only preserve them if the expressions if it is null when these fields are null. + */ + final ImmutableBitSet strongBitmap; + + /** + * Number of fields in the RelNode that the projection is being pushed past, + * if the RelNode is not a join. If the RelNode is a join, then this is the + * number of fields in the left hand side of the join. + * + *

The identity + * {@code nChildFields == nSysFields + nFields + nFieldsRight} + * holds. {@code nFields} does not include {@code nSysFields}. + * The output of a join looks like this: + * + *

+   * | nSysFields | nFields | nFieldsRight |
+   * 
+ * + *

The output of a single-input rel looks like this: + * + *

+   * | nSysFields | nFields |
+   * 
+ */ + final int nFields; + + /** + * Number of fields in the right hand side of a join, in the case where the + * projection is being pushed past a join. Always 0 otherwise. + */ + final int nFieldsRight; + + /** + * Number of system fields. System fields appear at the start of a join, + * before the first field from the left input. + */ + private final int nSysFields; + + /** + * Expressions referenced in the projection/filter that should be preserved. + * In the case where the projection is being pushed past a join, then the + * list only contains the expressions corresponding to the left hand side of + * the join. + */ + final List childPreserveExprs; + + /** + * Expressions referenced in the projection/filter that should be preserved, + * corresponding to expressions on the right hand side of the join, if the + * projection is being pushed past a join. Empty list otherwise. + */ + final List rightPreserveExprs; + + /** + * Number of system fields being projected. + */ + int nSystemProject; + + /** + * Number of fields being projected. In the case where the projection is + * being pushed past a join, the number of fields being projected from the + * left hand side of the join. + */ + int nProject; + + /** + * Number of fields being projected from the right hand side of a join, in + * the case where the projection is being pushed past a join. 0 otherwise. + */ + int nRightProject; + + /** + * Rex builder used to create new expressions. + */ + final RexBuilder rexBuilder; + + //~ Constructors ----------------------------------------------------------- + + /** + * Creates a PushProjector object for pushing projects past a RelNode. + * + * @param origProj the original projection that is being pushed; + * may be null if the projection is implied as a + * result of a projection having been trivially + * removed + * @param origFilter the filter that the projection must also be + * pushed past, if applicable + * @param childRel the RelNode that the projection is being + * pushed past + * @param preserveExprCondition condition for whether an expression should + * be preserved in the projection + */ + public PushProjector( + Project origProj, + RexNode origFilter, + RelNode childRel, + ExprCondition preserveExprCondition, + RelBuilder relBuilder) { + this.origProj = origProj; + this.origFilter = origFilter; + this.childRel = childRel; + this.preserveExprCondition = preserveExprCondition; + this.relBuilder = Preconditions.checkNotNull(relBuilder); + if (origProj == null) { + origProjExprs = ImmutableList.of(); + } else { + origProjExprs = origProj.getProjects(); + } + + childFields = childRel.getRowType().getFieldList(); + nChildFields = childFields.size(); + + projRefs = new BitSet(nChildFields); + if (childRel instanceof Join) { + Join joinRel = (Join) childRel; + List leftFields = + joinRel.getLeft().getRowType().getFieldList(); + List rightFields = + joinRel.getRight().getRowType().getFieldList(); + nFields = leftFields.size(); + nFieldsRight = childRel instanceof SemiJoin ? 0 : rightFields.size(); + nSysFields = joinRel.getSystemFieldList().size(); + childBitmap = + ImmutableBitSet.range(nSysFields, nFields + nSysFields); + rightBitmap = + ImmutableBitSet.range(nFields + nSysFields, nChildFields); + + switch (joinRel.getJoinType()) { + case INNER: + strongBitmap = ImmutableBitSet.of(); + break; + case RIGHT: // All the left-input's columns must be strong + strongBitmap = ImmutableBitSet.range(nSysFields, nFields + nSysFields); + break; + case LEFT: // All the right-input's columns must be strong + strongBitmap = ImmutableBitSet.range(nFields + nSysFields, nChildFields); + break; + case FULL: + default: + strongBitmap = ImmutableBitSet.range(nSysFields, nChildFields); + } + + } else { + nFields = nChildFields; + nFieldsRight = 0; + childBitmap = ImmutableBitSet.range(nChildFields); + rightBitmap = null; + nSysFields = 0; + strongBitmap = ImmutableBitSet.of(); + } + assert nChildFields == nSysFields + nFields + nFieldsRight; + + childPreserveExprs = new ArrayList(); + rightPreserveExprs = new ArrayList(); + + rexBuilder = childRel.getCluster().getRexBuilder(); + } + + //~ Methods ---------------------------------------------------------------- + + /** + * Decomposes a projection to the input references referenced by a + * projection and a filter, either of which is optional. If both are + * provided, the filter is underneath the project. + * + *

Creates a projection containing all input references as well as + * preserving any special expressions. Converts the original projection + * and/or filter to reference the new projection. Then, finally puts on top, + * a final projection corresponding to the original projection. + * + * @param defaultExpr expression to be used in the projection if no fields + * or special columns are selected + * @return the converted projection if it makes sense to push elements of + * the projection; otherwise returns null + */ + public RelNode convertProject(RexNode defaultExpr) { + // locate all fields referenced in the projection and filter + locateAllRefs(); + + // if all columns are being selected (either explicitly in the + // projection) or via a "select *", then there needs to be some + // special expressions to preserve in the projection; otherwise, + // there's no point in proceeding any further + if (origProj == null) { + if (childPreserveExprs.size() == 0) { + return null; + } + + // even though there is no projection, this is the same as + // selecting all fields + if (nChildFields > 0) { + // Calling with nChildFields == 0 should be safe but hits + // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6222207 + projRefs.set(0, nChildFields); + } + nProject = nChildFields; + } else if ( + (projRefs.cardinality() == nChildFields) + && (childPreserveExprs.size() == 0)) { + return null; + } + + // if nothing is being selected from the underlying rel, just + // project the default expression passed in as a parameter or the + // first column if there is no default expression + if ((projRefs.cardinality() == 0) && (childPreserveExprs.size() == 0)) { + if (defaultExpr != null) { + childPreserveExprs.add(defaultExpr); + } else if (nChildFields == 1) { + return null; + } else { + projRefs.set(0); + nProject = 1; + } + } + + // create a new projection referencing all fields referenced in + // either the project or the filter + RelNode newProject = createProjectRefsAndExprs(childRel, false, false); + + int[] adjustments = getAdjustments(); + + // if a filter was passed in, convert it to reference the projected + // columns, placing it on top of the project just created + RelNode projChild; + if (origFilter != null) { + RexNode newFilter = + convertRefsAndExprs( + origFilter, + newProject.getRowType().getFieldList(), + adjustments); + relBuilder.push(newProject); + relBuilder.filter(newFilter); + projChild = relBuilder.build(); + } else { + projChild = newProject; + } + + // put the original project on top of the filter/project, converting + // it to reference the modified projection list; otherwise, create + // a projection that essentially selects all fields + return createNewProject(projChild, adjustments); + } + + /** + * Locates all references found in either the projection expressions a + * filter, as well as references to expressions that should be preserved. + * Based on that, determines whether pushing the projection makes sense. + * + * @return true if all inputs from the child that the projection is being + * pushed past are referenced in the projection/filter and no special + * preserve expressions are referenced; in that case, it does not make sense + * to push the projection + */ + public boolean locateAllRefs() { + RexUtil.apply( + new InputSpecialOpFinder( + projRefs, + childBitmap, + rightBitmap, + strongBitmap, + preserveExprCondition, + childPreserveExprs, + rightPreserveExprs), + origProjExprs, + origFilter); + + // The system fields of each child are always used by the join, even if + // they are not projected out of it. + projRefs.set( + nSysFields, + nSysFields + nSysFields, + true); + projRefs.set( + nSysFields + nFields, + nSysFields + nFields + nSysFields, + true); + + // Count how many fields are projected. + nSystemProject = 0; + nProject = 0; + nRightProject = 0; + for (int bit : BitSets.toIter(projRefs)) { + if (bit < nSysFields) { + nSystemProject++; + } else if (bit < nSysFields + nFields) { + nProject++; + } else { + nRightProject++; + } + } + + assert nSystemProject + nProject + nRightProject + == projRefs.cardinality(); + + if ((childRel instanceof Join) + || (childRel instanceof SetOp)) { + // if nothing is projected from the children, arbitrarily project + // the first columns; this is necessary since Fennel doesn't + // handle 0-column projections + if ((nProject == 0) && (childPreserveExprs.size() == 0)) { + projRefs.set(0); + nProject = 1; + } + if (childRel instanceof Join) { + if ((nRightProject == 0) && (rightPreserveExprs.size() == 0)) { + projRefs.set(nFields); + nRightProject = 1; + } + } + } + + // no need to push projections if all children fields are being + // referenced and there are no special preserve expressions; note + // that we need to do this check after we've handled the 0-column + // project cases + if (projRefs.cardinality() == nChildFields + && childPreserveExprs.size() == 0 + && rightPreserveExprs.size() == 0) { + return true; + } + + return false; + } + + /** + * Creates a projection based on the inputs specified in a bitmap and the + * expressions that need to be preserved. The expressions are appended after + * the input references. + * + * @param projChild child that the projection will be created on top of + * @param adjust if true, need to create new projection expressions; + * otherwise, the existing ones are reused + * @param rightSide if true, creating a projection for the right hand side + * of a join + * @return created projection + */ + public Project createProjectRefsAndExprs( + RelNode projChild, + boolean adjust, + boolean rightSide) { + List preserveExprs; + int nInputRefs; + int offset; + + if (rightSide) { + preserveExprs = rightPreserveExprs; + nInputRefs = nRightProject; + offset = nSysFields + nFields; + } else { + preserveExprs = childPreserveExprs; + nInputRefs = nProject; + offset = nSysFields; + } + int refIdx = offset - 1; + List> newProjects = + new ArrayList>(); + List destFields = + projChild.getRowType().getFieldList(); + + // add on the input references + for (int i = 0; i < nInputRefs; i++) { + refIdx = projRefs.nextSetBit(refIdx + 1); + assert refIdx >= 0; + final RelDataTypeField destField = destFields.get(refIdx - offset); + newProjects.add( + Pair.of( + (RexNode) rexBuilder.makeInputRef( + destField.getType(), refIdx - offset), + destField.getName())); + } + + // add on the expressions that need to be preserved, converting the + // arguments to reference the projected columns (if necessary) + int[] adjustments = {}; + if ((preserveExprs.size() > 0) && adjust) { + adjustments = new int[childFields.size()]; + for (int idx = offset; idx < childFields.size(); idx++) { + adjustments[idx] = -offset; + } + } + for (RexNode projExpr : preserveExprs) { + RexNode newExpr; + if (adjust) { + newExpr = + projExpr.accept( + new RelOptUtil.RexInputConverter( + rexBuilder, + childFields, + destFields, + adjustments)); + } else { + newExpr = projExpr; + } + newProjects.add( + Pair.of( + newExpr, + null)); + } + + return (Project) RelOptUtil.createProject( + projChild, + Pair.left(newProjects), + Pair.right(newProjects), + false, + relBuilder); + } + + /** + * Determines how much each input reference needs to be adjusted as a result + * of projection + * + * @return array indicating how much each input needs to be adjusted by + */ + public int[] getAdjustments() { + int[] adjustments = new int[nChildFields]; + int newIdx = 0; + int rightOffset = childPreserveExprs.size(); + for (int pos : BitSets.toIter(projRefs)) { + adjustments[pos] = -(pos - newIdx); + if (pos >= nSysFields + nFields) { + adjustments[pos] += rightOffset; + } + newIdx++; + } + return adjustments; + } + + /** + * Clones an expression tree and walks through it, adjusting each + * RexInputRef index by some amount, and converting expressions that need to + * be preserved to field references. + * + * @param rex the expression + * @param destFields fields that the new expressions will be referencing + * @param adjustments the amount each input reference index needs to be + * adjusted by + * @return modified expression tree + */ + public RexNode convertRefsAndExprs( + RexNode rex, + List destFields, + int[] adjustments) { + return rex.accept( + new RefAndExprConverter( + rexBuilder, + childFields, + destFields, + adjustments, + childPreserveExprs, + nProject, + rightPreserveExprs, + nProject + childPreserveExprs.size() + nRightProject)); + } + + /** + * Creates a new projection based on the original projection, adjusting all + * input refs using an adjustment array passed in. If there was no original + * projection, create a new one that selects every field from the underlying + * rel. + * + *

If the resulting projection would be trivial, return the child. + * + * @param projChild child of the new project + * @param adjustments array indicating how much each input reference should + * be adjusted by + * @return the created projection + */ + public RelNode createNewProject(RelNode projChild, int[] adjustments) { + final List> projects = Lists.newArrayList(); + + if (origProj != null) { + for (Pair p : origProj.getNamedProjects()) { + projects.add( + Pair.of( + convertRefsAndExprs( + p.left, + projChild.getRowType().getFieldList(), + adjustments), + p.right)); + } + } else { + for (Ord field : Ord.zip(childFields)) { + projects.add( + Pair.of( + (RexNode) rexBuilder.makeInputRef( + field.e.getType(), field.i), field.e.getName())); + } + } + return RelOptUtil.createProject( + projChild, + Pair.left(projects), + Pair.right(projects), + true /* optimize to avoid trivial projections, as per javadoc */, + relBuilder); + } + + //~ Inner Classes ---------------------------------------------------------- + + /** + * Visitor which builds a bitmap of the inputs used by an expressions, as + * well as locating expressions corresponding to special operators. + */ + private class InputSpecialOpFinder extends RexVisitorImpl { + private final BitSet rexRefs; + private final ImmutableBitSet leftFields; + private final ImmutableBitSet rightFields; + private final ImmutableBitSet strongFields; + private final ExprCondition preserveExprCondition; + private final List preserveLeft; + private final List preserveRight; + private final Strong strong; + + public InputSpecialOpFinder( + BitSet rexRefs, + ImmutableBitSet leftFields, + ImmutableBitSet rightFields, + final ImmutableBitSet strongFields, + ExprCondition preserveExprCondition, + List preserveLeft, + List preserveRight) { + super(true); + this.rexRefs = rexRefs; + this.leftFields = leftFields; + this.rightFields = rightFields; + this.preserveExprCondition = preserveExprCondition; + this.preserveLeft = preserveLeft; + this.preserveRight = preserveRight; + + this.strongFields = strongFields; + this.strong = Strong.of(strongFields); + } + + public Void visitCall(RexCall call) { + if (preserve(call)) { + return null; + } + super.visitCall(call); + return null; + } + + private boolean isStrong(final ImmutableBitSet exprArgs, final RexNode call) { + // If the expressions do not use any of the inputs that require output to be null, + // no need to check. Otherwise, check that the expression is null. + // For example, in an "left outer join", we don't require that expressions + // pushed down into the left input to be strong. On the other hand, + // expressions pushed into the right input must be. In that case, + // strongFields == right input fields. + return !strongFields.intersects(exprArgs) || strong.isNull(call); + } + + private boolean preserve(RexNode call) { + if (preserveExprCondition.test(call)) { + // if the arguments of the expression only reference the + // left hand side, preserve it on the left; similarly, if + // it only references expressions on the right + final ImmutableBitSet exprArgs = RelOptUtil.InputFinder.bits(call); + if (exprArgs.cardinality() > 0) { + if (leftFields.contains(exprArgs) && isStrong(exprArgs, call)) { + addExpr(preserveLeft, call); + return true; + } else if (rightFields.contains(exprArgs) && isStrong(exprArgs, call)) { + assert preserveRight != null; + addExpr(preserveRight, call); + return true; + } + } + // if the expression arguments reference both the left and + // right, fall through and don't attempt to preserve the + // expression, but instead locate references and special + // ops in the call operands + } + return false; + } + + public Void visitInputRef(RexInputRef inputRef) { + rexRefs.set(inputRef.getIndex()); + return null; + } + + /** + * Adds an expression to a list if the same expression isn't already in + * the list. Expressions are identical if their digests are the same. + * + * @param exprList current list of expressions + * @param newExpr new expression to be added + */ + private void addExpr(List exprList, RexNode newExpr) { + String newExprString = newExpr.toString(); + for (RexNode expr : exprList) { + if (newExprString.compareTo(expr.toString()) == 0) { + return; + } + } + exprList.add(newExpr); + } + } + + /** + * Walks an expression tree, replacing input refs with new values to reflect + * projection and converting special expressions to field references. + */ + private class RefAndExprConverter extends RelOptUtil.RexInputConverter { + private final List preserveLeft; + private final int firstLeftRef; + private final List preserveRight; + private final int firstRightRef; + + public RefAndExprConverter( + RexBuilder rexBuilder, + List srcFields, + List destFields, + int[] adjustments, + List preserveLeft, + int firstLeftRef, + List preserveRight, + int firstRightRef) { + super(rexBuilder, srcFields, destFields, adjustments); + this.preserveLeft = preserveLeft; + this.firstLeftRef = firstLeftRef; + this.preserveRight = preserveRight; + this.firstRightRef = firstRightRef; + } + + public RexNode visitCall(RexCall call) { + // if the expression corresponds to one that needs to be preserved, + // convert it to a field reference; otherwise, convert the entire + // expression + int match = + findExprInLists( + call, + preserveLeft, + firstLeftRef, + preserveRight, + firstRightRef); + if (match >= 0) { + return rexBuilder.makeInputRef( + destFields.get(match).getType(), + match); + } + return super.visitCall(call); + } + + /** + * Looks for a matching RexNode from among two lists of RexNodes and + * returns the offset into the list corresponding to the match, adjusted + * by an amount, depending on whether the match was from the first or + * second list. + * + * @param rex RexNode that is being matched against + * @param rexList1 first list of RexNodes + * @param adjust1 adjustment if match occurred in first list + * @param rexList2 second list of RexNodes + * @param adjust2 adjustment if match occurred in the second list + * @return index in the list corresponding to the matching RexNode; -1 + * if no match + */ + private int findExprInLists( + RexNode rex, + List rexList1, + int adjust1, + List rexList2, + int adjust2) { + int match = findExprInList(rex, rexList1); + if (match >= 0) { + return match + adjust1; + } + + if (rexList2 != null) { + match = findExprInList(rex, rexList2); + if (match >= 0) { + return match + adjust2; + } + } + + return -1; + } + + private int findExprInList(RexNode rex, List rexList) { + int match = 0; + for (RexNode rexElement : rexList) { + if (rexElement.toString().compareTo(rex.toString()) == 0) { + return match; + } + match++; + } + return -1; + } + } + + /** + * A functor that replies true or false for a given expression. + * + * @see org.apache.calcite.rel.rules.PushProjector.OperatorExprCondition + */ + public interface ExprCondition extends Predicate { + /** + * Evaluates a condition for a given expression. + * + * @param expr Expression + * @return result of evaluating the condition + */ + boolean test(RexNode expr); + + /** + * Constant condition that replies {@code false} for all expressions. + */ + ExprCondition FALSE = + new ExprConditionImpl() { + @Override public boolean test(RexNode expr) { + return false; + } + }; + + /** + * Constant condition that replies {@code true} for all expressions. + */ + ExprCondition TRUE = + new ExprConditionImpl() { + @Override public boolean test(RexNode expr) { + return true; + } + }; + } + + /** Implementation of {@link ExprCondition}. */ + abstract static class ExprConditionImpl extends PredicateImpl + implements ExprCondition { + } + + /** + * An expression condition that evaluates to true if the expression is + * a call to one of a set of operators. + */ + class OperatorExprCondition extends ExprConditionImpl { + private final Set operatorSet; + + /** + * Creates an OperatorExprCondition. + * + * @param operatorSet Set of operators + */ + public OperatorExprCondition(Iterable operatorSet) { + this.operatorSet = ImmutableSet.copyOf(operatorSet); + } + + public boolean test(RexNode expr) { + return expr instanceof RexCall + && operatorSet.contains(((RexCall) expr).getOperator()); + } + } +} + +// End PushProjector.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlGroupFunction.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlGroupFunction.java deleted file mode 100644 index a57cf104c17f2..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlGroupFunction.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.fun; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-1761 IS FIXED. - */ - -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlOperatorBinding; -import org.apache.calcite.sql.type.ReturnTypes; -import org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.calcite.sql.validate.SqlMonotonicity; - -import com.google.common.collect.ImmutableList; - -import java.util.List; - -/** - * SQL function that computes keys by which rows can be partitioned and - * aggregated. - * - *

Grouped window functions always occur in the GROUP BY clause. They often - * have auxiliary functions that access information about the group. For - * example, {@code HOP} is a group function, and its auxiliary functions are - * {@code HOP_START} and {@code HOP_END}. Here they are used in a streaming - * query: - * - *

- * SELECT STREAM HOP_START(rowtime, INTERVAL '1' HOUR),
- *   HOP_END(rowtime, INTERVAL '1' HOUR),
- *   MIN(unitPrice)
- * FROM Orders
- * GROUP BY HOP(rowtime, INTERVAL '1' HOUR), productId
- * 
- */ -class SqlGroupFunction extends SqlFunction { - /** The grouped function, if this an auxiliary function; null otherwise. */ - final SqlGroupFunction groupFunction; - - /** Creates a SqlGroupFunction. - * - * @param kind Kind; also determines function name - * @param groupFunction Group function, if this is an auxiliary; - * null, if this is a group function - * @param operandTypeChecker Operand type checker - */ - SqlGroupFunction(SqlKind kind, SqlGroupFunction groupFunction, - SqlOperandTypeChecker operandTypeChecker) { - super(kind.name(), kind, ReturnTypes.ARG0, null, - operandTypeChecker, SqlFunctionCategory.SYSTEM); - this.groupFunction = groupFunction; - if (groupFunction != null) { - assert groupFunction.groupFunction == null; - } - } - - /** Creates an auxiliary function from this grouped window function. */ - SqlGroupFunction auxiliary(SqlKind kind) { - return new SqlGroupFunction(kind, this, getOperandTypeChecker()); - } - - /** Returns a list of this grouped window function's auxiliary functions. */ - List getAuxiliaryFunctions() { - return ImmutableList.of(); - } - - @Override public boolean isGroup() { - // Auxiliary functions are not group functions - return groupFunction == null; - } - - @Override public boolean isGroupAuxiliary() { - return groupFunction != null; - } - - @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) { - // Monotonic iff its first argument is, but not strict. - // - // Note: This strategy happens to works for all current group functions - // (HOP, TUMBLE, SESSION). When there are exceptions to this rule, we'll - // make the method abstract. - return call.getOperandMonotonicity(0).unstrict(); - } -} - -// End SqlGroupFunction.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java deleted file mode 100644 index 6faf8e84d3668..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java +++ /dev/null @@ -1,2133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.fun; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-1761 IS FIXED. - */ - -import org.apache.calcite.avatica.util.TimeUnit; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlAsOperator; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlBinaryOperator; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlFilterOperator; -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlInternalOperator; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLateralOperator; -import org.apache.calcite.sql.SqlLiteral; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNumericLiteral; -import org.apache.calcite.sql.SqlOperandCountRange; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlOperatorBinding; -import org.apache.calcite.sql.SqlOverOperator; -import org.apache.calcite.sql.SqlPostfixOperator; -import org.apache.calcite.sql.SqlPrefixOperator; -import org.apache.calcite.sql.SqlProcedureCallOperator; -import org.apache.calcite.sql.SqlRankFunction; -import org.apache.calcite.sql.SqlSampleSpec; -import org.apache.calcite.sql.SqlSetOperator; -import org.apache.calcite.sql.SqlSpecialOperator; -import org.apache.calcite.sql.SqlUnnestOperator; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlValuesOperator; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.type.InferTypes; -import org.apache.calcite.sql.type.IntervalSqlType; -import org.apache.calcite.sql.type.OperandTypes; -import org.apache.calcite.sql.type.ReturnTypes; -import org.apache.calcite.sql.type.SqlOperandCountRanges; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable; -import org.apache.calcite.sql.validate.SqlModality; -import org.apache.calcite.sql2rel.AuxiliaryConverter; -import org.apache.calcite.util.Litmus; -import org.apache.calcite.util.Pair; - -import com.google.common.collect.ImmutableList; - -import java.util.List; - -/** - * Implementation of {@link org.apache.calcite.sql.SqlOperatorTable} containing - * the standard operators and functions. - */ -public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable { - //~ Static fields/initializers --------------------------------------------- - - /** - * The standard operator table. - */ - private static SqlStdOperatorTable instance; - - //------------------------------------------------------------- - // SET OPERATORS - //------------------------------------------------------------- - // The set operators can be compared to the arithmetic operators - // UNION -> + - // EXCEPT -> - - // INTERSECT -> * - // which explains the different precedence values - public static final SqlSetOperator UNION = - new SqlSetOperator("UNION", SqlKind.UNION, 14, false); - - public static final SqlSetOperator UNION_ALL = - new SqlSetOperator("UNION ALL", SqlKind.UNION, 14, true); - - public static final SqlSetOperator EXCEPT = - new SqlSetOperator("EXCEPT", SqlKind.EXCEPT, 14, false); - - public static final SqlSetOperator EXCEPT_ALL = - new SqlSetOperator("EXCEPT ALL", SqlKind.EXCEPT, 14, true); - - public static final SqlSetOperator INTERSECT = - new SqlSetOperator("INTERSECT", SqlKind.INTERSECT, 18, false); - - public static final SqlSetOperator INTERSECT_ALL = - new SqlSetOperator("INTERSECT ALL", SqlKind.INTERSECT, 18, true); - - /** - * The "MULTISET UNION" operator. - */ - public static final SqlMultisetSetOperator MULTISET_UNION = - new SqlMultisetSetOperator("MULTISET UNION", 14, false); - - /** - * The "MULTISET UNION ALL" operator. - */ - public static final SqlMultisetSetOperator MULTISET_UNION_ALL = - new SqlMultisetSetOperator("MULTISET UNION ALL", 14, true); - - /** - * The "MULTISET EXCEPT" operator. - */ - public static final SqlMultisetSetOperator MULTISET_EXCEPT = - new SqlMultisetSetOperator("MULTISET EXCEPT", 14, false); - - /** - * The "MULTISET EXCEPT ALL" operator. - */ - public static final SqlMultisetSetOperator MULTISET_EXCEPT_ALL = - new SqlMultisetSetOperator("MULTISET EXCEPT ALL", 14, true); - - /** - * The "MULTISET INTERSECT" operator. - */ - public static final SqlMultisetSetOperator MULTISET_INTERSECT = - new SqlMultisetSetOperator("MULTISET INTERSECT", 18, false); - - /** - * The "MULTISET INTERSECT ALL" operator. - */ - public static final SqlMultisetSetOperator MULTISET_INTERSECT_ALL = - new SqlMultisetSetOperator("MULTISET INTERSECT ALL", 18, true); - - //------------------------------------------------------------- - // BINARY OPERATORS - //------------------------------------------------------------- - - /** - * Logical AND operator. - */ - public static final SqlBinaryOperator AND = - new SqlBinaryOperator( - "AND", - SqlKind.AND, - 24, - true, - ReturnTypes.BOOLEAN_NULLABLE_OPTIMIZED, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN_BOOLEAN); - - /** - * AS operator associates an expression in the SELECT clause - * with an alias. - */ - public static final SqlAsOperator AS = new SqlAsOperator(); - - /** - * ARGUMENT_ASSIGNMENT operator (=<) - * assigns an argument to a function call to a particular named parameter. - */ - public static final SqlSpecialOperator ARGUMENT_ASSIGNMENT = - new SqlArgumentAssignmentOperator(); - - /** - * DEFAULT operator indicates that an argument to a function call - * is to take its default value.. - */ - public static final SqlSpecialOperator DEFAULT = new SqlDefaultOperator(); - - /** FILTER operator filters which rows are included in an - * aggregate function. */ - public static final SqlFilterOperator FILTER = new SqlFilterOperator(); - - /** {@code CUBE} operator, occurs within {@code GROUP BY} clause - * or nested within a {@code GROUPING SETS}. */ - public static final SqlInternalOperator CUBE = - new SqlRollupOperator("CUBE", SqlKind.CUBE); - - /** {@code ROLLUP} operator, occurs within {@code GROUP BY} clause - * or nested within a {@code GROUPING SETS}. */ - public static final SqlInternalOperator ROLLUP = - new SqlRollupOperator("ROLLUP", SqlKind.ROLLUP); - - /** {@code GROUPING SETS} operator, occurs within {@code GROUP BY} clause - * or nested within a {@code GROUPING SETS}. */ - public static final SqlInternalOperator GROUPING_SETS = - new SqlRollupOperator("GROUPING SETS", SqlKind.GROUPING_SETS); - - /** {@code GROUPING} function. Occurs in similar places to an aggregate - * function ({@code SELECT}, {@code HAVING} clause, etc. of an aggregate - * query), but not technically an aggregate function. */ - public static final SqlGroupingFunction GROUPING = - new SqlGroupingFunction("GROUPING"); - - /** {@code GROUP_ID} function. */ - public static final SqlGroupIdFunction GROUP_ID = - new SqlGroupIdFunction(); - - /** {@code GROUP_ID} function is a synonym for {@code GROUPING}. - * - *

Some history. The {@code GROUPING} function is in the SQL standard, - * and originally supported only one argument. The {@code GROUP_ID} is not - * standard (though supported in Oracle and SQL Server) and supports zero or - * more arguments. - * - *

The SQL standard has changed to allow {@code GROUPING} to have multiple - * arguments. It is now equivalent to {@code GROUP_ID}, so we made - * {@code GROUP_ID} a synonym for {@code GROUPING}. */ - public static final SqlGroupingFunction GROUPING_ID = - new SqlGroupingFunction("GROUPING_ID"); - - /** {@code EXTEND} operator. */ - public static final SqlInternalOperator EXTEND = new SqlExtendOperator(); - - /** - * String concatenation operator, '||'. - */ - public static final SqlBinaryOperator CONCAT = - new SqlBinaryOperator( - "||", - SqlKind.OTHER, - 60, - true, - ReturnTypes.DYADIC_STRING_SUM_PRECISION_NULLABLE, - null, - OperandTypes.STRING_SAME_SAME); - - /** - * Arithmetic division operator, '/'. - */ - public static final SqlBinaryOperator DIVIDE = - new SqlBinaryOperator( - "/", - SqlKind.DIVIDE, - 60, - true, - ReturnTypes.QUOTIENT_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.DIVISION_OPERATOR); - - /** The {@code RAND_INTEGER([seed, ] bound)} function, which yields a random - * integer, optionally with seed. */ - public static final SqlRandIntegerFunction RAND_INTEGER = - new SqlRandIntegerFunction(); - - /** The {@code RAND([seed])} function, which yields a random double, - * optionally with seed. */ - public static final SqlRandFunction RAND = new SqlRandFunction(); - - /** - * Internal integer arithmetic division operator, '/INT'. This - * is only used to adjust scale for numerics. We distinguish it from - * user-requested division since some personalities want a floating-point - * computation, whereas for the internal scaling use of division, we always - * want integer division. - */ - public static final SqlBinaryOperator DIVIDE_INTEGER = - new SqlBinaryOperator( - "/INT", - SqlKind.DIVIDE, - 60, - true, - ReturnTypes.INTEGER_QUOTIENT_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.DIVISION_OPERATOR); - - /** - * Dot operator, '.', used for referencing fields of records. - */ - public static final SqlBinaryOperator DOT = - new SqlBinaryOperator( - ".", - SqlKind.DOT, - 80, - true, - null, - null, - OperandTypes.ANY_ANY); - - /** - * Logical equals operator, '='. - */ - public static final SqlBinaryOperator EQUALS = - new SqlBinaryOperator( - "=", - SqlKind.EQUALS, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED); - - /** - * Logical greater-than operator, '>'. - */ - public static final SqlBinaryOperator GREATER_THAN = - new SqlBinaryOperator( - ">", - SqlKind.GREATER_THAN, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_ORDERED_COMPARABLE_ORDERED); - - /** - * IS DISTINCT FROM operator. - */ - public static final SqlBinaryOperator IS_DISTINCT_FROM = - new SqlBinaryOperator( - "IS DISTINCT FROM", - SqlKind.IS_DISTINCT_FROM, - 30, - true, - ReturnTypes.BOOLEAN, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED); - - /** - * IS NOT DISTINCT FROM operator. Is equivalent to NOT(x - * IS DISTINCT FROM y) - */ - public static final SqlBinaryOperator IS_NOT_DISTINCT_FROM = - new SqlBinaryOperator( - "IS NOT DISTINCT FROM", - SqlKind.IS_NOT_DISTINCT_FROM, - 30, - true, - ReturnTypes.BOOLEAN, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED); - - /** - * The internal $IS_DIFFERENT_FROM operator is the same as the - * user-level {@link #IS_DISTINCT_FROM} in all respects except that - * the test for equality on character datatypes treats trailing spaces as - * significant. - */ - public static final SqlBinaryOperator IS_DIFFERENT_FROM = - new SqlBinaryOperator( - "$IS_DIFFERENT_FROM", - SqlKind.OTHER, - 30, - true, - ReturnTypes.BOOLEAN, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED); - - /** - * Logical greater-than-or-equal operator, '>='. - */ - public static final SqlBinaryOperator GREATER_THAN_OR_EQUAL = - new SqlBinaryOperator( - ">=", - SqlKind.GREATER_THAN_OR_EQUAL, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_ORDERED_COMPARABLE_ORDERED); - - /** - * IN operator tests for a value's membership in a sub-query or - * a list of values. - */ - public static final SqlBinaryOperator IN = new SqlInOperator(false); - - /** - * NOT IN operator tests for a value's membership in a sub-query - * or a list of values. - */ - public static final SqlBinaryOperator NOT_IN = - new SqlInOperator(true); - - /** - * Logical less-than operator, '<'. - */ - public static final SqlBinaryOperator LESS_THAN = - new SqlBinaryOperator( - "<", - SqlKind.LESS_THAN, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_ORDERED_COMPARABLE_ORDERED); - - /** - * Logical less-than-or-equal operator, '<='. - */ - public static final SqlBinaryOperator LESS_THAN_OR_EQUAL = - new SqlBinaryOperator( - "<=", - SqlKind.LESS_THAN_OR_EQUAL, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_ORDERED_COMPARABLE_ORDERED); - - /** - * Infix arithmetic minus operator, '-'. - * - *

Its precedence is less than the prefix {@link #UNARY_PLUS +} - * and {@link #UNARY_MINUS -} operators. - */ - public static final SqlBinaryOperator MINUS = - new SqlMonotonicBinaryOperator( - "-", - SqlKind.MINUS, - 40, - true, - - // Same type inference strategy as sum - ReturnTypes.NULLABLE_SUM, - InferTypes.FIRST_KNOWN, - OperandTypes.MINUS_OPERATOR); - - /** - * Arithmetic multiplication operator, '*'. - */ - public static final SqlBinaryOperator MULTIPLY = - new SqlMonotonicBinaryOperator( - "*", - SqlKind.TIMES, - 60, - true, - ReturnTypes.PRODUCT_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.MULTIPLY_OPERATOR); - - /** - * Logical not-equals operator, '<>'. - */ - public static final SqlBinaryOperator NOT_EQUALS = - new SqlBinaryOperator( - "<>", - SqlKind.NOT_EQUALS, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - InferTypes.FIRST_KNOWN, - OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED); - - /** - * Logical OR operator. - */ - public static final SqlBinaryOperator OR = - new SqlBinaryOperator( - "OR", - SqlKind.OR, - 22, - true, - ReturnTypes.BOOLEAN_NULLABLE_OPTIMIZED, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN_BOOLEAN); - - /** - * Infix arithmetic plus operator, '+'. - */ - public static final SqlBinaryOperator PLUS = - new SqlMonotonicBinaryOperator( - "+", - SqlKind.PLUS, - 40, - true, - ReturnTypes.NULLABLE_SUM, - InferTypes.FIRST_KNOWN, - OperandTypes.PLUS_OPERATOR); - - /** - * Infix datetime plus operator, 'DATETIME + INTERVAL'. - */ - public static final SqlSpecialOperator DATETIME_PLUS = - new SqlSpecialOperator("DATETIME_PLUS", SqlKind.PLUS, 40, true, null, - InferTypes.FIRST_KNOWN, OperandTypes.PLUS_OPERATOR) { - @Override public RelDataType - inferReturnType(SqlOperatorBinding opBinding) { - final RelDataTypeFactory typeFactory = opBinding.getTypeFactory(); - final RelDataType leftType = opBinding.getOperandType(0); - final IntervalSqlType unitType = - (IntervalSqlType) opBinding.getOperandType(1); - switch (unitType.getIntervalQualifier().getStartUnit()) { - case HOUR: - case MINUTE: - case SECOND: - case MILLISECOND: - case MICROSECOND: - return typeFactory.createTypeWithNullability( - typeFactory.createSqlType(SqlTypeName.TIMESTAMP), - leftType.isNullable() || unitType.isNullable()); - default: - return leftType; - } - } - }; - - /** - * Multiset {@code MEMBER OF}, which returns whether a element belongs to a - * multiset. - * - *

For example, the following returns false: - * - *

- * 'green' MEMBER OF MULTISET ['red','almost green','blue'] - *
- */ - public static final SqlBinaryOperator MEMBER_OF = - new SqlMultisetMemberOfOperator(); - - /** - * Submultiset. Checks to see if an multiset is a sub-set of another - * multiset. - * - *

For example, the following returns false: - * - *

- * MULTISET ['green'] SUBMULTISET OF - * MULTISET['red', 'almost green', 'blue'] - *
- * - *

The following returns true, in part because multisets are - * order-independent: - * - *

- * MULTISET ['blue', 'red'] SUBMULTISET OF - * MULTISET ['red', 'almost green', 'blue'] - *
- */ - public static final SqlBinaryOperator SUBMULTISET_OF = - - // TODO: check if precedence is correct - new SqlBinaryOperator( - "SUBMULTISET OF", - SqlKind.OTHER, - 30, - true, - ReturnTypes.BOOLEAN_NULLABLE, - null, - OperandTypes.MULTISET_MULTISET); - - //------------------------------------------------------------- - // POSTFIX OPERATORS - //------------------------------------------------------------- - public static final SqlPostfixOperator DESC = - new SqlPostfixOperator( - "DESC", - SqlKind.DESCENDING, - 20, - ReturnTypes.ARG0, - InferTypes.RETURN_TYPE, - OperandTypes.ANY); - - public static final SqlPostfixOperator NULLS_FIRST = - new SqlPostfixOperator( - "NULLS FIRST", - SqlKind.NULLS_FIRST, - 18, - ReturnTypes.ARG0, - InferTypes.RETURN_TYPE, - OperandTypes.ANY); - - public static final SqlPostfixOperator NULLS_LAST = - new SqlPostfixOperator( - "NULLS LAST", - SqlKind.NULLS_LAST, - 18, - ReturnTypes.ARG0, - InferTypes.RETURN_TYPE, - OperandTypes.ANY); - - public static final SqlPostfixOperator IS_NOT_NULL = - new SqlPostfixOperator( - "IS NOT NULL", - SqlKind.IS_NOT_NULL, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.VARCHAR_1024, - OperandTypes.ANY); - - public static final SqlPostfixOperator IS_NULL = - new SqlPostfixOperator( - "IS NULL", - SqlKind.IS_NULL, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.VARCHAR_1024, - OperandTypes.ANY); - - public static final SqlPostfixOperator IS_NOT_TRUE = - new SqlPostfixOperator( - "IS NOT TRUE", - SqlKind.IS_NOT_TRUE, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_TRUE = - new SqlPostfixOperator( - "IS TRUE", - SqlKind.IS_TRUE, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_NOT_FALSE = - new SqlPostfixOperator( - "IS NOT FALSE", - SqlKind.IS_NOT_FALSE, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_FALSE = - new SqlPostfixOperator( - "IS FALSE", - SqlKind.IS_FALSE, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_NOT_UNKNOWN = - new SqlPostfixOperator( - "IS NOT UNKNOWN", - SqlKind.IS_NOT_NULL, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_UNKNOWN = - new SqlPostfixOperator( - "IS UNKNOWN", - SqlKind.IS_NULL, - 28, - ReturnTypes.BOOLEAN_NOT_NULL, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - public static final SqlPostfixOperator IS_A_SET = - new SqlPostfixOperator( - "IS A SET", - SqlKind.OTHER, - 28, - ReturnTypes.BOOLEAN, - null, - OperandTypes.MULTISET); - - //------------------------------------------------------------- - // PREFIX OPERATORS - //------------------------------------------------------------- - public static final SqlPrefixOperator EXISTS = - new SqlPrefixOperator( - "EXISTS", - SqlKind.EXISTS, - 40, - ReturnTypes.BOOLEAN, - null, - OperandTypes.ANY) { - public boolean argumentMustBeScalar(int ordinal) { - return false; - } - - @Override public boolean validRexOperands(int count, Litmus litmus) { - if (count != 0) { - return litmus.fail("wrong operand count {} for {}", count, this); - } - return litmus.succeed(); - } - }; - - public static final SqlPrefixOperator NOT = - new SqlPrefixOperator( - "NOT", - SqlKind.NOT, - 26, - ReturnTypes.ARG0, - InferTypes.BOOLEAN, - OperandTypes.BOOLEAN); - - /** - * Prefix arithmetic minus operator, '-'. - * - *

Its precedence is greater than the infix '{@link #PLUS +}' and - * '{@link #MINUS -}' operators. - */ - public static final SqlPrefixOperator UNARY_MINUS = - new SqlPrefixOperator( - "-", - SqlKind.MINUS_PREFIX, - 80, - ReturnTypes.ARG0, - InferTypes.RETURN_TYPE, - OperandTypes.NUMERIC_OR_INTERVAL); - - /** - * Prefix arithmetic plus operator, '+'. - * - *

Its precedence is greater than the infix '{@link #PLUS +}' and - * '{@link #MINUS -}' operators. - */ - public static final SqlPrefixOperator UNARY_PLUS = - new SqlPrefixOperator( - "+", - SqlKind.PLUS_PREFIX, - 80, - ReturnTypes.ARG0, - InferTypes.RETURN_TYPE, - OperandTypes.NUMERIC_OR_INTERVAL); - - /** - * Keyword which allows an identifier to be explicitly flagged as a table. - * For example, select * from (TABLE t) or TABLE - * t. See also {@link #COLLECTION_TABLE}. - */ - public static final SqlPrefixOperator EXPLICIT_TABLE = - new SqlPrefixOperator( - "TABLE", - SqlKind.EXPLICIT_TABLE, - 2, - null, - null, - null); - - //------------------------------------------------------------- - // AGGREGATE OPERATORS - //------------------------------------------------------------- - /** - * SUM aggregate function. - */ - public static final SqlAggFunction SUM = new SqlSumAggFunction(null); - - /** - * COUNT aggregate function. - */ - public static final SqlAggFunction COUNT = new SqlCountAggFunction(); - - /** - * MIN aggregate function. - */ - public static final SqlAggFunction MIN = - new SqlMinMaxAggFunction(SqlKind.MIN); - - /** - * MAX aggregate function. - */ - public static final SqlAggFunction MAX = - new SqlMinMaxAggFunction(SqlKind.MAX); - - /** - * LAST_VALUE aggregate function. - */ - public static final SqlAggFunction LAST_VALUE = - new SqlFirstLastValueAggFunction(SqlKind.LAST_VALUE); - - /** - * FIRST_VALUE aggregate function. - */ - public static final SqlAggFunction FIRST_VALUE = - new SqlFirstLastValueAggFunction(SqlKind.FIRST_VALUE); - - /** - * LEAD aggregate function. - */ - public static final SqlAggFunction LEAD = - new SqlLeadLagAggFunction(SqlKind.LEAD); - - /** - * LAG aggregate function. - */ - public static final SqlAggFunction LAG = - new SqlLeadLagAggFunction(SqlKind.LAG); - - /** - * NTILE aggregate function. - */ - public static final SqlAggFunction NTILE = - new SqlNtileAggFunction(); - - /** - * SINGLE_VALUE aggregate function. - */ - public static final SqlAggFunction SINGLE_VALUE = - new SqlSingleValueAggFunction(null); - - /** - * AVG aggregate function. - */ - public static final SqlAggFunction AVG = - new SqlAvgAggFunction(SqlKind.AVG); - - /** - * STDDEV_POP aggregate function. - */ - public static final SqlAggFunction STDDEV_POP = - new SqlAvgAggFunction(SqlKind.STDDEV_POP); - - /** - * REGR_SXX aggregate function. - */ - public static final SqlAggFunction REGR_SXX = - new SqlCovarAggFunction(SqlKind.REGR_SXX); - - /** - * REGR_SYY aggregate function. - */ - public static final SqlAggFunction REGR_SYY = - new SqlCovarAggFunction(SqlKind.REGR_SYY); - - /** - * COVAR_POP aggregate function. - */ - public static final SqlAggFunction COVAR_POP = - new SqlCovarAggFunction(SqlKind.COVAR_POP); - - /** - * COVAR_SAMP aggregate function. - */ - public static final SqlAggFunction COVAR_SAMP = - new SqlCovarAggFunction(SqlKind.COVAR_SAMP); - - /** - * STDDEV_SAMP aggregate function. - */ - public static final SqlAggFunction STDDEV_SAMP = - new SqlAvgAggFunction(SqlKind.STDDEV_SAMP); - - /** - * VAR_POP aggregate function. - */ - public static final SqlAggFunction VAR_POP = - new SqlAvgAggFunction(SqlKind.VAR_POP); - - /** - * VAR_SAMP aggregate function. - */ - public static final SqlAggFunction VAR_SAMP = - new SqlAvgAggFunction(SqlKind.VAR_SAMP); - - //------------------------------------------------------------- - // WINDOW Aggregate Functions - //------------------------------------------------------------- - /** - * HISTOGRAM aggregate function support. Used by window - * aggregate versions of MIN/MAX - */ - public static final SqlAggFunction HISTOGRAM_AGG = - new SqlHistogramAggFunction(null); - - /** - * HISTOGRAM_MIN window aggregate function. - */ - public static final SqlFunction HISTOGRAM_MIN = - new SqlFunction( - "$HISTOGRAM_MIN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.NUMERIC_OR_STRING, - SqlFunctionCategory.NUMERIC); - - /** - * HISTOGRAM_MAX window aggregate function. - */ - public static final SqlFunction HISTOGRAM_MAX = - new SqlFunction( - "$HISTOGRAM_MAX", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.NUMERIC_OR_STRING, - SqlFunctionCategory.NUMERIC); - - /** - * HISTOGRAM_FIRST_VALUE window aggregate function. - */ - public static final SqlFunction HISTOGRAM_FIRST_VALUE = - new SqlFunction( - "$HISTOGRAM_FIRST_VALUE", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.NUMERIC_OR_STRING, - SqlFunctionCategory.NUMERIC); - - /** - * HISTOGRAM_LAST_VALUE window aggregate function. - */ - public static final SqlFunction HISTOGRAM_LAST_VALUE = - new SqlFunction( - "$HISTOGRAM_LAST_VALUE", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.NUMERIC_OR_STRING, - SqlFunctionCategory.NUMERIC); - - /** - * SUM0 aggregate function. - */ - public static final SqlAggFunction SUM0 = - new SqlSumEmptyIsZeroAggFunction(); - - //------------------------------------------------------------- - // WINDOW Rank Functions - //------------------------------------------------------------- - /** - * CUME_DIST window function. - */ - public static final SqlRankFunction CUME_DIST = - new SqlRankFunction(true, SqlKind.CUME_DIST); - - /** - * DENSE_RANK window function. - */ - public static final SqlRankFunction DENSE_RANK = - new SqlRankFunction(true, SqlKind.DENSE_RANK); - - /** - * PERCENT_RANK window function. - */ - public static final SqlRankFunction PERCENT_RANK = - new SqlRankFunction(true, SqlKind.PERCENT_RANK); - - /** - * RANK window function. - */ - public static final SqlRankFunction RANK = - new SqlRankFunction(true, SqlKind.RANK); - - /** - * ROW_NUMBER window function. - */ - public static final SqlRankFunction ROW_NUMBER = - new SqlRankFunction(false, SqlKind.ROW_NUMBER); - - //------------------------------------------------------------- - // SPECIAL OPERATORS - //------------------------------------------------------------- - public static final SqlRowOperator ROW = new SqlRowOperator("ROW"); - - /** - * A special operator for the subtraction of two DATETIMEs. The format of - * DATETIME subtraction is: - * - *

"(" <datetime> "-" <datetime> ")" - * <interval qualifier>
- * - *

This operator is special since it needs to hold the - * additional interval qualifier specification.

- */ - public static final SqlDatetimeSubtractionOperator MINUS_DATE = - new SqlDatetimeSubtractionOperator(); - - /** - * The MULTISET Value Constructor. e.g. "MULTISET[1,2,3]". - */ - public static final SqlMultisetValueConstructor MULTISET_VALUE = - new SqlMultisetValueConstructor(); - - /** - * The MULTISET Query Constructor. e.g. "SELECT dname, MULTISET(SELECT - * FROM emp WHERE deptno = dept.deptno) FROM dept". - */ - public static final SqlMultisetQueryConstructor MULTISET_QUERY = - new SqlMultisetQueryConstructor(); - - /** - * The ARRAY Query Constructor. e.g. "SELECT dname, ARRAY(SELECT - * FROM emp WHERE deptno = dept.deptno) FROM dept". - */ - public static final SqlMultisetQueryConstructor ARRAY_QUERY = - new SqlArrayQueryConstructor(); - - /** - * The MAP Query Constructor. e.g. "MAP(SELECT empno, deptno - * FROM emp)". - */ - public static final SqlMultisetQueryConstructor MAP_QUERY = - new SqlMapQueryConstructor(); - - /** - * The CURSOR constructor. e.g. "SELECT * FROM - * TABLE(DEDUP(CURSOR(SELECT * FROM EMPS), 'name'))". - */ - public static final SqlCursorConstructor CURSOR = - new SqlCursorConstructor(); - - /** - * The COLUMN_LIST constructor. e.g. the ROW() call in "SELECT * FROM - * TABLE(DEDUP(CURSOR(SELECT * FROM EMPS), ROW(name, empno)))". - */ - public static final SqlColumnListConstructor COLUMN_LIST = - new SqlColumnListConstructor(); - - /** - * The UNNEST operator. - */ - public static final SqlUnnestOperator UNNEST = - new SqlUnnestOperator(false); - - /** - * The UNNEST WITH ORDINALITY operator. - */ - public static final SqlUnnestOperator UNNEST_WITH_ORDINALITY = - new SqlUnnestOperator(true); - - /** - * The LATERAL operator. - */ - public static final SqlSpecialOperator LATERAL = - new SqlLateralOperator(SqlKind.LATERAL); - - /** - * The "table function derived table" operator, which a table-valued - * function into a relation, e.g. "SELECT * FROM - * TABLE(ramp(5))". - * - *

This operator has function syntax (with one argument), whereas - * {@link #EXPLICIT_TABLE} is a prefix operator. - */ - public static final SqlSpecialOperator COLLECTION_TABLE = - new SqlCollectionTableOperator("TABLE", SqlModality.RELATION); - - public static final SqlOverlapsOperator OVERLAPS = - new SqlOverlapsOperator(); - - public static final SqlSpecialOperator VALUES = - new SqlValuesOperator(); - - public static final SqlLiteralChainOperator LITERAL_CHAIN = - new SqlLiteralChainOperator(); - - public static final SqlThrowOperator THROW = new SqlThrowOperator(); - - public static final SqlBetweenOperator BETWEEN = - new SqlBetweenOperator( - SqlBetweenOperator.Flag.ASYMMETRIC, - false); - - public static final SqlBetweenOperator SYMMETRIC_BETWEEN = - new SqlBetweenOperator( - SqlBetweenOperator.Flag.SYMMETRIC, - false); - - public static final SqlBetweenOperator NOT_BETWEEN = - new SqlBetweenOperator( - SqlBetweenOperator.Flag.ASYMMETRIC, - true); - - public static final SqlBetweenOperator SYMMETRIC_NOT_BETWEEN = - new SqlBetweenOperator( - SqlBetweenOperator.Flag.SYMMETRIC, - true); - - public static final SqlSpecialOperator NOT_LIKE = - new SqlLikeOperator("NOT LIKE", SqlKind.LIKE, true); - - public static final SqlSpecialOperator LIKE = - new SqlLikeOperator("LIKE", SqlKind.LIKE, false); - - public static final SqlSpecialOperator NOT_SIMILAR_TO = - new SqlLikeOperator("NOT SIMILAR TO", SqlKind.SIMILAR, true); - - public static final SqlSpecialOperator SIMILAR_TO = - new SqlLikeOperator("SIMILAR TO", SqlKind.SIMILAR, false); - - /** - * Internal operator used to represent the ESCAPE clause of a LIKE or - * SIMILAR TO expression. - */ - public static final SqlSpecialOperator ESCAPE = - new SqlSpecialOperator("ESCAPE", SqlKind.ESCAPE, 0); - - public static final SqlCaseOperator CASE = SqlCaseOperator.INSTANCE; - - public static final SqlOperator PROCEDURE_CALL = - new SqlProcedureCallOperator(); - - public static final SqlOperator NEW = new SqlNewOperator(); - - /** - * The OVER operator, which applies an aggregate functions to a - * {@link SqlWindow window}. - * - *

Operands are as follows: - * - *

    - *
  1. name of window function ({@link org.apache.calcite.sql.SqlCall})
  2. - *
  3. window name ({@link org.apache.calcite.sql.SqlLiteral}) or window - * in-line specification (@link SqlWindowOperator})
  4. - *
- */ - public static final SqlBinaryOperator OVER = new SqlOverOperator(); - - /** - * An REINTERPRET operator is internal to the planner. When the - * physical storage of two types is the same, this operator may be used to - * reinterpret values of one type as the other. This operator is similar to - * a cast, except that it does not alter the data value. Like a regular cast - * it accepts one operand and stores the target type as the return type. It - * performs an overflow check if it has any second operand, whether - * true or not. - */ - public static final SqlSpecialOperator REINTERPRET = - new SqlSpecialOperator("Reinterpret", SqlKind.REINTERPRET) { - public SqlOperandCountRange getOperandCountRange() { - return SqlOperandCountRanges.between(1, 2); - } - }; - - /** Internal operator that extracts time periods (year, month, date) from a - * date in internal format (number of days since epoch). */ - public static final SqlSpecialOperator EXTRACT_DATE = - new SqlSpecialOperator("EXTRACT_DATE", SqlKind.EXTRACT); - - //------------------------------------------------------------- - // FUNCTIONS - //------------------------------------------------------------- - - /** - * The character substring function: SUBSTRING(string FROM start [FOR - * length]). - * - *

If the length parameter is a constant, the length of the result is the - * minimum of the length of the input and that length. Otherwise it is the - * length of the input. - */ - public static final SqlFunction SUBSTRING = new SqlSubstringFunction(); - - /** The {@code REPLACE(string, search, replace)} function. Not standard SQL, - * but in Oracle and Postgres. */ - public static final SqlFunction REPLACE = - new SqlFunction("REPLACE", SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE_VARYING, null, - OperandTypes.STRING_STRING_STRING, SqlFunctionCategory.STRING); - - public static final SqlFunction CONVERT = - new SqlConvertFunction("CONVERT"); - - /** - * The TRANSLATE(char_value USING translation_name) function - * alters the character set of a string value from one base character set to another. - * - *

It is defined in the SQL standard. See also non-standard - * {@link OracleSqlOperatorTable#TRANSLATE3}. - */ - public static final SqlFunction TRANSLATE = - new SqlConvertFunction("TRANSLATE"); - - public static final SqlFunction OVERLAY = new SqlOverlayFunction(); - - /** The "TRIM" function. */ - public static final SqlFunction TRIM = SqlTrimFunction.INSTANCE; - - public static final SqlFunction POSITION = new SqlPositionFunction(); - - public static final SqlFunction CHAR_LENGTH = - new SqlFunction( - "CHAR_LENGTH", - SqlKind.OTHER_FUNCTION, - ReturnTypes.INTEGER_NULLABLE, - null, - OperandTypes.CHARACTER, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction CHARACTER_LENGTH = - new SqlFunction( - "CHARACTER_LENGTH", - SqlKind.OTHER_FUNCTION, - ReturnTypes.INTEGER_NULLABLE, - null, - OperandTypes.CHARACTER, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction UPPER = - new SqlFunction( - "UPPER", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.CHARACTER, - SqlFunctionCategory.STRING); - - public static final SqlFunction LOWER = - new SqlFunction( - "LOWER", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.CHARACTER, - SqlFunctionCategory.STRING); - - public static final SqlFunction INITCAP = - new SqlFunction( - "INITCAP", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0_NULLABLE, - null, - OperandTypes.CHARACTER, - SqlFunctionCategory.STRING); - - /** - * Uses SqlOperatorTable.useDouble for its return type since we don't know - * what the result type will be by just looking at the operand types. For - * example POW(int, int) can return a non integer if the second operand is - * negative. - */ - public static final SqlFunction POWER = - new SqlFunction( - "POWER", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC_NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction SQRT = - new SqlFunction( - "SQRT", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction MOD = - // Return type is same as divisor (2nd operand) - // SQL2003 Part2 Section 6.27, Syntax Rules 9 - new SqlFunction( - "MOD", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG1_NULLABLE, - null, - OperandTypes.EXACT_NUMERIC_EXACT_NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction LN = - new SqlFunction( - "LN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction LOG10 = - new SqlFunction( - "LOG10", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ABS = - new SqlFunction( - "ABS", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0, - null, - OperandTypes.NUMERIC_OR_INTERVAL, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ACOS = - new SqlFunction( - "ACOS", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ASIN = - new SqlFunction( - "ASIN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ATAN = - new SqlFunction( - "ATAN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ATAN2 = - new SqlFunction( - "ATAN2", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC_NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction COS = - new SqlFunction( - "COS", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction COT = - new SqlFunction( - "COT", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction DEGREES = - new SqlFunction( - "DEGREES", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction EXP = - new SqlFunction( - "EXP", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction RADIANS = - new SqlFunction( - "RADIANS", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction ROUND = - new SqlFunction( - "ROUND", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0, - null, - OperandTypes.NUMERIC_INTEGER, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction SIGN = - new SqlFunction( - "SIGN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction SIN = - new SqlFunction( - "SIN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - - public static final SqlFunction TAN = - new SqlFunction( - "TAN", - SqlKind.OTHER_FUNCTION, - ReturnTypes.DOUBLE_NULLABLE, - null, - OperandTypes.NUMERIC, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction TRUNCATE = - new SqlFunction( - "TRUNCATE", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0, - null, - OperandTypes.NUMERIC_INTEGER, - SqlFunctionCategory.NUMERIC); - - public static final SqlFunction PI = - new SqlBaseContextVariable("PI", ReturnTypes.DOUBLE, - SqlFunctionCategory.NUMERIC); - - /** {@code FINAL} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction FINAL = - new SqlFunction("FINAL", SqlKind.FINAL, ReturnTypes.ARG0_NULLABLE, null, - OperandTypes.ANY, SqlFunctionCategory.MATCH_RECOGNIZE); - - /** {@code RUNNING} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction RUNNING = - new SqlFunction("RUNNING", SqlKind.RUNNING, ReturnTypes.ARG0_NULLABLE, - null, OperandTypes.ANY, SqlFunctionCategory.MATCH_RECOGNIZE); - - /** {@code FIRST} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction FIRST = - new SqlFunction("FIRST", SqlKind.FIRST, ReturnTypes.ARG0_NULLABLE, - null, OperandTypes.ANY_NUMERIC, SqlFunctionCategory.MATCH_RECOGNIZE); - - /** {@code LAST} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction LAST = - new SqlFunction("LAST", SqlKind.LAST, ReturnTypes.ARG0_NULLABLE, - null, OperandTypes.ANY_NUMERIC, SqlFunctionCategory.MATCH_RECOGNIZE); - - /** {@code PREV} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction PREV = - new SqlFunction("PREV", SqlKind.PREV, ReturnTypes.ARG0_NULLABLE, - null, OperandTypes.ANY_NUMERIC, SqlFunctionCategory.MATCH_RECOGNIZE); - - /** {@code NEXT} function to be used within {@code MATCH_RECOGNIZE}. */ - public static final SqlFunction NEXT = - new SqlFunction("NEXT", SqlKind.NEXT, ReturnTypes.ARG0_NULLABLE, null, - OperandTypes.ANY_NUMERIC, SqlFunctionCategory.MATCH_RECOGNIZE); - - public static final SqlFunction NULLIF = new SqlNullifFunction(); - - /** - * The COALESCE builtin function. - */ - public static final SqlFunction COALESCE = new SqlCoalesceFunction(); - - /** - * The FLOOR function. - */ - public static final SqlFunction FLOOR = new SqlFloorFunction(SqlKind.FLOOR); - - /** - * The CEIL function. - */ - public static final SqlFunction CEIL = new SqlFloorFunction(SqlKind.CEIL); - - /** - * The USER function. - */ - public static final SqlFunction USER = - new SqlStringContextVariable("USER"); - - /** - * The CURRENT_USER function. - */ - public static final SqlFunction CURRENT_USER = - new SqlStringContextVariable("CURRENT_USER"); - - /** - * The SESSION_USER function. - */ - public static final SqlFunction SESSION_USER = - new SqlStringContextVariable("SESSION_USER"); - - /** - * The SYSTEM_USER function. - */ - public static final SqlFunction SYSTEM_USER = - new SqlStringContextVariable("SYSTEM_USER"); - - /** - * The CURRENT_PATH function. - */ - public static final SqlFunction CURRENT_PATH = - new SqlStringContextVariable("CURRENT_PATH"); - - /** - * The CURRENT_ROLE function. - */ - public static final SqlFunction CURRENT_ROLE = - new SqlStringContextVariable("CURRENT_ROLE"); - - /** - * The CURRENT_CATALOG function. - */ - public static final SqlFunction CURRENT_CATALOG = - new SqlStringContextVariable("CURRENT_CATALOG"); - - /** - * The CURRENT_SCHEMA function. - */ - public static final SqlFunction CURRENT_SCHEMA = - new SqlStringContextVariable("CURRENT_SCHEMA"); - - /** - * The LOCALTIME [(precision)] function. - */ - public static final SqlFunction LOCALTIME = - new SqlAbstractTimeFunction("LOCALTIME", SqlTypeName.TIME); - - /** - * The LOCALTIMESTAMP [(precision)] function. - */ - public static final SqlFunction LOCALTIMESTAMP = - new SqlAbstractTimeFunction("LOCALTIMESTAMP", SqlTypeName.TIMESTAMP); - - /** - * The CURRENT_TIME [(precision)] function. - */ - public static final SqlFunction CURRENT_TIME = - new SqlAbstractTimeFunction("CURRENT_TIME", SqlTypeName.TIME); - - /** - * The CURRENT_TIMESTAMP [(precision)] function. - */ - public static final SqlFunction CURRENT_TIMESTAMP = - new SqlAbstractTimeFunction("CURRENT_TIMESTAMP", SqlTypeName.TIMESTAMP); - - /** - * The CURRENT_DATE function. - */ - public static final SqlFunction CURRENT_DATE = - new SqlCurrentDateFunction(); - - /** The TIMESTAMPADD function. */ - public static final SqlFunction TIMESTAMP_ADD = new SqlTimestampAddFunction(); - - /** The TIMESTAMPDIFF function. */ - public static final SqlFunction TIMESTAMP_DIFF = new SqlTimestampDiffFunction(); - - /** - * Use of the IN_FENNEL operator forces the argument to be - * evaluated in Fennel. Otherwise acts as identity function. - */ - public static final SqlFunction IN_FENNEL = - new SqlMonotonicUnaryFunction( - "IN_FENNEL", - SqlKind.OTHER_FUNCTION, - ReturnTypes.ARG0, - null, - OperandTypes.ANY, - SqlFunctionCategory.SYSTEM); - - /** - * The SQL CAST operator. - * - *

The SQL syntax is - * - *

CAST(expression AS type) - *
- * - *

When the CAST operator is applies as a {@link SqlCall}, it has two - * arguments: the expression and the type. The type must not include a - * constraint, so CAST(x AS INTEGER NOT NULL), for instance, is - * invalid.

- * - *

When the CAST operator is applied as a RexCall, the - * target type is simply stored as the return type, not an explicit operand. - * For example, the expression CAST(1 + 2 AS DOUBLE) will - * become a call to CAST with the expression 1 + 2 - * as its only operand.

- * - *

The RexCall form can also have a type which contains a - * NOT NULL constraint. When this expression is implemented, if - * the value is NULL, an exception will be thrown.

- */ - public static final SqlFunction CAST = new SqlCastFunction(); - - /** - * The SQL EXTRACT operator. Extracts a specified field value - * from a DATETIME or an INTERVAL. E.g.
- * EXTRACT(HOUR FROM INTERVAL '364 23:59:59') returns - * 23 - */ - public static final SqlFunction EXTRACT = new SqlExtractFunction(); - - /** - * The SQL YEAR operator. Returns the Year - * from a DATETIME E.g.
- * YEAR(date '2008-9-23') returns - * 2008 - */ - public static final SqlDatePartFunction YEAR = - new SqlDatePartFunction("YEAR", TimeUnit.YEAR); - - /** - * The SQL QUARTER operator. Returns the Quarter - * from a DATETIME E.g.
- * QUARTER(date '2008-9-23') returns - * 3 - */ - public static final SqlDatePartFunction QUARTER = - new SqlDatePartFunction("QUARTER", TimeUnit.QUARTER); - - /** - * The SQL MONTH operator. Returns the Month - * from a DATETIME E.g.
- * MONTH(date '2008-9-23') returns - * 9 - */ - public static final SqlDatePartFunction MONTH = - new SqlDatePartFunction("MONTH", TimeUnit.MONTH); - - /** - * The SQL WEEK operator. Returns the Week - * from a DATETIME E.g.
- * WEEK(date '2008-9-23') returns - * 39 - */ - public static final SqlDatePartFunction WEEK = - new SqlDatePartFunction("WEEK", TimeUnit.WEEK); - - /** - * The SQL DAYOFYEAR operator. Returns the DOY - * from a DATETIME E.g.
- * DAYOFYEAR(date '2008-9-23') returns - * 267 - */ - public static final SqlDatePartFunction DAYOFYEAR = - new SqlDatePartFunction("DAYOFYEAR", TimeUnit.DOY); - - /** - * The SQL DAYOFMONTH operator. Returns the Day - * from a DATETIME E.g.
- * DAYOFMONTH(date '2008-9-23') returns - * 23 - */ - public static final SqlDatePartFunction DAYOFMONTH = - new SqlDatePartFunction("DAYOFMONTH", TimeUnit.DAY); - - /** - * The SQL DAYOFWEEK operator. Returns the DOW - * from a DATETIME E.g.
- * DAYOFWEEK(date '2008-9-23') returns - * 2 - */ - public static final SqlDatePartFunction DAYOFWEEK = - new SqlDatePartFunction("DAYOFWEEK", TimeUnit.DOW); - - /** - * The SQL HOUR operator. Returns the Hour - * from a DATETIME E.g.
- * HOUR(timestamp '2008-9-23 01:23:45') returns - * 1 - */ - public static final SqlDatePartFunction HOUR = - new SqlDatePartFunction("HOUR", TimeUnit.HOUR); - - /** - * The SQL MINUTE operator. Returns the Minute - * from a DATETIME E.g.
- * MINUTE(timestamp '2008-9-23 01:23:45') returns - * 23 - */ - public static final SqlDatePartFunction MINUTE = - new SqlDatePartFunction("MINUTE", TimeUnit.MINUTE); - - /** - * The SQL SECOND operator. Returns the Second - * from a DATETIME E.g.
- * SECOND(timestamp '2008-9-23 01:23:45') returns - * 45 - */ - public static final SqlDatePartFunction SECOND = - new SqlDatePartFunction("SECOND", TimeUnit.SECOND); - - /** - * The ELEMENT operator, used to convert a multiset with only one item to a - * "regular" type. Example ... log(ELEMENT(MULTISET[1])) ... - */ - public static final SqlFunction ELEMENT = - new SqlFunction( - "ELEMENT", - SqlKind.OTHER_FUNCTION, - ReturnTypes.MULTISET_ELEMENT_NULLABLE, - null, - OperandTypes.COLLECTION, - SqlFunctionCategory.SYSTEM); - - /** - * The item operator {@code [ ... ]}, used to access a given element of an - * array or map. For example, {@code myArray[3]} or {@code "myMap['foo']"}. - * - *

The SQL standard calls the ARRAY variant a - * <array element reference>. Index is 1-based. The standard says - * to raise "data exception - array element error" but we currently return - * null.

- * - *

MAP is not standard SQL.

- */ - public static final SqlOperator ITEM = new SqlItemOperator(); - - /** - * The ARRAY Value Constructor. e.g. "ARRAY[1, 2, 3]". - */ - public static final SqlArrayValueConstructor ARRAY_VALUE_CONSTRUCTOR = - new SqlArrayValueConstructor(); - - /** - * The MAP Value Constructor, - * e.g. "MAP['washington', 1, 'obama', 44]". - */ - public static final SqlMapValueConstructor MAP_VALUE_CONSTRUCTOR = - new SqlMapValueConstructor(); - - /** - * The internal "$SLICE" operator takes a multiset of records and returns a - * multiset of the first column of those records. - * - *

It is introduced when multisets of scalar types are created, in order - * to keep types consistent. For example, MULTISET [5] has type - * INTEGER MULTISET but is translated to an expression of type - * RECORD(INTEGER EXPR$0) MULTISET because in our internal - * representation of multisets, every element must be a record. Applying the - * "$SLICE" operator to this result converts the type back to an - * INTEGER MULTISET multiset value. - * - *

$SLICE is often translated away when the multiset type is - * converted back to scalar values. - */ - public static final SqlInternalOperator SLICE = - new SqlInternalOperator( - "$SLICE", - SqlKind.OTHER, - 0, - false, - ReturnTypes.MULTISET_PROJECT0, - null, - OperandTypes.RECORD_COLLECTION) { - }; - - /** - * The internal "$ELEMENT_SLICE" operator returns the first field of the - * only element of a multiset. - * - *

It is introduced when multisets of scalar types are created, in order - * to keep types consistent. For example, ELEMENT(MULTISET [5]) - * is translated to $ELEMENT_SLICE(MULTISET (VALUES ROW (5 - * EXPR$0)) It is translated away when the multiset type is converted - * back to scalar values.

- * - *

NOTE: jhyde, 2006/1/9: Usages of this operator are commented out, but - * I'm not deleting the operator, because some multiset tests are disabled, - * and we may need this operator to get them working!

- */ - public static final SqlInternalOperator ELEMENT_SLICE = - new SqlInternalOperator( - "$ELEMENT_SLICE", - SqlKind.OTHER, - 0, - false, - ReturnTypes.MULTISET_RECORD, - null, - OperandTypes.MULTISET) { - public void unparse( - SqlWriter writer, - SqlCall call, - int leftPrec, - int rightPrec) { - SqlUtil.unparseFunctionSyntax( - this, - writer, call); - } - }; - - /** - * The internal "$SCALAR_QUERY" operator returns a scalar value from a - * record type. It assumes the record type only has one field, and returns - * that field as the output. - */ - public static final SqlInternalOperator SCALAR_QUERY = - new SqlInternalOperator( - "$SCALAR_QUERY", - SqlKind.SCALAR_QUERY, - 0, - false, - ReturnTypes.RECORD_TO_SCALAR, - null, - OperandTypes.RECORD_TO_SCALAR) { - public void unparse( - SqlWriter writer, - SqlCall call, - int leftPrec, - int rightPrec) { - final SqlWriter.Frame frame = writer.startList("(", ")"); - call.operand(0).unparse(writer, 0, 0); - writer.endList(frame); - } - - public boolean argumentMustBeScalar(int ordinal) { - // Obvious, really. - return false; - } - }; - - /** - * The CARDINALITY operator, used to retrieve the number of elements in a - * MULTISET, ARRAY or MAP. - */ - public static final SqlFunction CARDINALITY = - new SqlFunction( - "CARDINALITY", - SqlKind.OTHER_FUNCTION, - ReturnTypes.INTEGER_NULLABLE, - null, - OperandTypes.COLLECTION_OR_MAP, - SqlFunctionCategory.SYSTEM); - - /** - * The COLLECT operator. Multiset aggregator function. - */ - public static final SqlAggFunction COLLECT = - new SqlAggFunction("COLLECT", - null, - SqlKind.COLLECT, - ReturnTypes.TO_MULTISET, - null, - OperandTypes.ANY, - SqlFunctionCategory.SYSTEM, false, false) { - }; - - /** - * The FUSION operator. Multiset aggregator function. - */ - public static final SqlFunction FUSION = - new SqlAggFunction("FUSION", null, - SqlKind.FUSION, - ReturnTypes.ARG0, - null, - OperandTypes.MULTISET, - SqlFunctionCategory.SYSTEM, false, false) { - }; - - /** - * The sequence next value function: NEXT VALUE FOR sequence - */ - public static final SqlOperator NEXT_VALUE = - new SqlSequenceValueOperator(SqlKind.NEXT_VALUE); - - /** - * The sequence current value function: CURRENT VALUE FOR - * sequence - */ - public static final SqlOperator CURRENT_VALUE = - new SqlSequenceValueOperator(SqlKind.CURRENT_VALUE); - - /** - * The TABLESAMPLE operator. - * - *

Examples: - * - *

    - *
  • <query> TABLESAMPLE SUBSTITUTE('sampleName') - * (non-standard) - *
  • <query> TABLESAMPLE BERNOULLI(<percent>) - * [REPEATABLE(<seed>)] (standard, but not implemented for FTRS - * yet) - *
  • <query> TABLESAMPLE SYSTEM(<percent>) - * [REPEATABLE(<seed>)] (standard, but not implemented for FTRS - * yet) - *
- * - *

Operand #0 is a query or table; Operand #1 is a {@link SqlSampleSpec} - * wrapped in a {@link SqlLiteral}. - */ - public static final SqlSpecialOperator TABLESAMPLE = - new SqlSpecialOperator( - "TABLESAMPLE", - SqlKind.TABLESAMPLE, - 20, - true, - ReturnTypes.ARG0, - null, - OperandTypes.VARIADIC) { - public void unparse( - SqlWriter writer, - SqlCall call, - int leftPrec, - int rightPrec) { - call.operand(0).unparse(writer, leftPrec, 0); - writer.keyword("TABLESAMPLE"); - call.operand(1).unparse(writer, 0, rightPrec); - } - }; - - /** The {@code TUMBLE} group function. */ - public static final SqlGroupFunction TUMBLE = - new SqlGroupFunction(SqlKind.TUMBLE, null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, - OperandTypes.DATETIME_INTERVAL_TIME)) { - @Override List getAuxiliaryFunctions() { - return ImmutableList.of(TUMBLE_START, TUMBLE_END); - } - }; - - /** The {@code TUMBLE_START} auxiliary function of - * the {@code TUMBLE} group function. */ - public static final SqlGroupFunction TUMBLE_START = - TUMBLE.auxiliary(SqlKind.TUMBLE_START); - - /** The {@code TUMBLE_END} auxiliary function of - * the {@code TUMBLE} group function. */ - public static final SqlGroupFunction TUMBLE_END = - TUMBLE.auxiliary(SqlKind.TUMBLE_END); - - /** The {@code HOP} group function. */ - public static final SqlGroupFunction HOP = - new SqlGroupFunction(SqlKind.HOP, null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL_INTERVAL, - OperandTypes.DATETIME_INTERVAL_INTERVAL_TIME)) { - @Override List getAuxiliaryFunctions() { - return ImmutableList.of(HOP_START, HOP_END); - } - }; - - /** The {@code HOP_START} auxiliary function of - * the {@code HOP} group function. */ - public static final SqlGroupFunction HOP_START = - HOP.auxiliary(SqlKind.HOP_START); - - /** The {@code HOP_END} auxiliary function of - * the {@code HOP} group function. */ - public static final SqlGroupFunction HOP_END = - HOP.auxiliary(SqlKind.HOP_END); - - /** The {@code SESSION} group function. */ - public static final SqlGroupFunction SESSION = - new SqlGroupFunction(SqlKind.SESSION, null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, - OperandTypes.DATETIME_INTERVAL_TIME)) { - @Override List getAuxiliaryFunctions() { - return ImmutableList.of(SESSION_START, SESSION_END); - } - }; - - /** The {@code SESSION_START} auxiliary function of - * the {@code SESSION} group function. */ - public static final SqlGroupFunction SESSION_START = - SESSION.auxiliary(SqlKind.SESSION_START); - - /** The {@code SESSION_END} auxiliary function of - * the {@code SESSION} group function. */ - public static final SqlGroupFunction SESSION_END = - SESSION.auxiliary(SqlKind.SESSION_END); - - /** {@code |} operator to create alternate patterns - * within {@code MATCH_RECOGNIZE}. - * - *

If {@code p1} and {@code p2} are patterns then {@code p1 | p2} is a - * pattern that matches {@code p1} or {@code p2}. */ - public static final SqlBinaryOperator PATTERN_ALTER = - new SqlBinaryOperator("|", SqlKind.PATTERN_ALTER, 70, true, null, null, null); - - /** Operator to concatenate patterns within {@code MATCH_RECOGNIZE}. - * - *

If {@code p1} and {@code p2} are patterns then {@code p1 p2} is a - * pattern that matches {@code p1} followed by {@code p2}. */ - public static final SqlBinaryOperator PATTERN_CONCAT = - new SqlBinaryOperator("", SqlKind.PATTERN_CONCAT, 80, true, null, null, null); - - /** Operator to quantify patterns within {@code MATCH_RECOGNIZE}. - * - *

If {@code p} is a pattern then {@code p{3, 5}} is a - * pattern that matches between 3 and 5 occurrences of {@code p}. */ - public static final SqlSpecialOperator PATTERN_QUANTIFIER = - new SqlSpecialOperator("PATTERN_QUANTIFIER", SqlKind.PATTERN_QUANTIFIER, - 90) { - @Override public void unparse(SqlWriter writer, SqlCall call, - int leftPrec, int rightPrec) { - call.operand(0).unparse(writer, this.getLeftPrec(), this.getRightPrec()); - int startNum = ((SqlNumericLiteral) call.operand(1)).intValue(true); - SqlNumericLiteral endRepNum = call.operand(2); - boolean isReluctant = ((SqlLiteral) call.operand(3)).booleanValue(); - int endNum = endRepNum.intValue(true); - if (startNum == endNum) { - writer.keyword("{ " + startNum + " }"); - } else { - if (endNum == -1) { - if (startNum == 0) { - writer.keyword("*"); - } else if (startNum == 1) { - writer.keyword("+"); - } else { - writer.keyword("{ " + startNum + ", }"); - } - } else { - if (startNum == 0 && endNum == 1) { - writer.keyword("?"); - } else if (startNum == -1) { - writer.keyword("{ , " + endNum + " }"); - } else { - writer.keyword("{ " + startNum + ", " + endNum + " }"); - } - } - if (isReluctant) { - writer.keyword("?"); - } - } - } - }; - - /** {@code PERMUTE} operator to combine patterns within - * {@code MATCH_RECOGNIZE}. - * - *

If {@code p1} and {@code p2} are patterns then {@code PERMUTE (p1, p2)} - * is a pattern that matches all permutations of {@code p1} and - * {@code p2}. */ - public static final SqlSpecialOperator PATTERN_PERMUTE = - new SqlSpecialOperator("PATTERN_PERMUTE", SqlKind.PATTERN_PERMUTE, 100) { - @Override public void unparse(SqlWriter writer, SqlCall call, - int leftPrec, int rightPrec) { - writer.keyword("PERMUTE"); - SqlWriter.Frame frame = writer.startList("(", ")"); - for (int i = 0; i < call.getOperandList().size(); i++) { - SqlNode pattern = call.getOperandList().get(i); - pattern.unparse(writer, 0, 0); - if (i != call.getOperandList().size() - 1) { - writer.print(","); - } - } - writer.endList(frame); - } - }; - - /** {@code EXCLUDE} operator within {@code MATCH_RECOGNIZE}. - * - *

If {@code p} is a pattern then {@code {- p -} }} is a - * pattern that excludes {@code p} from the output. */ - public static final SqlSpecialOperator PATTERN_EXCLUDE = - new SqlSpecialOperator("PATTERN_EXCLUDE", SqlKind.PATTERN_EXCLUDED, - 100) { - @Override public void unparse(SqlWriter writer, SqlCall call, - int leftPrec, int rightPrec) { - SqlWriter.Frame frame = writer.startList("{-", "-}"); - SqlNode node = call.getOperandList().get(0); - node.unparse(writer, 0, 0); - writer.endList(frame); - } - }; - - //~ Methods ---------------------------------------------------------------- - - /** - * Returns the standard operator table, creating it if necessary. - */ - public static synchronized SqlStdOperatorTable instance() { - if (instance == null) { - // Creates and initializes the standard operator table. - // Uses two-phase construction, because we can't initialize the - // table until the constructor of the sub-class has completed. - instance = new SqlStdOperatorTable(); - instance.init(); - } - return instance; - } - - /** Returns the group function for which a given kind is an auxiliary - * function, or null if it is not an auxiliary function. */ - public static SqlGroupFunction auxiliaryToGroup(SqlKind kind) { - switch (kind) { - case TUMBLE_START: - case TUMBLE_END: - return TUMBLE; - case HOP_START: - case HOP_END: - return HOP; - case SESSION_START: - case SESSION_END: - return SESSION; - default: - return null; - } - } - - /** Converts a call to a grouped auxiliary function - * to a call to the grouped window function. For other calls returns null. - * - *

For example, converts {@code TUMBLE_START(rowtime, INTERVAL '1' HOUR))} - * to {@code TUMBLE(rowtime, INTERVAL '1' HOUR))}. */ - public static SqlCall convertAuxiliaryToGroupCall(SqlCall call) { - final SqlOperator op = call.getOperator(); - if (op instanceof SqlGroupFunction - && op.isGroupAuxiliary()) { - return copy(call, ((SqlGroupFunction) op).groupFunction); - } - return null; - } - - /** Converts a call to a grouped window function to a call to its auxiliary - * window function(s). For other calls returns null. - * - *

For example, converts {@code TUMBLE_START(rowtime, INTERVAL '1' HOUR))} - * to {@code TUMBLE(rowtime, INTERVAL '1' HOUR))}. */ - public static List> - convertGroupToAuxiliaryCalls(SqlCall call) { - final SqlOperator op = call.getOperator(); - if (op instanceof SqlGroupFunction - && op.isGroup()) { - ImmutableList.Builder> builder = - ImmutableList.builder(); - for (final SqlGroupFunction f - : ((SqlGroupFunction) op).getAuxiliaryFunctions()) { - builder.add( - Pair.of(copy(call, f), - new AuxiliaryConverter.Impl(f))); - } - return builder.build(); - } - return ImmutableList.of(); - } - - /** Creates a copy of a call with a new operator. */ - private static SqlCall copy(SqlCall call, SqlOperator operator) { - final List list = call.getOperandList(); - return new SqlBasicCall(operator, list.toArray(new SqlNode[list.size()]), - call.getParserPosition()); - } - -} - -// End SqlStdOperatorTable.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/AggChecker.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/AggChecker.java deleted file mode 100644 index 56a364fd545b9..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/AggChecker.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.validate; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-1761 IS FIXED. - */ - -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.calcite.util.Litmus; - -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.List; - -import static org.apache.calcite.util.Static.RESOURCE; - -/** - * Visitor which throws an exception if any component of the expression is not a - * group expression. - */ -class AggChecker extends SqlBasicVisitor { - //~ Instance fields -------------------------------------------------------- - - private final Deque scopes = new ArrayDeque<>(); - private final List extraExprs; - private final List groupExprs; - private boolean distinct; - private SqlValidatorImpl validator; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates an AggChecker. - * - * @param validator Validator - * @param scope Scope - * @param groupExprs Expressions in GROUP BY (or SELECT DISTINCT) clause, - * that are therefore available - * @param distinct Whether aggregation checking is because of a SELECT - * DISTINCT clause - */ - AggChecker( - SqlValidatorImpl validator, - AggregatingScope scope, - List extraExprs, - List groupExprs, - boolean distinct) { - this.validator = validator; - this.extraExprs = extraExprs; - this.groupExprs = groupExprs; - this.distinct = distinct; - this.scopes.push(scope); - } - - //~ Methods ---------------------------------------------------------------- - - boolean isGroupExpr(SqlNode expr) { - for (SqlNode groupExpr : groupExprs) { - if (groupExpr.equalsDeep(expr, Litmus.IGNORE)) { - return true; - } - } - - for (SqlNode extraExpr : extraExprs) { - if (extraExpr.equalsDeep(expr, Litmus.IGNORE)) { - return true; - } - } - return false; - } - - public Void visit(SqlIdentifier id) { - if (isGroupExpr(id) || id.isStar()) { - // Star may validly occur in "SELECT COUNT(*) OVER w" - return null; - } - - // Is it a call to a parentheses-free function? - SqlCall call = - SqlUtil.makeCall( - validator.getOperatorTable(), - id); - if (call != null) { - return call.accept(this); - } - - // Didn't find the identifier in the group-by list as is, now find - // it fully-qualified. - // TODO: It would be better if we always compared fully-qualified - // to fully-qualified. - final SqlQualified fqId = scopes.peek().fullyQualify(id); - if (isGroupExpr(fqId.identifier)) { - return null; - } - SqlNode originalExpr = validator.getOriginal(id); - final String exprString = originalExpr.toString(); - throw validator.newValidationError(originalExpr, - distinct - ? RESOURCE.notSelectDistinctExpr(exprString) - : RESOURCE.notGroupExpr(exprString)); - } - - public Void visit(SqlCall call) { - final SqlValidatorScope scope = scopes.peek(); - if (call.getOperator().isAggregator()) { - if (distinct) { - if (scope instanceof AggregatingSelectScope) { - SqlNodeList selectList = - ((SqlSelect) scope.getNode()).getSelectList(); - - // Check if this aggregation function is just an element in the select - for (SqlNode sqlNode : selectList) { - if (sqlNode.getKind() == SqlKind.AS) { - sqlNode = ((SqlCall) sqlNode).operand(0); - } - - if (validator.expand(sqlNode, scope) - .equalsDeep(call, Litmus.IGNORE)) { - return null; - } - } - } - - // Cannot use agg fun in ORDER BY clause if have SELECT DISTINCT. - SqlNode originalExpr = validator.getOriginal(call); - final String exprString = originalExpr.toString(); - throw validator.newValidationError(call, - RESOURCE.notSelectDistinctExpr(exprString)); - } - - // For example, 'sum(sal)' in 'SELECT sum(sal) FROM emp GROUP - // BY deptno' - return null; - } - if (call.getKind() == SqlKind.FILTER) { - call.operand(0).accept(this); - return null; - } - // Visit the operand in window function - if (call.getKind() == SqlKind.OVER) { - for (SqlNode operand : call.operand(0).getOperandList()) { - operand.accept(this); - } - // Check the OVER clause - final SqlNode over = call.operand(1); - if (over instanceof SqlCall) { - over.accept(this); - } else if (over instanceof SqlIdentifier) { - // Check the corresponding SqlWindow in WINDOW clause - final SqlWindow window = - scope.lookupWindow(((SqlIdentifier) over).getSimple()); - window.getPartitionList().accept(this); - window.getOrderList().accept(this); - } - } - if (isGroupExpr(call)) { - // This call matches an expression in the GROUP BY clause. - return null; - } - - final SqlCall groupCall = - SqlStdOperatorTable.convertAuxiliaryToGroupCall(call); - if (groupCall != null) { - if (isGroupExpr(groupCall)) { - // This call is an auxiliary function that matches a group call in the - // GROUP BY clause. - // - // For example TUMBLE_START is an auxiliary of the TUMBLE - // group function, and - // TUMBLE_START(rowtime, INTERVAL '1' HOUR) - // matches - // TUMBLE(rowtime, INTERVAL '1' HOUR') - return null; - } - throw validator.newValidationError(groupCall, - RESOURCE.auxiliaryWithoutMatchingGroupCall( - call.getOperator().getName(), groupCall.getOperator().getName())); - } - - if (call.isA(SqlKind.QUERY)) { - // Allow queries for now, even though they may contain - // references to forbidden columns. - return null; - } - - // Switch to new scope. - SqlValidatorScope newScope = scope.getOperandScope(call); - scopes.push(newScope); - - // Visit the operands (only expressions). - call.getOperator() - .acceptCall(this, call, true, ArgHandlerImpl.instance()); - - // Restore scope. - scopes.pop(); - return null; - } - -} - -// End AggChecker.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java deleted file mode 100644 index 3733d615726fa..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.validate; - -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.schema.AggregateFunction; -import org.apache.calcite.schema.FunctionParameter; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.util.Util; - -import com.google.common.base.Function; -import com.google.common.collect.Lists; - -import java.util.List; - -/** - * User-defined aggregate function. - * - *

Created by the validator, after resolving a function call to a function - * defined in a Calcite schema.

- */ -public class SqlUserDefinedAggFunction extends SqlAggFunction { - public final AggregateFunction function; - - public SqlUserDefinedAggFunction(SqlIdentifier opName, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, AggregateFunction function) { - this(opName, returnTypeInference, operandTypeInference, operandTypeChecker, function,false); - } - - public SqlUserDefinedAggFunction(SqlIdentifier opName, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - AggregateFunction function, - Boolean requestsOver) { - super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION, - returnTypeInference, operandTypeInference, operandTypeChecker, - SqlFunctionCategory.USER_DEFINED_FUNCTION, false, requestsOver); - this.function = function; - } - - @SuppressWarnings("deprecation") - public List getParameterTypes( - final RelDataTypeFactory typeFactory) { - return Lists.transform(function.getParameters(), - new Function() { - public RelDataType apply(FunctionParameter input) { - return input.getType(typeFactory); - } - }); - } - - @SuppressWarnings("deprecation") - public RelDataType getReturnType(RelDataTypeFactory typeFactory) { - return function.getReturnType(typeFactory); - } -} - -// End SqlUserDefinedAggFunction.java diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java deleted file mode 100644 index b7f9ce78b4fb7..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ /dev/null @@ -1,5356 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql2rel; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-1761 IS FIXED. - */ - -import org.apache.calcite.avatica.util.Spaces; -import org.apache.calcite.linq4j.Ord; -import org.apache.calcite.plan.Convention; -import org.apache.calcite.plan.RelOptCluster; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelOptSamplingParameters; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.plan.RelTraitSet; -import org.apache.calcite.prepare.Prepare; -import org.apache.calcite.prepare.RelOptTableImpl; -import org.apache.calcite.rel.RelCollation; -import org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.calcite.rel.RelCollations; -import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.RelRoot; -import org.apache.calcite.rel.SingleRel; -import org.apache.calcite.rel.core.Aggregate; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.core.Collect; -import org.apache.calcite.rel.core.CorrelationId; -import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.JoinInfo; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.core.RelFactories; -import org.apache.calcite.rel.core.Sample; -import org.apache.calcite.rel.core.Sort; -import org.apache.calcite.rel.core.Uncollect; -import org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.calcite.rel.logical.LogicalCorrelate; -import org.apache.calcite.rel.logical.LogicalFilter; -import org.apache.calcite.rel.logical.LogicalIntersect; -import org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.calcite.rel.logical.LogicalMinus; -import org.apache.calcite.rel.logical.LogicalProject; -import org.apache.calcite.rel.logical.LogicalSort; -import org.apache.calcite.rel.logical.LogicalTableFunctionScan; -import org.apache.calcite.rel.logical.LogicalTableModify; -import org.apache.calcite.rel.logical.LogicalTableScan; -import org.apache.calcite.rel.logical.LogicalUnion; -import org.apache.calcite.rel.logical.LogicalValues; -import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; -import org.apache.calcite.rel.metadata.RelColumnMapping; -import org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.calcite.rel.stream.Delta; -import org.apache.calcite.rel.stream.LogicalDelta; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexCallBinding; -import org.apache.calcite.rex.RexCorrelVariable; -import org.apache.calcite.rex.RexDynamicParam; -import org.apache.calcite.rex.RexFieldAccess; -import org.apache.calcite.rex.RexFieldCollation; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexLiteral; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexRangeRef; -import org.apache.calcite.rex.RexShuttle; -import org.apache.calcite.rex.RexSubQuery; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.rex.RexWindowBound; -import org.apache.calcite.schema.ModifiableTable; -import org.apache.calcite.schema.ModifiableView; -import org.apache.calcite.schema.Table; -import org.apache.calcite.schema.TranslatableTable; -import org.apache.calcite.schema.Wrapper; -import org.apache.calcite.sql.JoinConditionType; -import org.apache.calcite.sql.JoinType; -import org.apache.calcite.sql.SemiJoinType; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlDelete; -import org.apache.calcite.sql.SqlDynamicParam; -import org.apache.calcite.sql.SqlExplainFormat; -import org.apache.calcite.sql.SqlExplainLevel; -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLiteral; -import org.apache.calcite.sql.SqlMatchRecognize; -import org.apache.calcite.sql.SqlMerge; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlNumericLiteral; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlOperatorTable; -import org.apache.calcite.sql.SqlOrderBy; -import org.apache.calcite.sql.SqlSampleSpec; -import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.SqlSelectKeyword; -import org.apache.calcite.sql.SqlSetOperator; -import org.apache.calcite.sql.SqlUnnestOperator; -import org.apache.calcite.sql.SqlUpdate; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlValuesOperator; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.SqlWith; -import org.apache.calcite.sql.SqlWithItem; -import org.apache.calcite.sql.fun.SqlCountAggFunction; -import org.apache.calcite.sql.fun.SqlInOperator; -import org.apache.calcite.sql.fun.SqlRowOperator; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.type.SqlTypeUtil; -import org.apache.calcite.sql.type.TableFunctionReturnTypeInference; -import org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.calcite.sql.util.SqlVisitor; -import org.apache.calcite.sql.validate.AggregatingSelectScope; -import org.apache.calcite.sql.validate.CollectNamespace; -import org.apache.calcite.sql.validate.DelegatingScope; -import org.apache.calcite.sql.validate.ListScope; -import org.apache.calcite.sql.validate.ParameterScope; -import org.apache.calcite.sql.validate.SelectScope; -import org.apache.calcite.sql.validate.SqlMonotonicity; -import org.apache.calcite.sql.validate.SqlNameMatcher; -import org.apache.calcite.sql.validate.SqlQualified; -import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction; -import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro; -import org.apache.calcite.sql.validate.SqlValidator; -import org.apache.calcite.sql.validate.SqlValidatorImpl; -import org.apache.calcite.sql.validate.SqlValidatorNamespace; -import org.apache.calcite.sql.validate.SqlValidatorScope; -import org.apache.calcite.sql.validate.SqlValidatorTable; -import org.apache.calcite.sql.validate.SqlValidatorUtil; -import org.apache.calcite.tools.RelBuilder; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; -import org.apache.calcite.util.Litmus; -import org.apache.calcite.util.NlsString; -import org.apache.calcite.util.NumberUtil; -import org.apache.calcite.util.Pair; -import org.apache.calcite.util.Util; -import org.apache.calcite.util.trace.CalciteTrace; - -import com.google.common.base.Function; -import org.apache.flink.util.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -import org.slf4j.Logger; - -import java.lang.reflect.Type; -import java.math.BigDecimal; -import java.util.AbstractList; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Deque; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - -import static org.apache.calcite.sql.SqlUtil.stripAs; -import static org.apache.calcite.util.Static.RESOURCE; - -/** - * Converts a SQL parse tree (consisting of - * {@link org.apache.calcite.sql.SqlNode} objects) into a relational algebra - * expression (consisting of {@link org.apache.calcite.rel.RelNode} objects). - * - *

The public entry points are: {@link #convertQuery}, - * {@link #convertExpression(SqlNode)}. - */ -public class SqlToRelConverter { - //~ Static fields/initializers --------------------------------------------- - - protected static final Logger SQL2REL_LOGGER = - CalciteTrace.getSqlToRelTracer(); - - private static final BigDecimal TWO = BigDecimal.valueOf(2L); - - /** Size of the smallest IN list that will be converted to a semijoin to a - * static table. */ - public static final int DEFAULT_IN_SUB_QUERY_THRESHOLD = 20; - - @Deprecated // to be removed before 2.0 - public static final int DEFAULT_IN_SUBQUERY_THRESHOLD = - DEFAULT_IN_SUB_QUERY_THRESHOLD; - - //~ Instance fields -------------------------------------------------------- - - protected final SqlValidator validator; - protected final RexBuilder rexBuilder; - protected final Prepare.CatalogReader catalogReader; - protected final RelOptCluster cluster; - private SubQueryConverter subQueryConverter; - protected final List leaves = new ArrayList<>(); - private final List dynamicParamSqlNodes = new ArrayList<>(); - private final SqlOperatorTable opTab; - protected final RelDataTypeFactory typeFactory; - private final SqlNodeToRexConverter exprConverter; - private int explainParamCount; - public final SqlToRelConverter.Config config; - - /** - * Fields used in name resolution for correlated sub-queries. - */ - private final Map mapCorrelToDeferred = - new HashMap<>(); - - /** - * Stack of names of datasets requested by the - * TABLE(SAMPLE(<datasetName>, <query>)) construct. - */ - private final Deque datasetStack = new ArrayDeque<>(); - - /** - * Mapping of non-correlated sub-queries that have been converted to their - * equivalent constants. Used to avoid re-evaluating the sub-query if it's - * already been evaluated. - */ - private final Map mapConvertedNonCorrSubqs = - new HashMap<>(); - - public final RelOptTable.ViewExpander viewExpander; - - //~ Constructors ----------------------------------------------------------- - /** - * Creates a converter. - * - * @param viewExpander Preparing statement - * @param validator Validator - * @param catalogReader Schema - * @param planner Planner - * @param rexBuilder Rex builder - * @param convertletTable Expression converter - */ - @Deprecated // to be removed before 2.0 - public SqlToRelConverter( - RelOptTable.ViewExpander viewExpander, - SqlValidator validator, - Prepare.CatalogReader catalogReader, - RelOptPlanner planner, - RexBuilder rexBuilder, - SqlRexConvertletTable convertletTable) { - this(viewExpander, validator, catalogReader, - RelOptCluster.create(planner, rexBuilder), convertletTable, - Config.DEFAULT); - } - - @Deprecated // to be removed before 2.0 - public SqlToRelConverter( - RelOptTable.ViewExpander viewExpander, - SqlValidator validator, - Prepare.CatalogReader catalogReader, - RelOptCluster cluster, - SqlRexConvertletTable convertletTable) { - this(viewExpander, validator, catalogReader, cluster, convertletTable, - Config.DEFAULT); - } - - /* Creates a converter. */ - public SqlToRelConverter( - RelOptTable.ViewExpander viewExpander, - SqlValidator validator, - Prepare.CatalogReader catalogReader, - RelOptCluster cluster, - SqlRexConvertletTable convertletTable, - Config config) { - this.viewExpander = viewExpander; - this.opTab = - (validator - == null) ? SqlStdOperatorTable.instance() - : validator.getOperatorTable(); - this.validator = validator; - this.catalogReader = catalogReader; - this.subQueryConverter = new NoOpSubQueryConverter(); - this.rexBuilder = cluster.getRexBuilder(); - this.typeFactory = rexBuilder.getTypeFactory(); - this.cluster = Preconditions.checkNotNull(cluster); - this.exprConverter = new SqlNodeToRexConverterImpl(convertletTable); - this.explainParamCount = 0; - this.config = new ConfigBuilder().withConfig(config).build(); - } - - //~ Methods ---------------------------------------------------------------- - - /** - * @return the RelOptCluster in use. - */ - public RelOptCluster getCluster() { - return cluster; - } - - /** - * Returns the row-expression builder. - */ - public RexBuilder getRexBuilder() { - return rexBuilder; - } - - /** - * Returns the number of dynamic parameters encountered during translation; - * this must only be called after {@link #convertQuery}. - * - * @return number of dynamic parameters - */ - public int getDynamicParamCount() { - return dynamicParamSqlNodes.size(); - } - - /** - * Returns the type inferred for a dynamic parameter. - * - * @param index 0-based index of dynamic parameter - * @return inferred type, never null - */ - public RelDataType getDynamicParamType(int index) { - SqlNode sqlNode = dynamicParamSqlNodes.get(index); - if (sqlNode == null) { - throw Util.needToImplement("dynamic param type inference"); - } - return validator.getValidatedNodeType(sqlNode); - } - - /** - * Returns the current count of the number of dynamic parameters in an - * EXPLAIN PLAN statement. - * - * @param increment if true, increment the count - * @return the current count before the optional increment - */ - public int getDynamicParamCountInExplain(boolean increment) { - int retVal = explainParamCount; - if (increment) { - ++explainParamCount; - } - return retVal; - } - - /** - * @return mapping of non-correlated sub-queries that have been converted to - * the constants that they evaluate to - */ - public Map getMapConvertedNonCorrSubqs() { - return mapConvertedNonCorrSubqs; - } - - /** - * Adds to the current map of non-correlated converted sub-queries the - * elements from another map that contains non-correlated sub-queries that - * have been converted by another SqlToRelConverter. - * - * @param alreadyConvertedNonCorrSubqs the other map - */ - public void addConvertedNonCorrSubqs( - Map alreadyConvertedNonCorrSubqs) { - mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs); - } - - /** - * Sets a new SubQueryConverter. To have any effect, this must be called - * before any convert method. - * - * @param converter new SubQueryConverter - */ - public void setSubQueryConverter(SubQueryConverter converter) { - subQueryConverter = converter; - } - - /** - * Sets the number of dynamic parameters in the current EXPLAIN PLAN - * statement. - * - * @param explainParamCount number of dynamic parameters in the statement - */ - public void setDynamicParamCountInExplain(int explainParamCount) { - assert config.isExplain(); - this.explainParamCount = explainParamCount; - } - - private void checkConvertedType(SqlNode query, RelNode result) { - if (query.isA(SqlKind.DML)) { - return; - } - // Verify that conversion from SQL to relational algebra did - // not perturb any type information. (We can't do this if the - // SQL statement is something like an INSERT which has no - // validator type information associated with its result, - // hence the namespace check above.) - final List validatedFields = - validator.getValidatedNodeType(query).getFieldList(); - final RelDataType validatedRowType = - validator.getTypeFactory().createStructType( - Pair.right(validatedFields), - SqlValidatorUtil.uniquify(Pair.left(validatedFields), - catalogReader.nameMatcher().isCaseSensitive())); - - final List convertedFields = - result.getRowType().getFieldList().subList(0, validatedFields.size()); - final RelDataType convertedRowType = - validator.getTypeFactory().createStructType(convertedFields); - - if (!RelOptUtil.equal("validated row type", validatedRowType, - "converted row type", convertedRowType, Litmus.IGNORE)) { - throw new AssertionError("Conversion to relational algebra failed to " - + "preserve datatypes:\n" - + "validated type:\n" - + validatedRowType.getFullTypeString() - + "\nconverted type:\n" - + convertedRowType.getFullTypeString() - + "\nrel:\n" - + RelOptUtil.toString(result)); - } - } - - public RelNode flattenTypes( - RelNode rootRel, - boolean restructure) { - RelStructuredTypeFlattener typeFlattener = - new RelStructuredTypeFlattener(rexBuilder, createToRelContext(), restructure); - return typeFlattener.rewrite(rootRel); - } - - /** - * If sub-query is correlated and decorrelation is enabled, performs - * decorrelation. - * - * @param query Query - * @param rootRel Root relational expression - * @return New root relational expression after decorrelation - */ - public RelNode decorrelate(SqlNode query, RelNode rootRel) { - if (!enableDecorrelation()) { - return rootRel; - } - final RelNode result = decorrelateQuery(rootRel); - if (result != rootRel) { - checkConvertedType(query, result); - } - return result; - } - - /** - * Walks over a tree of relational expressions, replacing each - * {@link RelNode} with a 'slimmed down' relational expression that projects - * only the fields required by its consumer. - * - *

This may make things easier for the optimizer, by removing crud that - * would expand the search space, but is difficult for the optimizer itself - * to do it, because optimizer rules must preserve the number and type of - * fields. Hence, this transform that operates on the entire tree, similar - * to the {@link RelStructuredTypeFlattener type-flattening transform}. - * - *

Currently this functionality is disabled in farrago/luciddb; the - * default implementation of this method does nothing. - * - * @param ordered Whether the relational expression must produce results in - * a particular order (typically because it has an ORDER BY at top level) - * @param rootRel Relational expression that is at the root of the tree - * @return Trimmed relational expression - */ - public RelNode trimUnusedFields(boolean ordered, RelNode rootRel) { - // Trim fields that are not used by their consumer. - if (isTrimUnusedFields()) { - final RelFieldTrimmer trimmer = newFieldTrimmer(); - final List collations = - rootRel.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE); - rootRel = trimmer.trim(rootRel); - if (!ordered - && collations != null - && !collations.isEmpty() - && !collations.equals(ImmutableList.of(RelCollations.EMPTY))) { - final RelTraitSet traitSet = rootRel.getTraitSet() - .replace(RelCollationTraitDef.INSTANCE, collations); - rootRel = rootRel.copy(traitSet, rootRel.getInputs()); - } - if (SQL2REL_LOGGER.isDebugEnabled()) { - SQL2REL_LOGGER.debug( - RelOptUtil.dumpPlan("Plan after trimming unused fields", rootRel, - SqlExplainFormat.TEXT, SqlExplainLevel.EXPPLAN_ATTRIBUTES)); - } - } - return rootRel; - } - - /** - * Creates a RelFieldTrimmer. - * - * @return Field trimmer - */ - protected RelFieldTrimmer newFieldTrimmer() { - final RelBuilder relBuilder = - RelFactories.LOGICAL_BUILDER.create(cluster, null); - return new RelFieldTrimmer(validator, relBuilder); - } - - /** - * Converts an unvalidated query's parse tree into a relational expression. - * - * @param query Query to convert - * @param needsValidation Whether to validate the query before converting; - * false if the query has already been - * validated. - * @param top Whether the query is top-level, say if its result - * will become a JDBC result set; false if - * the query will be part of a view. - */ - public RelRoot convertQuery( - SqlNode query, - final boolean needsValidation, - final boolean top) { - if (needsValidation) { - query = validator.validate(query); - } - - RelMetadataQuery.THREAD_PROVIDERS.set( - JaninoRelMetadataProvider.of(cluster.getMetadataProvider())); - RelNode result = convertQueryRecursive(query, top, null).rel; - if (top) { - if (isStream(query)) { - result = new LogicalDelta(cluster, result.getTraitSet(), result); - } - } - RelCollation collation = RelCollations.EMPTY; - if (!query.isA(SqlKind.DML)) { - if (isOrdered(query)) { - collation = requiredCollation(result); - } - } - checkConvertedType(query, result); - - if (SQL2REL_LOGGER.isDebugEnabled()) { - SQL2REL_LOGGER.debug( - RelOptUtil.dumpPlan("Plan after converting SqlNode to RelNode", - result, SqlExplainFormat.TEXT, - SqlExplainLevel.EXPPLAN_ATTRIBUTES)); - } - - final RelDataType validatedRowType = validator.getValidatedNodeType(query); - return RelRoot.of(result, validatedRowType, query.getKind()) - .withCollation(collation); - } - - private static boolean isStream(SqlNode query) { - return query instanceof SqlSelect - && ((SqlSelect) query).isKeywordPresent(SqlSelectKeyword.STREAM); - } - - public static boolean isOrdered(SqlNode query) { - switch (query.getKind()) { - case SELECT: - return ((SqlSelect) query).getOrderList() != null - && ((SqlSelect) query).getOrderList().size() > 0; - case WITH: - return isOrdered(((SqlWith) query).body); - case ORDER_BY: - return ((SqlOrderBy) query).orderList.size() > 0; - default: - return false; - } - } - - private RelCollation requiredCollation(RelNode r) { - if (r instanceof Sort) { - return ((Sort) r).collation; - } - if (r instanceof Project) { - return requiredCollation(((Project) r).getInput()); - } - if (r instanceof Delta) { - return requiredCollation(((Delta) r).getInput()); - } - throw new AssertionError(); - } - - /** - * Converts a SELECT statement's parse tree into a relational expression. - */ - public RelNode convertSelect(SqlSelect select, boolean top) { - final SqlValidatorScope selectScope = validator.getWhereScope(select); - final Blackboard bb = createBlackboard(selectScope, null, top); - convertSelectImpl(bb, select); - return bb.root; - } - - /** - * Factory method for creating translation workspace. - */ - protected Blackboard createBlackboard(SqlValidatorScope scope, - Map nameToNodeMap, boolean top) { - return new Blackboard(scope, nameToNodeMap, top); - } - - /** - * Implementation of {@link #convertSelect(SqlSelect, boolean)}; - * derived class may override. - */ - protected void convertSelectImpl( - final Blackboard bb, - SqlSelect select) { - convertFrom( - bb, - select.getFrom()); - convertWhere( - bb, - select.getWhere()); - - final List orderExprList = new ArrayList<>(); - final List collationList = new ArrayList<>(); - gatherOrderExprs( - bb, - select, - select.getOrderList(), - orderExprList, - collationList); - final RelCollation collation = - cluster.traitSet().canonize(RelCollations.of(collationList)); - - if (validator.isAggregate(select)) { - convertAgg( - bb, - select, - orderExprList); - } else { - convertSelectList( - bb, - select, - orderExprList); - } - - if (select.isDistinct()) { - distinctify(bb, true); - } - convertOrder( - select, bb, collation, orderExprList, select.getOffset(), - select.getFetch()); - bb.setRoot(bb.root, true); - } - - /** - * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds - * a relational expression to make the results unique. - * - *

If the SELECT clause contains duplicate expressions, adds - * {@link org.apache.calcite.rel.logical.LogicalProject}s so that we are - * grouping on the minimal set of keys. The performance gain isn't huge, but - * it is difficult to detect these duplicate expressions later. - * - * @param bb Blackboard - * @param checkForDupExprs Check for duplicate expressions - */ - private void distinctify( - Blackboard bb, - boolean checkForDupExprs) { - // Look for duplicate expressions in the project. - // Say we have 'select x, y, x, z'. - // Then dups will be {[2, 0]} - // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]} - RelNode rel = bb.root; - if (checkForDupExprs && (rel instanceof LogicalProject)) { - LogicalProject project = (LogicalProject) rel; - final List projectExprs = project.getProjects(); - final List origins = new ArrayList<>(); - int dupCount = 0; - for (int i = 0; i < projectExprs.size(); i++) { - int x = findExpr(projectExprs.get(i), projectExprs, i); - if (x >= 0) { - origins.add(x); - ++dupCount; - } else { - origins.add(i); - } - } - if (dupCount == 0) { - distinctify(bb, false); - return; - } - - final Map squished = Maps.newHashMap(); - final List fields = rel.getRowType().getFieldList(); - final List> newProjects = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i++) { - if (origins.get(i) == i) { - squished.put(i, newProjects.size()); - newProjects.add(RexInputRef.of2(i, fields)); - } - } - rel = - LogicalProject.create(rel, Pair.left(newProjects), - Pair.right(newProjects)); - bb.root = rel; - distinctify(bb, false); - rel = bb.root; - - // Create the expressions to reverse the mapping. - // Project($0, $1, $0, $2). - final List> undoProjects = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i++) { - final int origin = origins.get(i); - RelDataTypeField field = fields.get(i); - undoProjects.add( - Pair.of( - (RexNode) new RexInputRef( - squished.get(origin), field.getType()), - field.getName())); - } - - rel = - LogicalProject.create(rel, Pair.left(undoProjects), - Pair.right(undoProjects)); - bb.setRoot( - rel, - false); - - return; - } - - // Usual case: all of the expressions in the SELECT clause are - // different. - final ImmutableBitSet groupSet = - ImmutableBitSet.range(rel.getRowType().getFieldCount()); - rel = - createAggregate(bb, false, groupSet, ImmutableList.of(groupSet), - ImmutableList.of()); - - bb.setRoot( - rel, - false); - } - - private int findExpr(RexNode seek, List exprs, int count) { - for (int i = 0; i < count; i++) { - RexNode expr = exprs.get(i); - if (expr.toString().equals(seek.toString())) { - return i; - } - } - return -1; - } - - /** - * Converts a query's ORDER BY clause, if any. - * - * @param select Query - * @param bb Blackboard - * @param collation Collation list - * @param orderExprList Method populates this list with orderBy expressions - * not present in selectList - * @param offset Expression for number of rows to discard before - * returning first row - * @param fetch Expression for number of rows to fetch - */ - protected void convertOrder( - SqlSelect select, - Blackboard bb, - RelCollation collation, - List orderExprList, - SqlNode offset, - SqlNode fetch) { - if (select.getOrderList() == null - || select.getOrderList().getList().isEmpty()) { - assert collation.getFieldCollations().isEmpty(); - if ((offset == null - || ((SqlLiteral) offset).bigDecimalValue().equals(BigDecimal.ZERO)) - && fetch == null) { - return; - } - } - - // Create a sorter using the previously constructed collations. - bb.setRoot( - LogicalSort.create(bb.root, collation, - offset == null ? null : convertExpression(offset), - fetch == null ? null : convertExpression(fetch)), - false); - - // If extra expressions were added to the project list for sorting, - // add another project to remove them. But make the collation empty, because - // we can't represent the real collation. - // - // If it is the top node, use the real collation, but don't trim fields. - if (orderExprList.size() > 0 && !bb.top) { - final List exprs = new ArrayList<>(); - final RelDataType rowType = bb.root.getRowType(); - final int fieldCount = - rowType.getFieldCount() - orderExprList.size(); - for (int i = 0; i < fieldCount; i++) { - exprs.add(rexBuilder.makeInputRef(bb.root, i)); - } - bb.setRoot( - LogicalProject.create(bb.root, exprs, - rowType.getFieldNames().subList(0, fieldCount)), - false); - } - } - - /** - * Returns whether a given node contains a {@link SqlInOperator}. - * - * @param node a RexNode tree - */ - private static boolean containsInOperator( - SqlNode node) { - try { - SqlVisitor visitor = - new SqlBasicVisitor() { - public Void visit(SqlCall call) { - if (call.getOperator() instanceof SqlInOperator) { - throw new Util.FoundOne(call); - } - return super.visit(call); - } - }; - node.accept(visitor); - return false; - } catch (Util.FoundOne e) { - Util.swallow(e, null); - return true; - } - } - - /** - * Push down all the NOT logical operators into any IN/NOT IN operators. - * - * @param scope Scope where {@code sqlNode} occurs - * @param sqlNode the root node from which to look for NOT operators - * @return the transformed SqlNode representation with NOT pushed down. - */ - private static SqlNode pushDownNotForIn(SqlValidatorScope scope, - SqlNode sqlNode) { - if ((sqlNode instanceof SqlCall) && containsInOperator(sqlNode)) { - SqlCall sqlCall = (SqlCall) sqlNode; - if ((sqlCall.getOperator() == SqlStdOperatorTable.AND) - || (sqlCall.getOperator() == SqlStdOperatorTable.OR)) { - SqlNode[] sqlOperands = ((SqlBasicCall) sqlCall).operands; - for (int i = 0; i < sqlOperands.length; i++) { - sqlOperands[i] = pushDownNotForIn(scope, sqlOperands[i]); - } - return reg(scope, sqlNode); - } else if (sqlCall.getOperator() == SqlStdOperatorTable.NOT) { - SqlNode childNode = sqlCall.operand(0); - assert childNode instanceof SqlCall; - SqlBasicCall childSqlCall = (SqlBasicCall) childNode; - if (childSqlCall.getOperator() == SqlStdOperatorTable.AND) { - SqlNode[] andOperands = childSqlCall.getOperands(); - SqlNode[] orOperands = new SqlNode[andOperands.length]; - for (int i = 0; i < orOperands.length; i++) { - orOperands[i] = reg(scope, - SqlStdOperatorTable.NOT.createCall(SqlParserPos.ZERO, - andOperands[i])); - } - for (int i = 0; i < orOperands.length; i++) { - orOperands[i] = pushDownNotForIn(scope, orOperands[i]); - } - return reg(scope, - SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO, - orOperands[0], orOperands[1])); - } else if (childSqlCall.getOperator() == SqlStdOperatorTable.OR) { - SqlNode[] orOperands = childSqlCall.getOperands(); - SqlNode[] andOperands = new SqlNode[orOperands.length]; - for (int i = 0; i < andOperands.length; i++) { - andOperands[i] = reg(scope, - SqlStdOperatorTable.NOT.createCall(SqlParserPos.ZERO, - orOperands[i])); - } - for (int i = 0; i < andOperands.length; i++) { - andOperands[i] = pushDownNotForIn(scope, andOperands[i]); - } - return reg(scope, - SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO, - andOperands[0], andOperands[1])); - } else if (childSqlCall.getOperator() == SqlStdOperatorTable.NOT) { - SqlNode[] notOperands = childSqlCall.getOperands(); - assert notOperands.length == 1; - return pushDownNotForIn(scope, notOperands[0]); - } else if (childSqlCall.getOperator() instanceof SqlInOperator) { - SqlNode[] inOperands = childSqlCall.getOperands(); - SqlInOperator inOp = - (SqlInOperator) childSqlCall.getOperator(); - if (inOp.isNotIn()) { - return reg(scope, - SqlStdOperatorTable.IN.createCall(SqlParserPos.ZERO, - inOperands[0], inOperands[1])); - } else { - return reg(scope, - SqlStdOperatorTable.NOT_IN.createCall(SqlParserPos.ZERO, - inOperands[0], inOperands[1])); - } - } else { - // childSqlCall is "leaf" node in a logical expression tree - // (only considering AND, OR, NOT) - return sqlNode; - } - } else { - // sqlNode is "leaf" node in a logical expression tree - // (only considering AND, OR, NOT) - return sqlNode; - } - } else { - // tree rooted at sqlNode does not contain inOperator - return sqlNode; - } - } - - /** Registers with the validator a {@link SqlNode} that has been created - * during the Sql-to-Rel process. */ - private static SqlNode reg(SqlValidatorScope scope, SqlNode e) { - scope.getValidator().deriveType(scope, e); - return e; - } - - /** - * Converts a WHERE clause. - * - * @param bb Blackboard - * @param where WHERE clause, may be null - */ - private void convertWhere( - final Blackboard bb, - final SqlNode where) { - if (where == null) { - return; - } - SqlNode newWhere = pushDownNotForIn(bb.scope, where); - replaceSubQueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE); - final RexNode convertedWhere = bb.convertExpression(newWhere); - - // only allocate filter if the condition is not TRUE - if (convertedWhere.isAlwaysTrue()) { - return; - } - - final RelFactories.FilterFactory factory = - RelFactories.DEFAULT_FILTER_FACTORY; - final RelNode filter = factory.createFilter(bb.root, convertedWhere); - final RelNode r; - final CorrelationUse p = getCorrelationUse(bb, filter); - if (p != null) { - assert p.r instanceof Filter; - Filter f = (Filter) p.r; - r = LogicalFilter.create(f.getInput(), f.getCondition(), - ImmutableSet.of(p.id)); - } else { - r = filter; - } - - bb.setRoot(r, false); - } - - private void replaceSubQueries( - final Blackboard bb, - final SqlNode expr, - RelOptUtil.Logic logic) { - findSubQueries(bb, expr, logic, false); - for (SubQuery node : bb.subQueryList) { - substituteSubQuery(bb, node); - } - } - - private void substituteSubQuery(Blackboard bb, SubQuery subQuery) { - final RexNode expr = subQuery.expr; - if (expr != null) { - // Already done. - return; - } - - final SqlBasicCall call; - final RelNode rel; - final SqlNode query; - final RelOptUtil.Exists converted; - switch (subQuery.node.getKind()) { - case CURSOR: - convertCursor(bb, subQuery); - return; - - case MULTISET_QUERY_CONSTRUCTOR: - case MULTISET_VALUE_CONSTRUCTOR: - case ARRAY_QUERY_CONSTRUCTOR: - rel = convertMultisets(ImmutableList.of(subQuery.node), bb); - subQuery.expr = bb.register(rel, JoinRelType.INNER); - return; - - case IN: - call = (SqlBasicCall) subQuery.node; - query = call.operand(1); - if (!config.isExpand() && !(query instanceof SqlNodeList)) { - return; - } - final SqlNode leftKeyNode = call.operand(0); - - final List leftKeys; - switch (leftKeyNode.getKind()) { - case ROW: - leftKeys = Lists.newArrayList(); - for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) { - leftKeys.add(bb.convertExpression(sqlExpr)); - } - break; - default: - leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode)); - } - - final boolean notIn = ((SqlInOperator) call.getOperator()).isNotIn(); - if (query instanceof SqlNodeList) { - SqlNodeList valueList = (SqlNodeList) query; - if (!containsNullLiteral(valueList) - && valueList.size() < config.getInSubQueryThreshold()) { - // We're under the threshold, so convert to OR. - subQuery.expr = - convertInToOr( - bb, - leftKeys, - valueList, - notIn); - return; - } - - // Otherwise, let convertExists translate - // values list into an inline table for the - // reference to Q below. - } - - // Project out the search columns from the left side - - // Q1: - // "select from emp where emp.deptno in (select col1 from T)" - // - // is converted to - // - // "select from - // emp inner join (select distinct col1 from T)) q - // on emp.deptno = q.col1 - // - // Q2: - // "select from emp where emp.deptno not in (Q)" - // - // is converted to - // - // "select from - // emp left outer join (select distinct col1, TRUE from T) q - // on emp.deptno = q.col1 - // where emp.deptno <> null - // and q.indicator <> TRUE" - // - final RelDataType targetRowType = - SqlTypeUtil.promoteToRowType(typeFactory, - validator.getValidatedNodeType(leftKeyNode), null); - converted = - convertExists(query, RelOptUtil.SubQueryType.IN, subQuery.logic, - notIn, targetRowType); - if (converted.indicator) { - // Generate - // emp CROSS JOIN (SELECT COUNT(*) AS c, - // COUNT(deptno) AS ck FROM dept) - final RelDataType longType = - typeFactory.createSqlType(SqlTypeName.BIGINT); - final RelNode seek = converted.r.getInput(0); // fragile - final int keyCount = leftKeys.size(); - final List args = ImmutableIntList.range(0, keyCount); - LogicalAggregate aggregate = - LogicalAggregate.create(seek, false, ImmutableBitSet.of(), null, - ImmutableList.of( - AggregateCall.create(SqlStdOperatorTable.COUNT, false, - ImmutableList.of(), -1, longType, null), - AggregateCall.create(SqlStdOperatorTable.COUNT, false, - args, -1, longType, null))); - LogicalJoin join = - LogicalJoin.create(bb.root, aggregate, rexBuilder.makeLiteral(true), - ImmutableSet.of(), JoinRelType.INNER); - bb.setRoot(join, false); - } - final RexNode rex = - bb.register(converted.r, - converted.outerJoin ? JoinRelType.LEFT : JoinRelType.INNER, - leftKeys); - - RelOptUtil.Logic logic = subQuery.logic; - switch (logic) { - case TRUE_FALSE_UNKNOWN: - case UNKNOWN_AS_TRUE: - if (!converted.indicator) { - logic = RelOptUtil.Logic.TRUE_FALSE; - } - } - subQuery.expr = translateIn(logic, bb.root, rex); - if (notIn) { - subQuery.expr = - rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr); - } - return; - - case EXISTS: - // "select from emp where exists (select a from T)" - // - // is converted to the following if the sub-query is correlated: - // - // "select from emp left outer join (select AGG_TRUE() as indicator - // from T group by corr_var) q where q.indicator is true" - // - // If there is no correlation, the expression is replaced with a - // boolean indicating whether the sub-query returned 0 or >= 1 row. - call = (SqlBasicCall) subQuery.node; - query = call.operand(0); - if (!config.isExpand()) { - return; - } - converted = convertExists(query, RelOptUtil.SubQueryType.EXISTS, - subQuery.logic, true, null); - assert !converted.indicator; - if (convertNonCorrelatedSubQuery(subQuery, bb, converted.r, true)) { - return; - } - subQuery.expr = bb.register(converted.r, JoinRelType.LEFT); - return; - - case SCALAR_QUERY: - // Convert the sub-query. If it's non-correlated, convert it - // to a constant expression. - if (!config.isExpand()) { - return; - } - call = (SqlBasicCall) subQuery.node; - query = call.operand(0); - converted = convertExists(query, RelOptUtil.SubQueryType.SCALAR, - subQuery.logic, true, null); - assert !converted.indicator; - if (convertNonCorrelatedSubQuery(subQuery, bb, converted.r, false)) { - return; - } - rel = convertToSingleValueSubq(query, converted.r); - subQuery.expr = bb.register(rel, JoinRelType.LEFT); - return; - - case SELECT: - // This is used when converting multiset queries: - // - // select * from unnest(select multiset[deptno] from emps); - // - converted = convertExists(subQuery.node, RelOptUtil.SubQueryType.SCALAR, - subQuery.logic, true, null); - assert !converted.indicator; - subQuery.expr = bb.register(converted.r, JoinRelType.LEFT); - return; - - default: - throw new AssertionError("unexpected kind of sub-query: " - + subQuery.node); - } - } - - private RexNode translateIn(RelOptUtil.Logic logic, RelNode root, - final RexNode rex) { - switch (logic) { - case TRUE: - return rexBuilder.makeLiteral(true); - - case TRUE_FALSE: - case UNKNOWN_AS_FALSE: - assert rex instanceof RexRangeRef; - final int fieldCount = rex.getType().getFieldCount(); - RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1); - rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode); - - // Then append the IS NOT NULL(leftKeysForIn). - // - // RexRangeRef contains the following fields: - // leftKeysForIn, - // rightKeysForIn (the original sub-query select list), - // nullIndicator - // - // The first two lists contain the same number of fields. - final int k = (fieldCount - 1) / 2; - for (int i = 0; i < k; i++) { - rexNode = - rexBuilder.makeCall( - SqlStdOperatorTable.AND, - rexNode, - rexBuilder.makeCall( - SqlStdOperatorTable.IS_NOT_NULL, - rexBuilder.makeFieldAccess(rex, i))); - } - return rexNode; - - case TRUE_FALSE_UNKNOWN: - case UNKNOWN_AS_TRUE: - // select e.deptno, - // case - // when ct.c = 0 then false - // when dt.i is not null then true - // when e.deptno is null then null - // when ct.ck < ct.c then null - // else false - // end - // from e - // cross join (select count(*) as c, count(deptno) as ck from v) as ct - // left join (select distinct deptno, true as i from v) as dt - // on e.deptno = dt.deptno - final Join join = (Join) root; - final Project left = (Project) join.getLeft(); - final RelNode leftLeft = ((Join) left.getInput()).getLeft(); - final int leftLeftCount = leftLeft.getRowType().getFieldCount(); - final RelDataType longType = - typeFactory.createSqlType(SqlTypeName.BIGINT); - final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount); - final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1); - final RexNode iRef = - rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1); - - final RexLiteral zero = - rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType); - final RexLiteral trueLiteral = rexBuilder.makeLiteral(true); - final RexLiteral falseLiteral = rexBuilder.makeLiteral(false); - final RexNode unknownLiteral = - rexBuilder.makeNullLiteral(trueLiteral.getType()); - - final ImmutableList.Builder args = ImmutableList.builder(); - args.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero), - falseLiteral, - rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef), - trueLiteral); - final JoinInfo joinInfo = join.analyzeCondition(); - for (int leftKey : joinInfo.leftKeys) { - final RexNode kRef = rexBuilder.makeInputRef(root, leftKey); - args.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef), - unknownLiteral); - } - args.add(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef), - unknownLiteral, - falseLiteral); - - return rexBuilder.makeCall(SqlStdOperatorTable.CASE, args.build()); - - default: - throw new AssertionError(logic); - } - } - - private static boolean containsNullLiteral(SqlNodeList valueList) { - for (SqlNode node : valueList.getList()) { - if (node instanceof SqlLiteral) { - SqlLiteral lit = (SqlLiteral) node; - if (lit.getValue() == null) { - return true; - } - } - } - return false; - } - - /** - * Determines if a sub-query is non-correlated and if so, converts it to a - * constant. - * - * @param subQuery the call that references the sub-query - * @param bb blackboard used to convert the sub-query - * @param converted RelNode tree corresponding to the sub-query - * @param isExists true if the sub-query is part of an EXISTS expression - * @return Whether the sub-query can be converted to a constant - */ - private boolean convertNonCorrelatedSubQuery( - SubQuery subQuery, - Blackboard bb, - RelNode converted, - boolean isExists) { - SqlCall call = (SqlBasicCall) subQuery.node; - if (subQueryConverter.canConvertSubQuery() - && isSubQueryNonCorrelated(converted, bb)) { - // First check if the sub-query has already been converted - // because it's a nested sub-query. If so, don't re-evaluate - // it again. - RexNode constExpr = mapConvertedNonCorrSubqs.get(call); - if (constExpr == null) { - constExpr = - subQueryConverter.convertSubQuery( - call, - this, - isExists, - config.isExplain()); - } - if (constExpr != null) { - subQuery.expr = constExpr; - mapConvertedNonCorrSubqs.put(call, constExpr); - return true; - } - } - return false; - } - - /** - * Converts the RelNode tree for a select statement to a select that - * produces a single value. - * - * @param query the query - * @param plan the original RelNode tree corresponding to the statement - * @return the converted RelNode tree - */ - public RelNode convertToSingleValueSubq( - SqlNode query, - RelNode plan) { - // Check whether query is guaranteed to produce a single value. - if (query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) query; - SqlNodeList selectList = select.getSelectList(); - SqlNodeList groupList = select.getGroup(); - - if ((selectList.size() == 1) - && ((groupList == null) || (groupList.size() == 0))) { - SqlNode selectExpr = selectList.get(0); - if (selectExpr instanceof SqlCall) { - SqlCall selectExprCall = (SqlCall) selectExpr; - if (Util.isSingleValue(selectExprCall)) { - return plan; - } - } - - // If there is a limit with 0 or 1, - // it is ensured to produce a single value - if (select.getFetch() != null - && select.getFetch() instanceof SqlNumericLiteral) { - SqlNumericLiteral limitNum = (SqlNumericLiteral) select.getFetch(); - if (((BigDecimal) limitNum.getValue()).intValue() < 2) { - return plan; - } - } - } - } else if (query instanceof SqlCall) { - // If the query is (values ...), - // it is necessary to look into the operands to determine - // whether SingleValueAgg is necessary - SqlCall exprCall = (SqlCall) query; - if (exprCall.getOperator() - instanceof SqlValuesOperator - && Util.isSingleValue(exprCall)) { - return plan; - } - } - - // If not, project SingleValueAgg - return RelOptUtil.createSingleValueAggRel( - cluster, - plan); - } - - /** - * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...". - * - * @param leftKeys LHS - * @param valuesList RHS - * @param isNotIn is this a NOT IN operator - * @return converted expression - */ - private RexNode convertInToOr( - final Blackboard bb, - final List leftKeys, - SqlNodeList valuesList, - boolean isNotIn) { - final List comparisons = new ArrayList<>(); - for (SqlNode rightVals : valuesList) { - RexNode rexComparison; - if (leftKeys.size() == 1) { - rexComparison = - rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, - leftKeys.get(0), - ensureSqlType(leftKeys.get(0).getType(), - bb.convertExpression(rightVals))); - } else { - assert rightVals instanceof SqlCall; - final SqlBasicCall call = (SqlBasicCall) rightVals; - assert (call.getOperator() instanceof SqlRowOperator) - && call.operandCount() == leftKeys.size(); - rexComparison = - RexUtil.composeConjunction( - rexBuilder, - Iterables.transform( - Pair.zip(leftKeys, call.getOperandList()), - new Function, RexNode>() { - public RexNode apply(Pair pair) { - return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, - pair.left, - ensureSqlType(pair.left.getType(), - bb.convertExpression(pair.right))); - } - }), - false); - } - comparisons.add(rexComparison); - } - - RexNode result = - RexUtil.composeDisjunction(rexBuilder, comparisons, true); - assert result != null; - - if (isNotIn) { - result = - rexBuilder.makeCall( - SqlStdOperatorTable.NOT, - result); - } - - return result; - } - - /** Ensures that an expression has a given {@link SqlTypeName}, applying a - * cast if necessary. If the expression already has the right type family, - * returns the expression unchanged. */ - private RexNode ensureSqlType(RelDataType type, RexNode node) { - if (type.getSqlTypeName() == node.getType().getSqlTypeName() - || (type.getSqlTypeName() == SqlTypeName.VARCHAR - && node.getType().getSqlTypeName() == SqlTypeName.CHAR)) { - return node; - } - return rexBuilder.ensureType(type, node, true); - } - - /** - * Gets the list size threshold under which {@link #convertInToOr} is used. - * Lists of this size or greater will instead be converted to use a join - * against an inline table - * ({@link org.apache.calcite.rel.logical.LogicalValues}) rather than a - * predicate. A threshold of 0 forces usage of an inline table in all cases; a - * threshold of Integer.MAX_VALUE forces usage of OR in all cases - * - * @return threshold, default {@link #DEFAULT_IN_SUB_QUERY_THRESHOLD} - */ - @Deprecated // to be removed before 2.0 - protected int getInSubqueryThreshold() { - return config.getInSubQueryThreshold(); - } - - /** - * Converts an EXISTS or IN predicate into a join. For EXISTS, the sub-query - * produces an indicator variable, and the result is a relational expression - * which outer joins that indicator to the original query. After performing - * the outer join, the condition will be TRUE if the EXISTS condition holds, - * NULL otherwise. - * - * @param seek A query, for example 'select * from emp' or - * 'values (1,2,3)' or '('Foo', 34)'. - * @param subQueryType Whether sub-query is IN, EXISTS or scalar - * @param logic Whether the answer needs to be in full 3-valued logic (TRUE, - * FALSE, UNKNOWN) will be required, or whether we can accept an - * approximation (say representing UNKNOWN as FALSE) - * @param notIn Whether the operation is NOT IN - * @return join expression - */ - private RelOptUtil.Exists convertExists( - SqlNode seek, - RelOptUtil.SubQueryType subQueryType, - RelOptUtil.Logic logic, - boolean notIn, - RelDataType targetDataType) { - final SqlValidatorScope seekScope = - (seek instanceof SqlSelect) - ? validator.getSelectScope((SqlSelect) seek) - : null; - final Blackboard seekBb = createBlackboard(seekScope, null, false); - RelNode seekRel = convertQueryOrInList(seekBb, seek, targetDataType); - - return RelOptUtil.createExistsPlan(seekRel, subQueryType, logic, notIn); - } - - private RelNode convertQueryOrInList( - Blackboard bb, - SqlNode seek, - RelDataType targetRowType) { - // NOTE: Once we start accepting single-row queries as row constructors, - // there will be an ambiguity here for a case like X IN ((SELECT Y FROM - // Z)). The SQL standard resolves the ambiguity by saying that a lone - // select should be interpreted as a table expression, not a row - // expression. The semantic difference is that a table expression can - // return multiple rows. - if (seek instanceof SqlNodeList) { - return convertRowValues( - bb, - seek, - ((SqlNodeList) seek).getList(), - false, - targetRowType); - } else { - return convertQueryRecursive(seek, false, null).project(); - } - } - - private RelNode convertRowValues( - Blackboard bb, - SqlNode rowList, - Collection rows, - boolean allowLiteralsOnly, - RelDataType targetRowType) { - // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of - // literals into a single LogicalValues; this gives the optimizer a smaller - // input tree. For everything else (computed expressions, row - // sub-queries), we union each row in as a projection on top of a - // LogicalOneRow. - - final ImmutableList.Builder> tupleList = - ImmutableList.builder(); - final RelDataType rowType; - if (targetRowType != null) { - rowType = targetRowType; - } else { - rowType = - SqlTypeUtil.promoteToRowType( - typeFactory, - validator.getValidatedNodeType(rowList), - null); - } - - final List unionInputs = new ArrayList<>(); - for (SqlNode node : rows) { - SqlBasicCall call; - if (isRowConstructor(node)) { - call = (SqlBasicCall) node; - ImmutableList.Builder tuple = ImmutableList.builder(); - for (Ord operand : Ord.zip(call.operands)) { - RexLiteral rexLiteral = - convertLiteralInValuesList( - operand.e, - bb, - rowType, - operand.i); - if ((rexLiteral == null) && allowLiteralsOnly) { - return null; - } - if ((rexLiteral == null) || !config.isCreateValuesRel()) { - // fallback to convertRowConstructor - tuple = null; - break; - } - tuple.add(rexLiteral); - } - if (tuple != null) { - tupleList.add(tuple.build()); - continue; - } - } else { - RexLiteral rexLiteral = - convertLiteralInValuesList( - node, - bb, - rowType, - 0); - if ((rexLiteral != null) && config.isCreateValuesRel()) { - tupleList.add(ImmutableList.of(rexLiteral)); - continue; - } else { - if ((rexLiteral == null) && allowLiteralsOnly) { - return null; - } - } - - // convert "1" to "row(1)" - call = - (SqlBasicCall) SqlStdOperatorTable.ROW.createCall( - SqlParserPos.ZERO, - node); - } - unionInputs.add(convertRowConstructor(bb, call)); - } - LogicalValues values = - LogicalValues.create(cluster, rowType, tupleList.build()); - RelNode resultRel; - if (unionInputs.isEmpty()) { - resultRel = values; - } else { - if (!values.getTuples().isEmpty()) { - unionInputs.add(values); - } - resultRel = LogicalUnion.create(unionInputs, true); - } - leaves.add(resultRel); - return resultRel; - } - - private RexLiteral convertLiteralInValuesList( - SqlNode sqlNode, - Blackboard bb, - RelDataType rowType, - int iField) { - if (!(sqlNode instanceof SqlLiteral)) { - return null; - } - RelDataTypeField field = rowType.getFieldList().get(iField); - RelDataType type = field.getType(); - if (type.isStruct()) { - // null literals for weird stuff like UDT's need - // special handling during type flattening, so - // don't use LogicalValues for those - return null; - } - - RexNode literalExpr = - exprConverter.convertLiteral( - bb, - (SqlLiteral) sqlNode); - - if (!(literalExpr instanceof RexLiteral)) { - assert literalExpr.isA(SqlKind.CAST); - RexNode child = ((RexCall) literalExpr).getOperands().get(0); - assert RexLiteral.isNullLiteral(child); - - // NOTE jvs 22-Nov-2006: we preserve type info - // in LogicalValues digest, so it's OK to lose it here - return (RexLiteral) child; - } - - RexLiteral literal = (RexLiteral) literalExpr; - - Comparable value = literal.getValue(); - - if (SqlTypeUtil.isExactNumeric(type) && SqlTypeUtil.hasScale(type)) { - BigDecimal roundedValue = - NumberUtil.rescaleBigDecimal( - (BigDecimal) value, - type.getScale()); - return rexBuilder.makeExactLiteral( - roundedValue, - type); - } - - if ((value instanceof NlsString) - && (type.getSqlTypeName() == SqlTypeName.CHAR)) { - // pad fixed character type - NlsString unpadded = (NlsString) value; - return rexBuilder.makeCharLiteral( - new NlsString( - Spaces.padRight(unpadded.getValue(), type.getPrecision()), - unpadded.getCharsetName(), - unpadded.getCollation())); - } - return literal; - } - - private boolean isRowConstructor(SqlNode node) { - if (!(node.getKind() == SqlKind.ROW)) { - return false; - } - SqlCall call = (SqlCall) node; - return call.getOperator().getName().equalsIgnoreCase("row"); - } - - /** - * Builds a list of all IN or EXISTS operators - * inside SQL parse tree. Does not traverse inside queries. - * - * @param bb blackboard - * @param node the SQL parse tree - * @param logic Whether the answer needs to be in full 3-valued logic (TRUE, - * FALSE, UNKNOWN) will be required, or whether we can accept - * an approximation (say representing UNKNOWN as FALSE) - * @param registerOnlyScalarSubQueries if set to true and the parse tree - * corresponds to a variation of a select - * node, only register it if it's a scalar - * sub-query - */ - private void findSubQueries( - Blackboard bb, - SqlNode node, - RelOptUtil.Logic logic, - boolean registerOnlyScalarSubQueries) { - final SqlKind kind = node.getKind(); - switch (kind) { - case EXISTS: - case SELECT: - case MULTISET_QUERY_CONSTRUCTOR: - case MULTISET_VALUE_CONSTRUCTOR: - case ARRAY_QUERY_CONSTRUCTOR: - case CURSOR: - case SCALAR_QUERY: - if (!registerOnlyScalarSubQueries - || (kind == SqlKind.SCALAR_QUERY)) { - bb.registerSubQuery(node, RelOptUtil.Logic.TRUE_FALSE); - } - return; - case IN: - if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) { - logic = logic.negate(); - } - break; - case NOT: - logic = logic.negate(); - break; - } - if (node instanceof SqlCall) { - for (SqlNode operand : ((SqlCall) node).getOperandList()) { - if (operand != null) { - // In the case of an IN expression, locate scalar - // sub-queries so we can convert them to constants - findSubQueries( - bb, - operand, - logic, - kind == SqlKind.IN || registerOnlyScalarSubQueries); - } - } - } else if (node instanceof SqlNodeList) { - for (SqlNode child : (SqlNodeList) node) { - findSubQueries( - bb, - child, - logic, - kind == SqlKind.IN || registerOnlyScalarSubQueries); - } - } - - // Now that we've located any scalar sub-queries inside the IN - // expression, register the IN expression itself. We need to - // register the scalar sub-queries first so they can be converted - // before the IN expression is converted. - if (kind == SqlKind.IN) { - switch (logic) { - case TRUE_FALSE_UNKNOWN: - if (validator.getValidatedNodeType(node).isNullable()) { - break; - } else if (true) { - break; - } - // fall through - case UNKNOWN_AS_FALSE: - logic = RelOptUtil.Logic.TRUE; - } - bb.registerSubQuery(node, logic); - } - } - - /** - * Converts an expression from {@link SqlNode} to {@link RexNode} format. - * - * @param node Expression to translate - * @return Converted expression - */ - public RexNode convertExpression( - SqlNode node) { - Map nameToTypeMap = Collections.emptyMap(); - final ParameterScope scope = - new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap); - final Blackboard bb = createBlackboard(scope, null, false); - return bb.convertExpression(node); - } - - /** - * Converts an expression from {@link SqlNode} to {@link RexNode} format, - * mapping identifier references to predefined expressions. - * - * @param node Expression to translate - * @param nameToNodeMap map from String to {@link RexNode}; when an - * {@link SqlIdentifier} is encountered, it is used as a - * key and translated to the corresponding value from - * this map - * @return Converted expression - */ - public RexNode convertExpression( - SqlNode node, - Map nameToNodeMap) { - final Map nameToTypeMap = new HashMap<>(); - for (Map.Entry entry : nameToNodeMap.entrySet()) { - nameToTypeMap.put(entry.getKey(), entry.getValue().getType()); - } - final ParameterScope scope = - new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap); - final Blackboard bb = createBlackboard(scope, nameToNodeMap, false); - return bb.convertExpression(node); - } - - /** - * Converts a non-standard expression. - * - *

This method is an extension-point that derived classes can override. If - * this method returns a null result, the normal expression translation - * process will proceed. The default implementation always returns null. - * - * @param node Expression - * @param bb Blackboard - * @return null to proceed with the usual expression translation process - */ - protected RexNode convertExtendedExpression( - SqlNode node, - Blackboard bb) { - return null; - } - - private RexNode convertOver(Blackboard bb, SqlNode node) { - SqlCall call = (SqlCall) node; - SqlCall aggCall = call.operand(0); - SqlNode windowOrRef = call.operand(1); - final SqlWindow window = - validator.resolveWindow(windowOrRef, bb.scope, true); - - // ROW_NUMBER() expects specific kind of framing. - if (aggCall.getKind() == SqlKind.ROW_NUMBER) { - window.setLowerBound(SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO)); - window.setUpperBound(SqlWindow.createCurrentRow(SqlParserPos.ZERO)); - window.setRows(SqlLiteral.createBoolean(true, SqlParserPos.ZERO)); - } - final SqlNodeList partitionList = window.getPartitionList(); - final ImmutableList.Builder partitionKeys = - ImmutableList.builder(); - for (SqlNode partition : partitionList) { - partitionKeys.add(bb.convertExpression(partition)); - } - RexNode lowerBound = bb.convertExpression(window.getLowerBound()); - RexNode upperBound = bb.convertExpression(window.getUpperBound()); - SqlNodeList orderList = window.getOrderList(); - if ((orderList.size() == 0) && !window.isRows()) { - // A logical range requires an ORDER BY clause. Use the implicit - // ordering of this relation. There must be one, otherwise it would - // have failed validation. - orderList = bb.scope.getOrderList(); - if (orderList == null) { - throw new AssertionError( - "Relation should have sort key for implicit ORDER BY"); - } - } - final ImmutableList.Builder orderKeys = - ImmutableList.builder(); - final Set flags = EnumSet.noneOf(SqlKind.class); - for (SqlNode order : orderList) { - flags.clear(); - RexNode e = bb.convertSortExpression(order, flags); - orderKeys.add(new RexFieldCollation(e, flags)); - } - try { - Preconditions.checkArgument(bb.window == null, - "already in window agg mode"); - bb.window = window; - RexNode rexAgg = exprConverter.convertCall(bb, aggCall); - rexAgg = - rexBuilder.ensureType( - validator.getValidatedNodeType(call), rexAgg, false); - - // Walk over the tree and apply 'over' to all agg functions. This is - // necessary because the returned expression is not necessarily a call - // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x). - final RexShuttle visitor = - new HistogramShuttle( - partitionKeys.build(), orderKeys.build(), - RexWindowBound.create(window.getLowerBound(), lowerBound), - RexWindowBound.create(window.getUpperBound(), upperBound), - window); - return rexAgg.accept(visitor); - } finally { - bb.window = null; - } - } - - /** - * Converts a FROM clause into a relational expression. - * - * @param bb Scope within which to resolve identifiers - * @param from FROM clause of a query. Examples include: - * - *

    - *
  • a single table ("SALES.EMP"), - *
  • an aliased table ("EMP AS E"), - *
  • a list of tables ("EMP, DEPT"), - *
  • an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO = - * DEPT.DEPTNO"), - *
  • a VALUES clause ("VALUES ('Fred', 20)"), - *
  • a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"), - *
  • or any combination of the above. - *
- */ - protected void convertFrom( - Blackboard bb, - SqlNode from) { - if (from == null) { - bb.setRoot(LogicalValues.createOneRow(cluster), false); - return; - } - - final SqlCall call; - final SqlNode[] operands; - switch (from.getKind()) { - case MATCH_RECOGNIZE: - convertMatchRecognize(bb, (SqlCall) from); - return; - - case AS: - convertFrom(bb, ((SqlCall) from).operand(0)); - return; - - case WITH_ITEM: - convertFrom(bb, ((SqlWithItem) from).query); - return; - - case WITH: - convertFrom(bb, ((SqlWith) from).body); - return; - - case TABLESAMPLE: - operands = ((SqlBasicCall) from).getOperands(); - SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]); - if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) { - String sampleName = - ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec) - .getName(); - datasetStack.push(sampleName); - convertFrom(bb, operands[0]); - datasetStack.pop(); - } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) { - SqlSampleSpec.SqlTableSampleSpec tableSampleSpec = - (SqlSampleSpec.SqlTableSampleSpec) sampleSpec; - convertFrom(bb, operands[0]); - RelOptSamplingParameters params = - new RelOptSamplingParameters( - tableSampleSpec.isBernoulli(), - tableSampleSpec.getSamplePercentage(), - tableSampleSpec.isRepeatable(), - tableSampleSpec.getRepeatableSeed()); - bb.setRoot(new Sample(cluster, bb.root, params), false); - } else { - throw new AssertionError("unknown TABLESAMPLE type: " + sampleSpec); - } - return; - - case IDENTIFIER: - convertIdentifier(bb, (SqlIdentifier) from, null); - return; - - case EXTEND: - call = (SqlCall) from; - SqlIdentifier id = (SqlIdentifier) call.getOperandList().get(0); - SqlNodeList extendedColumns = (SqlNodeList) call.getOperandList().get(1); - convertIdentifier(bb, id, extendedColumns); - return; - - case JOIN: - final SqlJoin join = (SqlJoin) from; - final SqlValidatorScope scope = validator.getJoinScope(from); - final Blackboard fromBlackboard = createBlackboard(scope, null, false); - SqlNode left = join.getLeft(); - SqlNode right = join.getRight(); - final boolean isNatural = join.isNatural(); - final JoinType joinType = join.getJoinType(); - final SqlValidatorScope leftScope = - Util.first(validator.getJoinScope(left), - ((DelegatingScope) bb.scope).getParent()); - final Blackboard leftBlackboard = - createBlackboard(leftScope, null, false); - final SqlValidatorScope rightScope = - Util.first(validator.getJoinScope(right), - ((DelegatingScope) bb.scope).getParent()); - final Blackboard rightBlackboard = - createBlackboard(rightScope, null, false); - convertFrom(leftBlackboard, left); - RelNode leftRel = leftBlackboard.root; - convertFrom(rightBlackboard, right); - RelNode rightRel = rightBlackboard.root; - JoinRelType convertedJoinType = convertJoinType(joinType); - RexNode conditionExp; - final SqlValidatorNamespace leftNamespace = validator.getNamespace(left); - final SqlValidatorNamespace rightNamespace = validator.getNamespace(right); - if (isNatural) { - final RelDataType leftRowType = leftNamespace.getRowType(); - final RelDataType rightRowType = rightNamespace.getRowType(); - final List columnList = - SqlValidatorUtil.deriveNaturalJoinColumnList(leftRowType, - rightRowType); - conditionExp = convertUsing(leftNamespace, rightNamespace, - columnList); - } else { - conditionExp = - convertJoinCondition( - fromBlackboard, - leftNamespace, - rightNamespace, - join.getCondition(), - join.getConditionType(), - leftRel, - rightRel); - } - - final RelNode joinRel = - createJoin( - fromBlackboard, - leftRel, - rightRel, - conditionExp, - convertedJoinType); - bb.setRoot(joinRel, false); - return; - - case SELECT: - case INTERSECT: - case EXCEPT: - case UNION: - final RelNode rel = convertQueryRecursive(from, false, null).project(); - bb.setRoot(rel, true); - return; - - case VALUES: - convertValuesImpl(bb, (SqlCall) from, null); - return; - - case UNNEST: - call = (SqlCall) from; - final List nodes = call.getOperandList(); - final SqlUnnestOperator operator = (SqlUnnestOperator) call.getOperator(); - for (SqlNode node : nodes) { - replaceSubQueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - } - final List exprs = new ArrayList<>(); - final List fieldNames = new ArrayList<>(); - for (Ord node : Ord.zip(nodes)) { - exprs.add(bb.convertExpression(node.e)); - fieldNames.add(validator.deriveAlias(node.e, node.i)); - } - final RelNode input = - RelOptUtil.createProject( - (null != bb.root) ? bb.root : LogicalValues.createOneRow(cluster), - exprs, fieldNames, true); - - Uncollect uncollect = - new Uncollect(cluster, cluster.traitSetOf(Convention.NONE), - input, operator.withOrdinality); - bb.setRoot(uncollect, true); - return; - - case COLLECTION_TABLE: - call = (SqlCall) from; - - // Dig out real call; TABLE() wrapper is just syntactic. - assert call.getOperandList().size() == 1; - final SqlCall call2 = call.operand(0); - convertCollectionTable(bb, call2); - return; - - default: - throw new AssertionError("not a join operator " + from); - } - } - - protected void convertMatchRecognize(Blackboard bb, SqlCall call) { - final SqlMatchRecognize matchRecognize = (SqlMatchRecognize) call; - final SqlValidatorNamespace ns = validator.getNamespace(matchRecognize); - final SqlValidatorScope scope = validator.getMatchRecognizeScope(matchRecognize); - - final Blackboard mrBlackBoard = createBlackboard(scope, null, false); - final RelDataType rowType = ns.getRowType(); - // convert inner query, could be a table name or a derived table - SqlNode expr = matchRecognize.getTableRef(); - convertFrom(mrBlackBoard, expr); - final RelNode input = mrBlackBoard.root; - - // convert pattern - final Set patternVarsSet = new HashSet<>(); - SqlNode pattern = matchRecognize.getPattern(); - final SqlBasicVisitor patternVarVisitor = - new SqlBasicVisitor() { - @Override public RexNode visit(SqlCall call) { - List operands = call.getOperandList(); - List newOperands = Lists.newArrayList(); - for (SqlNode node : operands) { - newOperands.add(node.accept(this)); - } - return rexBuilder.makeCall( - validator.getUnknownType(), call.getOperator(), newOperands); - } - - @Override public RexNode visit(SqlIdentifier id) { - assert id.isSimple(); - patternVarsSet.add(id.getSimple()); - return rexBuilder.makeLiteral(id.getSimple()); - } - - @Override public RexNode visit(SqlLiteral literal) { - if (literal instanceof SqlNumericLiteral) { - return rexBuilder.makeExactLiteral(BigDecimal.valueOf(literal.intValue(true))); - } else { - return rexBuilder.makeLiteral(literal.booleanValue()); - } - } - }; - final RexNode patternNode = pattern.accept(patternVarVisitor); - - mrBlackBoard.setPatternVarRef(true); - - // convert definitions - final ImmutableMap.Builder definitionNodes = - ImmutableMap.builder(); - for (SqlNode def : matchRecognize.getPatternDefList()) { - List operands = ((SqlCall) def).getOperandList(); - String alias = ((SqlIdentifier) operands.get(1)).getSimple(); - RexNode rex = mrBlackBoard.convertExpression(operands.get(0)); - definitionNodes.put(alias, rex); - } - - mrBlackBoard.setPatternVarRef(false); - - final RelFactories.MatchFactory factory = - RelFactories.DEFAULT_MATCH_FACTORY; - final RelNode rel = - factory.createMatchRecognize(input, patternNode, - matchRecognize.getStrictStart().booleanValue(), - matchRecognize.getStrictEnd().booleanValue(), - definitionNodes.build(), - rowType); - bb.setRoot(rel, false); - } - - private void convertIdentifier(Blackboard bb, SqlIdentifier id, - SqlNodeList extendedColumns) { - final SqlValidatorNamespace fromNamespace = - validator.getNamespace(id).resolve(); - if (fromNamespace.getNode() != null) { - convertFrom(bb, fromNamespace.getNode()); - return; - } - final String datasetName = - datasetStack.isEmpty() ? null : datasetStack.peek(); - final boolean[] usedDataset = {false}; - RelOptTable table = - SqlValidatorUtil.getRelOptTable(fromNamespace, catalogReader, - datasetName, usedDataset); - if (extendedColumns != null && extendedColumns.size() > 0) { - assert table != null; - final SqlValidatorTable validatorTable = - table.unwrap(SqlValidatorTable.class); - final List extendedFields = - SqlValidatorUtil.getExtendedColumns(validator, validatorTable, - extendedColumns); - table = table.extend(extendedFields); - } - final RelNode tableRel; - if (config.isConvertTableAccess()) { - tableRel = toRel(table); - } else { - tableRel = LogicalTableScan.create(cluster, table); - } - bb.setRoot(tableRel, true); - if (usedDataset[0]) { - bb.setDataset(datasetName); - } - } - - protected void convertCollectionTable( - Blackboard bb, - SqlCall call) { - final SqlOperator operator = call.getOperator(); - if (operator == SqlStdOperatorTable.TABLESAMPLE) { - final String sampleName = (String) SqlLiteral.value(call.operand(0)); - datasetStack.push(sampleName); - SqlCall cursorCall = call.operand(1); - SqlNode query = cursorCall.operand(0); - RelNode converted = convertQuery(query, false, false).rel; - bb.setRoot(converted, false); - datasetStack.pop(); - return; - } - replaceSubQueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - - // Expand table macro if possible. It's more efficient than - // LogicalTableFunctionScan. - final SqlCallBinding callBinding = - new SqlCallBinding(bb.scope.getValidator(), bb.scope, call); - if (operator instanceof SqlUserDefinedTableMacro) { - final SqlUserDefinedTableMacro udf = - (SqlUserDefinedTableMacro) operator; - final TranslatableTable table = - udf.getTable(typeFactory, callBinding.operands()); - final RelDataType rowType = table.getRowType(typeFactory); - RelOptTable relOptTable = RelOptTableImpl.create(null, rowType, table, - udf.getNameAsId().names); - RelNode converted = toRel(relOptTable); - bb.setRoot(converted, true); - return; - } - - Type elementType; - if (operator instanceof SqlUserDefinedTableFunction) { - SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator; - elementType = udtf.getElementType(typeFactory, callBinding.operands()); - } else { - elementType = null; - } - - RexNode rexCall = bb.convertExpression(call); - final List inputs = bb.retrieveCursors(); - Set columnMappings = - getColumnMappings(operator); - LogicalTableFunctionScan callRel = - LogicalTableFunctionScan.create( - cluster, - inputs, - rexCall, - elementType, - validator.getValidatedNodeType(call), - columnMappings); - bb.setRoot(callRel, true); - afterTableFunction(bb, call, callRel); - } - - protected void afterTableFunction( - SqlToRelConverter.Blackboard bb, - SqlCall call, - LogicalTableFunctionScan callRel) { - } - - private Set getColumnMappings(SqlOperator op) { - SqlReturnTypeInference rti = op.getReturnTypeInference(); - if (rti == null) { - return null; - } - if (rti instanceof TableFunctionReturnTypeInference) { - TableFunctionReturnTypeInference tfrti = - (TableFunctionReturnTypeInference) rti; - return tfrti.getColumnMappings(); - } else { - return null; - } - } - - protected RelNode createJoin( - Blackboard bb, - RelNode leftRel, - RelNode rightRel, - RexNode joinCond, - JoinRelType joinType) { - assert joinCond != null; - - final CorrelationUse p = getCorrelationUse(bb, rightRel); - if (p != null) { - LogicalCorrelate corr = LogicalCorrelate.create(leftRel, p.r, - p.id, p.requiredColumns, SemiJoinType.of(joinType)); - if (!joinCond.isAlwaysTrue()) { - final RelFactories.FilterFactory factory = - RelFactories.DEFAULT_FILTER_FACTORY; - return factory.createFilter(corr, joinCond); - } - return corr; - } - - final Join originalJoin = - (Join) RelFactories.DEFAULT_JOIN_FACTORY.createJoin(leftRel, rightRel, - joinCond, ImmutableSet.of(), joinType, false); - - return RelOptUtil.pushDownJoinConditions(originalJoin); - } - - private CorrelationUse getCorrelationUse(Blackboard bb, final RelNode r0) { - final Set correlatedVariables = - RelOptUtil.getVariablesUsed(r0); - if (correlatedVariables.isEmpty()) { - return null; - } - final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder(); - final List correlNames = Lists.newArrayList(); - - // All correlations must refer the same namespace since correlation - // produces exactly one correlation source. - // The same source might be referenced by different variables since - // DeferredLookups are not de-duplicated at create time. - SqlValidatorNamespace prevNs = null; - - for (CorrelationId correlName : correlatedVariables) { - DeferredLookup lookup = - mapCorrelToDeferred.get(correlName); - RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName); - String originalRelName = lookup.getOriginalRelName(); - String originalFieldName = fieldAccess.getField().getName(); - - final SqlNameMatcher nameMatcher = - lookup.bb.scope.getValidator().getCatalogReader().nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - lookup.bb.scope.resolve(ImmutableList.of(originalRelName), - nameMatcher, false, resolved); - assert resolved.count() == 1; - final SqlValidatorScope.Resolve resolve = resolved.only(); - final SqlValidatorNamespace foundNs = resolve.namespace; - final RelDataType rowType = resolve.rowType(); - final int childNamespaceIndex = resolve.path.steps().get(0).i; - final SqlValidatorScope ancestorScope = resolve.scope; - boolean correlInCurrentScope = ancestorScope == bb.scope; - - if (!correlInCurrentScope) { - continue; - } - - if (prevNs == null) { - prevNs = foundNs; - } else { - assert prevNs == foundNs : "All correlation variables should resolve" - + " to the same namespace." - + " Prev ns=" + prevNs - + ", new ns=" + foundNs; - } - - int namespaceOffset = 0; - if (childNamespaceIndex > 0) { - // If not the first child, need to figure out the width - // of output types from all the preceding namespaces - assert ancestorScope instanceof ListScope; - List children = - ((ListScope) ancestorScope).getChildren(); - - for (int i = 0; i < childNamespaceIndex; i++) { - SqlValidatorNamespace child = children.get(i); - namespaceOffset += - child.getRowType().getFieldCount(); - } - } - - RexFieldAccess topLevelFieldAccess = fieldAccess; - while (topLevelFieldAccess.getReferenceExpr() instanceof RexFieldAccess) { - topLevelFieldAccess = (RexFieldAccess) topLevelFieldAccess.getReferenceExpr(); - } - final RelDataTypeField field = rowType.getFieldList() - .get(topLevelFieldAccess.getField().getIndex() - namespaceOffset); - int pos = namespaceOffset + field.getIndex(); - - assert field.getType() - == topLevelFieldAccess.getField().getType(); - - assert pos != -1; - - if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) { - // bb.root is an aggregate and only projects group by - // keys. - Map exprProjection = - bb.mapRootRelToFieldProjection.get(bb.root); - - // sub-query can reference group by keys projected from - // the root of the outer relation. - if (exprProjection.containsKey(pos)) { - pos = exprProjection.get(pos); - } else { - // correl not grouped - throw new AssertionError("Identifier '" + originalRelName + "." - + originalFieldName + "' is not a group expr"); - } - } - - requiredColumns.set(pos); - correlNames.add(correlName); - } - - if (correlNames.isEmpty()) { - // None of the correlating variables originated in this scope. - return null; - } - - RelNode r = r0; - if (correlNames.size() > 1) { - // The same table was referenced more than once. - // So we deduplicate - r = DeduplicateCorrelateVariables.go(rexBuilder, correlNames.get(0), - Util.skip(correlNames), r0); - } - return new CorrelationUse(correlNames.get(0), requiredColumns.build(), r); - } - - /** - * Determines whether a sub-query is non-correlated. Note that a - * non-correlated sub-query can contain correlated references, provided those - * references do not reference select statements that are parents of the - * sub-query. - * - * @param subq the sub-query - * @param bb blackboard used while converting the sub-query, i.e., the - * blackboard of the parent query of this sub-query - * @return true if the sub-query is non-correlated - */ - private boolean isSubQueryNonCorrelated(RelNode subq, Blackboard bb) { - Set correlatedVariables = RelOptUtil.getVariablesUsed(subq); - for (CorrelationId correlName : correlatedVariables) { - DeferredLookup lookup = mapCorrelToDeferred.get(correlName); - String originalRelName = lookup.getOriginalRelName(); - - final SqlNameMatcher nameMatcher = - lookup.bb.scope.getValidator().getCatalogReader().nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - lookup.bb.scope.resolve(ImmutableList.of(originalRelName), nameMatcher, - false, resolved); - - SqlValidatorScope ancestorScope = resolved.only().scope; - - // If the correlated reference is in a scope that's "above" the - // sub-query, then this is a correlated sub-query. - SqlValidatorScope parentScope = bb.scope; - do { - if (ancestorScope == parentScope) { - return false; - } - if (parentScope instanceof DelegatingScope) { - parentScope = ((DelegatingScope) parentScope).getParent(); - } else { - break; - } - } while (parentScope != null); - } - return true; - } - - /** - * Returns a list of fields to be prefixed to each relational expression. - * - * @return List of system fields - */ - protected List getSystemFields() { - return Collections.emptyList(); - } - - private RexNode convertJoinCondition(Blackboard bb, - SqlValidatorNamespace leftNamespace, - SqlValidatorNamespace rightNamespace, - SqlNode condition, - JoinConditionType conditionType, - RelNode leftRel, - RelNode rightRel) { - if (condition == null) { - return rexBuilder.makeLiteral(true); - } - bb.setRoot(ImmutableList.of(leftRel, rightRel)); - replaceSubQueries(bb, condition, RelOptUtil.Logic.UNKNOWN_AS_FALSE); - switch (conditionType) { - case ON: - bb.setRoot(ImmutableList.of(leftRel, rightRel)); - return bb.convertExpression(condition); - case USING: - final SqlNodeList list = (SqlNodeList) condition; - final List nameList = new ArrayList<>(); - for (SqlNode columnName : list) { - final SqlIdentifier id = (SqlIdentifier) columnName; - String name = id.getSimple(); - nameList.add(name); - } - return convertUsing(leftNamespace, rightNamespace, nameList); - default: - throw Util.unexpected(conditionType); - } - } - - /** - * Returns an expression for matching columns of a USING clause or inferred - * from NATURAL JOIN. "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y = - * b.y". Returns null if the column list is empty. - * - * @param leftNamespace Namespace of left input to join - * @param rightNamespace Namespace of right input to join - * @param nameList List of column names to join on - * @return Expression to match columns from name list, or true if name list - * is empty - */ - private RexNode convertUsing(SqlValidatorNamespace leftNamespace, - SqlValidatorNamespace rightNamespace, - List nameList) { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final List list = Lists.newArrayList(); - for (String name : nameList) { - List operands = new ArrayList<>(); - int offset = 0; - for (SqlValidatorNamespace n : ImmutableList.of(leftNamespace, - rightNamespace)) { - final RelDataType rowType = n.getRowType(); - final RelDataTypeField field = nameMatcher.field(rowType, name); - operands.add( - rexBuilder.makeInputRef(field.getType(), - offset + field.getIndex())); - offset += rowType.getFieldList().size(); - } - list.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, operands)); - } - return RexUtil.composeConjunction(rexBuilder, list, false); - } - - private static JoinRelType convertJoinType(JoinType joinType) { - switch (joinType) { - case COMMA: - case INNER: - case CROSS: - return JoinRelType.INNER; - case FULL: - return JoinRelType.FULL; - case LEFT: - return JoinRelType.LEFT; - case RIGHT: - return JoinRelType.RIGHT; - default: - throw Util.unexpected(joinType); - } - } - - /** - * Converts the SELECT, GROUP BY and HAVING clauses of an aggregate query. - * - *

This method extracts SELECT, GROUP BY and HAVING clauses, and creates - * an {@link AggConverter}, then delegates to {@link #createAggImpl}. - * Derived class may override this method to change any of those clauses or - * specify a different {@link AggConverter}. - * - * @param bb Scope within which to resolve identifiers - * @param select Query - * @param orderExprList Additional expressions needed to implement ORDER BY - */ - protected void convertAgg( - Blackboard bb, - SqlSelect select, - List orderExprList) { - assert bb.root != null : "precondition: child != null"; - SqlNodeList groupList = select.getGroup(); - SqlNodeList selectList = select.getSelectList(); - SqlNode having = select.getHaving(); - - final AggConverter aggConverter = new AggConverter(bb, select); - createAggImpl( - bb, - aggConverter, - selectList, - groupList, - having, - orderExprList); - } - - protected final void createAggImpl( - Blackboard bb, - final AggConverter aggConverter, - SqlNodeList selectList, - SqlNodeList groupList, - SqlNode having, - List orderExprList) { - // Find aggregate functions in SELECT and HAVING clause - final AggregateFinder aggregateFinder = new AggregateFinder(); - selectList.accept(aggregateFinder); - if (having != null) { - having.accept(aggregateFinder); - } - - // first replace the sub-queries inside the aggregates - // because they will provide input rows to the aggregates. - replaceSubQueries(bb, aggregateFinder.list, - RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - - // If group-by clause is missing, pretend that it has zero elements. - if (groupList == null) { - groupList = SqlNodeList.EMPTY; - } - - replaceSubQueries(bb, groupList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - - // register the group exprs - - // build a map to remember the projections from the top scope to the - // output of the current root. - // - // Calcite allows expressions, not just column references in - // group by list. This is not SQL 2003 compliant, but hey. - - final AggregatingSelectScope scope = aggConverter.aggregatingSelectScope; - final AggregatingSelectScope.Resolved r = scope.resolved.get(); - for (SqlNode groupExpr : r.groupExprList) { - aggConverter.addGroupExpr(groupExpr); - } - - RexNode havingExpr = null; - final List> projects = Lists.newArrayList(); - - try { - Preconditions.checkArgument(bb.agg == null, "already in agg mode"); - bb.agg = aggConverter; - - // convert the select and having expressions, so that the - // agg converter knows which aggregations are required - - selectList.accept(aggConverter); - // Assert we don't have dangling items left in the stack - assert !aggConverter.inOver; - for (SqlNode expr : orderExprList) { - expr.accept(aggConverter); - assert !aggConverter.inOver; - } - if (having != null) { - having.accept(aggConverter); - assert !aggConverter.inOver; - } - - // compute inputs to the aggregator - List> preExprs = aggConverter.getPreExprs(); - - if (preExprs.size() == 0) { - // Special case for COUNT(*), where we can end up with no inputs - // at all. The rest of the system doesn't like 0-tuples, so we - // select a dummy constant here. - final RexNode zero = rexBuilder.makeExactLiteral(BigDecimal.ZERO); - preExprs = ImmutableList.of(Pair.of(zero, (String) null)); - } - - final RelNode inputRel = bb.root; - - // Project the expressions required by agg and having. - bb.setRoot( - RelOptUtil.createProject( - inputRel, - preExprs, - true), - false); - bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection); - - // REVIEW jvs 31-Oct-2007: doesn't the declaration of - // monotonicity here assume sort-based aggregation at - // the physical level? - - // Tell bb which of group columns are sorted. - bb.columnMonotonicities.clear(); - for (SqlNode groupItem : groupList) { - bb.columnMonotonicities.add( - bb.scope.getMonotonicity(groupItem)); - } - - // Add the aggregator - bb.setRoot( - createAggregate(bb, r.indicator, r.groupSet, r.groupSets, - aggConverter.getAggCalls()), - false); - - // Generate NULL values for rolled-up not-null fields. - final Aggregate aggregate = (Aggregate) bb.root; - if (aggregate.getGroupType() != Aggregate.Group.SIMPLE) { - assert aggregate.indicator; - List> projects2 = Lists.newArrayList(); - int converted = 0; - final int groupCount = aggregate.getGroupSet().cardinality(); - for (RelDataTypeField field : aggregate.getRowType().getFieldList()) { - final int i = field.getIndex(); - final RexNode rex; - if (i < groupCount && r.isNullable(i)) { - ++converted; - - rex = rexBuilder.makeCall(SqlStdOperatorTable.CASE, - rexBuilder.makeInputRef(aggregate, groupCount + i), - rexBuilder.makeCast( - typeFactory.createTypeWithNullability( - field.getType(), true), - rexBuilder.constantNull()), - rexBuilder.makeInputRef(aggregate, i)); - } else { - rex = rexBuilder.makeInputRef(aggregate, i); - } - projects2.add(Pair.of(rex, field.getName())); - } - if (converted > 0) { - bb.setRoot( - RelOptUtil.createProject(bb.root, projects2, true), - false); - } - } - - bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection); - - // Replace sub-queries in having here and modify having to use - // the replaced expressions - if (having != null) { - SqlNode newHaving = pushDownNotForIn(bb.scope, having); - replaceSubQueries(bb, newHaving, RelOptUtil.Logic.UNKNOWN_AS_FALSE); - havingExpr = bb.convertExpression(newHaving); - if (havingExpr.isAlwaysTrue()) { - havingExpr = null; - } - } - - // Now convert the other sub-queries in the select list. - // This needs to be done separately from the sub-query inside - // any aggregate in the select list, and after the aggregate rel - // is allocated. - replaceSubQueries(bb, selectList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - - // Now sub-queries in the entire select list have been converted. - // Convert the select expressions to get the final list to be - // projected. - int k = 0; - - // For select expressions, use the field names previously assigned - // by the validator. If we derive afresh, we might generate names - // like "EXPR$2" that don't match the names generated by the - // validator. This is especially the case when there are system - // fields; system fields appear in the relnode's rowtype but do not - // (yet) appear in the validator type. - final SelectScope selectScope = - SqlValidatorUtil.getEnclosingSelectScope(bb.scope); - assert selectScope != null; - final SqlValidatorNamespace selectNamespace = - validator.getNamespace(selectScope.getNode()); - final List names = - selectNamespace.getRowType().getFieldNames(); - int sysFieldCount = selectList.size() - names.size(); - for (SqlNode expr : selectList) { - projects.add( - Pair.of(bb.convertExpression(expr), - k < sysFieldCount - ? validator.deriveAlias(expr, k++) - : names.get(k++ - sysFieldCount))); - } - - for (SqlNode expr : orderExprList) { - projects.add( - Pair.of(bb.convertExpression(expr), - validator.deriveAlias(expr, k++))); - } - } finally { - bb.agg = null; - } - - // implement HAVING (we have already checked that it is non-trivial) - if (havingExpr != null) { - final RelFactories.FilterFactory factory = - RelFactories.DEFAULT_FILTER_FACTORY; - bb.setRoot(factory.createFilter(bb.root, havingExpr), false); - } - - // implement the SELECT list - bb.setRoot( - RelOptUtil.createProject( - bb.root, - projects, - true), - false); - - // Tell bb which of group columns are sorted. - bb.columnMonotonicities.clear(); - for (SqlNode selectItem : selectList) { - bb.columnMonotonicities.add( - bb.scope.getMonotonicity(selectItem)); - } - } - - /** - * Creates an Aggregate. - * - *

In case the aggregate rel changes the order in which it projects - * fields, the groupExprProjection parameter is provided, and - * the implementation of this method may modify it. - * - *

The sortedCount parameter is the number of expressions - * known to be monotonic. These expressions must be on the leading edge of - * the grouping keys. The default implementation of this method ignores this - * parameter. - * - * @param bb Blackboard - * @param indicator Whether to output fields indicating grouping sets - * @param groupSet Bit set of ordinals of grouping columns - * @param groupSets Grouping sets - * @param aggCalls Array of calls to aggregate functions - * @return LogicalAggregate - */ - protected RelNode createAggregate(Blackboard bb, boolean indicator, - ImmutableBitSet groupSet, ImmutableList groupSets, - List aggCalls) { - return LogicalAggregate.create( - bb.root, indicator, groupSet, groupSets, aggCalls); - } - - public RexDynamicParam convertDynamicParam( - final SqlDynamicParam dynamicParam) { - // REVIEW jvs 8-Jan-2005: dynamic params may be encountered out of - // order. Should probably cross-check with the count from the parser - // at the end and make sure they all got filled in. Why doesn't List - // have a resize() method?!? Make this a utility. - while (dynamicParam.getIndex() >= dynamicParamSqlNodes.size()) { - dynamicParamSqlNodes.add(null); - } - - dynamicParamSqlNodes.set( - dynamicParam.getIndex(), - dynamicParam); - return rexBuilder.makeDynamicParam( - getDynamicParamType(dynamicParam.getIndex()), - dynamicParam.getIndex()); - } - - /** - * Creates a list of collations required to implement the ORDER BY clause, - * if there is one. Populates extraOrderExprs with any sort - * expressions which are not in the select clause. - * - * @param bb Scope within which to resolve identifiers - * @param select Select clause. Never null, because we invent a - * dummy SELECT if ORDER BY is applied to a set - * operation (UNION etc.) - * @param orderList Order by clause, may be null - * @param extraOrderExprs Sort expressions which are not in the select - * clause (output) - * @param collationList List of collations (output) - */ - protected void gatherOrderExprs( - Blackboard bb, - SqlSelect select, - SqlNodeList orderList, - List extraOrderExprs, - List collationList) { - // TODO: add validation rules to SqlValidator also - assert bb.root != null : "precondition: child != null"; - assert select != null; - if (orderList == null) { - return; - } - for (SqlNode orderItem : orderList) { - collationList.add( - convertOrderItem( - select, - orderItem, - extraOrderExprs, - RelFieldCollation.Direction.ASCENDING, - RelFieldCollation.NullDirection.UNSPECIFIED)); - } - } - - protected RelFieldCollation convertOrderItem( - SqlSelect select, - SqlNode orderItem, List extraExprs, - RelFieldCollation.Direction direction, - RelFieldCollation.NullDirection nullDirection) { - assert select != null; - // Handle DESC keyword, e.g. 'select a, b from t order by a desc'. - switch (orderItem.getKind()) { - case DESCENDING: - return convertOrderItem( - select, - ((SqlCall) orderItem).operand(0), - extraExprs, - RelFieldCollation.Direction.DESCENDING, - nullDirection); - case NULLS_FIRST: - return convertOrderItem( - select, - ((SqlCall) orderItem).operand(0), - extraExprs, - direction, - RelFieldCollation.NullDirection.FIRST); - case NULLS_LAST: - return convertOrderItem( - select, - ((SqlCall) orderItem).operand(0), - extraExprs, - direction, - RelFieldCollation.NullDirection.LAST); - } - - SqlNode converted = validator.expandOrderExpr(select, orderItem); - - switch (nullDirection) { - case UNSPECIFIED: - nullDirection = validator.getDefaultNullCollation().last(desc(direction)) - ? RelFieldCollation.NullDirection.LAST - : RelFieldCollation.NullDirection.FIRST; - } - - // Scan the select list and order exprs for an identical expression. - final SelectScope selectScope = validator.getRawSelectScope(select); - int ordinal = -1; - for (SqlNode selectItem : selectScope.getExpandedSelectList()) { - ++ordinal; - if (converted.equalsDeep(stripAs(selectItem), Litmus.IGNORE)) { - return new RelFieldCollation(ordinal, direction, nullDirection); - } - } - - for (SqlNode extraExpr : extraExprs) { - ++ordinal; - if (converted.equalsDeep(extraExpr, Litmus.IGNORE)) { - return new RelFieldCollation(ordinal, direction, nullDirection); - } - } - - // TODO: handle collation sequence - // TODO: flag expressions as non-standard - - extraExprs.add(converted); - return new RelFieldCollation(ordinal + 1, direction, nullDirection); - } - - private static boolean desc(RelFieldCollation.Direction direction) { - switch (direction) { - case DESCENDING: - case STRICTLY_DESCENDING: - return true; - default: - return false; - } - } - - @Deprecated // to be removed before 2.0 - protected boolean enableDecorrelation() { - // disable sub-query decorrelation when needed. - // e.g. if outer joins are not supported. - return config.isDecorrelationEnabled(); - } - - protected RelNode decorrelateQuery(RelNode rootRel) { - return RelDecorrelator.decorrelateQuery(rootRel); - } - - /** - * Returns whether to trim unused fields as part of the conversion process. - * - * @return Whether to trim unused fields - */ - @Deprecated // to be removed before 2.0 - public boolean isTrimUnusedFields() { - return config.isTrimUnusedFields(); - } - - /** - * Recursively converts a query to a relational expression. - * - * @param query Query - * @param top Whether this query is the top-level query of the - * statement - * @param targetRowType Target row type, or null - * @return Relational expression - */ - protected RelRoot convertQueryRecursive(SqlNode query, boolean top, - RelDataType targetRowType) { - final SqlKind kind = query.getKind(); - switch (kind) { - case SELECT: - return RelRoot.of(convertSelect((SqlSelect) query, top), kind); - case INSERT: - return RelRoot.of(convertInsert((SqlInsert) query), kind); - case DELETE: - return RelRoot.of(convertDelete((SqlDelete) query), kind); - case UPDATE: - return RelRoot.of(convertUpdate((SqlUpdate) query), kind); - case MERGE: - return RelRoot.of(convertMerge((SqlMerge) query), kind); - case UNION: - case INTERSECT: - case EXCEPT: - return RelRoot.of(convertSetOp((SqlCall) query), kind); - case WITH: - return convertWith((SqlWith) query, top); - case VALUES: - return RelRoot.of(convertValues((SqlCall) query, targetRowType), kind); - default: - throw new AssertionError("not a query: " + query); - } - } - - /** - * Converts a set operation (UNION, INTERSECT, MINUS) into relational - * expressions. - * - * @param call Call to set operator - * @return Relational expression - */ - protected RelNode convertSetOp(SqlCall call) { - final RelNode left = - convertQueryRecursive(call.operand(0), false, null).project(); - final RelNode right = - convertQueryRecursive(call.operand(1), false, null).project(); - switch (call.getKind()) { - case UNION: - return LogicalUnion.create(ImmutableList.of(left, right), all(call)); - - case INTERSECT: - return LogicalIntersect.create(ImmutableList.of(left, right), all(call)); - - case EXCEPT: - return LogicalMinus.create(ImmutableList.of(left, right), all(call)); - - default: - throw Util.unexpected(call.getKind()); - } - } - - private boolean all(SqlCall call) { - return ((SqlSetOperator) call.getOperator()).isAll(); - } - - protected RelNode convertInsert(SqlInsert call) { - RelOptTable targetTable = getTargetTable(call); - - final RelDataType targetRowType = - validator.getValidatedNodeType(call); - assert targetRowType != null; - RelNode sourceRel = - convertQueryRecursive(call.getSource(), false, targetRowType).project(); - RelNode massagedRel = convertColumnList(call, sourceRel); - - return createModify(targetTable, massagedRel); - } - - /** Creates a relational expression to modify a table or modifiable view. */ - private RelNode createModify(RelOptTable targetTable, RelNode source) { - final ModifiableTable modifiableTable = - targetTable.unwrap(ModifiableTable.class); - if (modifiableTable != null) { - return modifiableTable.toModificationRel(cluster, targetTable, - catalogReader, source, LogicalTableModify.Operation.INSERT, null, - null, false); - } - final ModifiableView modifiableView = - targetTable.unwrap(ModifiableView.class); - if (modifiableView != null) { - final Table delegateTable = modifiableView.getTable(); - final RelDataType delegateRowType = delegateTable.getRowType(typeFactory); - final RelOptTable delegateRelOptTable = - RelOptTableImpl.create(null, delegateRowType, delegateTable, - modifiableView.getTablePath()); - final RelNode newSource = - createSource(targetTable, source, modifiableView, delegateRowType); - return createModify(delegateRelOptTable, newSource); - } - return LogicalTableModify.create(targetTable, catalogReader, source, - LogicalTableModify.Operation.INSERT, null, null, false); - } - - /** Wraps a relational expression in the projects and filters implied by - * a {@link ModifiableView}. - * - *

The input relational expression is suitable for inserting into the view, - * and the returned relational expression is suitable for inserting into its - * delegate table. - * - *

In principle, the delegate table of a view might be another modifiable - * view, and if so, the process can be repeated. */ - private RelNode createSource(RelOptTable targetTable, RelNode source, - ModifiableView modifiableView, RelDataType delegateRowType) { - final ImmutableIntList mapping = modifiableView.getColumnMapping(); - assert mapping.size() == targetTable.getRowType().getFieldCount(); - - // For columns represented in the mapping, the expression is just a field - // reference. - final Map projectMap = new HashMap<>(); - final List filters = new ArrayList<>(); - for (int i = 0; i < mapping.size(); i++) { - int target = mapping.get(i); - if (target >= 0) { - projectMap.put(target, RexInputRef.of(i, source.getRowType())); - } - } - - // For columns that are not in the mapping, and have a constraint of the - // form "column = value", the expression is the literal "value". - // - // If a column has multiple constraints, the extra ones will become a - // filter. - final RexNode constraint = - modifiableView.getConstraint(rexBuilder, delegateRowType); - RelOptUtil.inferViewPredicates(projectMap, filters, constraint); - final List> projects = new ArrayList<>(); - for (RelDataTypeField field : delegateRowType.getFieldList()) { - RexNode node = projectMap.get(field.getIndex()); - if (node == null) { - node = rexBuilder.makeNullLiteral(field.getType()); - } - projects.add( - Pair.of(rexBuilder.ensureType(field.getType(), node, false), - field.getName())); - } - - source = RelOptUtil.createProject(source, projects, true); - if (filters.size() > 0) { - source = RelOptUtil.createFilter(source, filters); - } - return source; - } - - private RelOptTable.ToRelContext createToRelContext() { - return new RelOptTable.ToRelContext() { - public RelOptCluster getCluster() { - return cluster; - } - - @Override public RelRoot expandView( - RelDataType rowType, - String queryString, - List schemaPath, - List viewPath) { - return viewExpander.expandView(rowType, queryString, schemaPath, viewPath); - } - - }; - } - - public RelNode toRel(RelOptTable table) { - return table.toRel(createToRelContext()); - } - - protected RelOptTable getTargetTable(SqlNode call) { - SqlValidatorNamespace targetNs = validator.getNamespace(call).resolve(); - return SqlValidatorUtil.getRelOptTable(targetNs, catalogReader, null, null); - } - - /** - * Creates a source for an INSERT statement. - * - *

If the column list is not specified, source expressions match target - * columns in order. - * - *

If the column list is specified, Source expressions are mapped to - * target columns by name via targetColumnList, and may not cover the entire - * target table. So, we'll make up a full row, using a combination of - * default values and the source expressions provided. - * - * @param call Insert expression - * @param sourceRel Source relational expression - * @return Converted INSERT statement - */ - protected RelNode convertColumnList( - SqlInsert call, - RelNode sourceRel) { - RelDataType sourceRowType = sourceRel.getRowType(); - final RexNode sourceRef = - rexBuilder.makeRangeReference(sourceRowType, 0, false); - final List targetColumnNames = new ArrayList<>(); - final List columnExprs = new ArrayList<>(); - collectInsertTargets(call, sourceRef, targetColumnNames, columnExprs); - - final RelOptTable targetTable = getTargetTable(call); - final RelDataType targetRowType = targetTable.getRowType(); - final List targetFields = - targetRowType.getFieldList(); - final List sourceExps = - new ArrayList<>( - Collections.nCopies(targetFields.size(), null)); - final List fieldNames = - new ArrayList<>( - Collections.nCopies(targetFields.size(), null)); - - final InitializerExpressionFactory initializerFactory = - getInitializerFactory(validator.getNamespace(call).getTable()); - - // Walk the name list and place the associated value in the - // expression list according to the ordinal value returned from - // the table construct, leaving nulls in the list for columns - // that are not referenced. - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - for (Pair p : Pair.zip(targetColumnNames, columnExprs)) { - RelDataTypeField field = nameMatcher.field(targetRowType, p.left); - assert field != null : "column " + p.left + " not found"; - sourceExps.set(field.getIndex(), p.right); - } - - // Walk the expression list and get default values for any columns - // that were not supplied in the statement. Get field names too. - for (int i = 0; i < targetFields.size(); ++i) { - final RelDataTypeField field = targetFields.get(i); - final String fieldName = field.getName(); - fieldNames.set(i, fieldName); - if (sourceExps.get(i) != null) { - if (initializerFactory.isGeneratedAlways(targetTable, i)) { - throw RESOURCE.insertIntoAlwaysGenerated(fieldName).ex(); - } - continue; - } - sourceExps.set(i, - initializerFactory.newColumnDefaultValue(targetTable, i)); - - // bare nulls are dangerous in the wrong hands - sourceExps.set(i, - castNullLiteralIfNeeded(sourceExps.get(i), field.getType())); - } - - return RelOptUtil.createProject(sourceRel, sourceExps, fieldNames, true); - } - - private InitializerExpressionFactory getInitializerFactory( - SqlValidatorTable validatorTable) { - // We might unwrap a null instead of a InitializerExpressionFactory. - final Table table = unwrap(validatorTable, Table.class); - if (table != null) { - InitializerExpressionFactory f = - unwrap(table, InitializerExpressionFactory.class); - if (f != null) { - return f; - } - } - return new NullInitializerExpressionFactory(typeFactory); - } - - private static T unwrap(Object o, Class clazz) { - if (o instanceof Wrapper) { - return ((Wrapper) o).unwrap(clazz); - } - return null; - } - - private RexNode castNullLiteralIfNeeded(RexNode node, RelDataType type) { - if (!RexLiteral.isNullLiteral(node)) { - return node; - } - return rexBuilder.makeCast(type, node); - } - - /** - * Given an INSERT statement, collects the list of names to be populated and - * the expressions to put in them. - * - * @param call Insert statement - * @param sourceRef Expression representing a row from the source - * relational expression - * @param targetColumnNames List of target column names, to be populated - * @param columnExprs List of expressions, to be populated - */ - protected void collectInsertTargets( - SqlInsert call, - final RexNode sourceRef, - final List targetColumnNames, - List columnExprs) { - final RelOptTable targetTable = getTargetTable(call); - final RelDataType tableRowType = targetTable.getRowType(); - SqlNodeList targetColumnList = call.getTargetColumnList(); - if (targetColumnList == null) { - if (validator.getConformance().isInsertSubsetColumnsAllowed()) { - final RelDataType targetRowType = - typeFactory.createStructType( - tableRowType.getFieldList() - .subList(0, sourceRef.getType().getFieldCount())); - targetColumnNames.addAll(targetRowType.getFieldNames()); - } else { - targetColumnNames.addAll(tableRowType.getFieldNames()); - } - } else { - for (int i = 0; i < targetColumnList.size(); i++) { - SqlIdentifier id = (SqlIdentifier) targetColumnList.get(i); - RelDataTypeField field = - SqlValidatorUtil.getTargetField( - tableRowType, typeFactory, id, catalogReader, targetTable); - assert field != null : "column " + id.toString() + " not found"; - targetColumnNames.add(field.getName()); - } - } - - for (int i = 0; i < targetColumnNames.size(); i++) { - final RexNode expr = rexBuilder.makeFieldAccess(sourceRef, i); - columnExprs.add(expr); - } - } - - private RelNode convertDelete(SqlDelete call) { - RelOptTable targetTable = getTargetTable(call); - RelNode sourceRel = convertSelect(call.getSourceSelect(), false); - return LogicalTableModify.create(targetTable, catalogReader, sourceRel, - LogicalTableModify.Operation.DELETE, null, null, false); - } - - private RelNode convertUpdate(SqlUpdate call) { - final SqlValidatorScope scope = validator.getWhereScope(call.getSourceSelect()); - Blackboard bb = createBlackboard(scope, null, false); - - Builder rexNodeSourceExpressionListBuilder = ImmutableList.builder(); - for (SqlNode n : call.getSourceExpressionList()) { - RexNode rn = bb.convertExpression(n); - rexNodeSourceExpressionListBuilder.add(rn); - } - - RelOptTable targetTable = getTargetTable(call); - - // convert update column list from SqlIdentifier to String - final List targetColumnNameList = new ArrayList<>(); - final RelDataType targetRowType = targetTable.getRowType(); - for (SqlNode node : call.getTargetColumnList()) { - SqlIdentifier id = (SqlIdentifier) node; - RelDataTypeField field = - SqlValidatorUtil.getTargetField( - targetRowType, typeFactory, id, catalogReader, targetTable); - assert field != null : "column " + id.toString() + " not found"; - targetColumnNameList.add(field.getName()); - } - - RelNode sourceRel = convertSelect(call.getSourceSelect(), false); - - return LogicalTableModify.create(targetTable, catalogReader, sourceRel, - LogicalTableModify.Operation.UPDATE, targetColumnNameList, - rexNodeSourceExpressionListBuilder.build(), false); - } - - private RelNode convertMerge(SqlMerge call) { - RelOptTable targetTable = getTargetTable(call); - - // convert update column list from SqlIdentifier to String - final List targetColumnNameList = new ArrayList<>(); - final RelDataType targetRowType = targetTable.getRowType(); - SqlUpdate updateCall = call.getUpdateCall(); - if (updateCall != null) { - for (SqlNode targetColumn : updateCall.getTargetColumnList()) { - SqlIdentifier id = (SqlIdentifier) targetColumn; - RelDataTypeField field = - SqlValidatorUtil.getTargetField( - targetRowType, typeFactory, id, catalogReader, targetTable); - assert field != null : "column " + id.toString() + " not found"; - targetColumnNameList.add(field.getName()); - } - } - - // replace the projection of the source select with a - // projection that contains the following: - // 1) the expressions corresponding to the new insert row (if there is - // an insert) - // 2) all columns from the target table (if there is an update) - // 3) the set expressions in the update call (if there is an update) - - // first, convert the merge's source select to construct the columns - // from the target table and the set expressions in the update call - RelNode mergeSourceRel = convertSelect(call.getSourceSelect(), false); - - // then, convert the insert statement so we can get the insert - // values expressions - SqlInsert insertCall = call.getInsertCall(); - int nLevel1Exprs = 0; - List level1InsertExprs = null; - List level2InsertExprs = null; - if (insertCall != null) { - RelNode insertRel = convertInsert(insertCall); - - // if there are 2 level of projections in the insert source, combine - // them into a single project; level1 refers to the topmost project; - // the level1 projection contains references to the level2 - // expressions, except in the case where no target expression was - // provided, in which case, the expression is the default value for - // the column; or if the expressions directly map to the source - // table - level1InsertExprs = - ((LogicalProject) insertRel.getInput(0)).getProjects(); - if (insertRel.getInput(0).getInput(0) instanceof LogicalProject) { - level2InsertExprs = - ((LogicalProject) insertRel.getInput(0).getInput(0)) - .getProjects(); - } - nLevel1Exprs = level1InsertExprs.size(); - } - - LogicalJoin join = (LogicalJoin) mergeSourceRel.getInput(0); - int nSourceFields = join.getLeft().getRowType().getFieldCount(); - final List projects = new ArrayList<>(); - for (int level1Idx = 0; level1Idx < nLevel1Exprs; level1Idx++) { - if ((level2InsertExprs != null) - && (level1InsertExprs.get(level1Idx) instanceof RexInputRef)) { - int level2Idx = - ((RexInputRef) level1InsertExprs.get(level1Idx)).getIndex(); - projects.add(level2InsertExprs.get(level2Idx)); - } else { - projects.add(level1InsertExprs.get(level1Idx)); - } - } - if (updateCall != null) { - final LogicalProject project = (LogicalProject) mergeSourceRel; - projects.addAll( - Util.skip(project.getProjects(), nSourceFields)); - } - - RelNode massagedRel = - RelOptUtil.createProject(join, projects, null, true); - - return LogicalTableModify.create(targetTable, catalogReader, massagedRel, - LogicalTableModify.Operation.MERGE, targetColumnNameList, null, false); - } - - /** - * Converts an identifier into an expression in a given scope. For example, - * the "empno" in "select empno from emp join dept" becomes "emp.empno". - */ - private RexNode convertIdentifier( - Blackboard bb, - SqlIdentifier identifier) { - // first check for reserved identifiers like CURRENT_USER - final SqlCall call = SqlUtil.makeCall(opTab, identifier); - if (call != null) { - return bb.convertExpression(call); - } - - String pv = null; - if (bb.isPatternVarRef && identifier.names.size() > 1) { - pv = identifier.names.get(0); - } - - final SqlQualified qualified; - if (bb.scope != null) { - qualified = bb.scope.fullyQualify(identifier); - } else { - qualified = SqlQualified.create(null, 1, null, identifier); - } - final Pair> e0 = bb.lookupExp(qualified); - RexNode e = e0.left; - for (String name : qualified.suffixTranslated()) { - if (e == e0.left && e0.right != null) { - int i = e0.right.get(name); - e = rexBuilder.makeFieldAccess(e, i); - } else { - final boolean caseSensitive = true; // name already fully-qualified - e = rexBuilder.makeFieldAccess(e, name, caseSensitive); - } - } - if (e instanceof RexInputRef) { - // adjust the type to account for nulls introduced by outer joins - e = adjustInputRef(bb, (RexInputRef) e); - } - - if (e0.left instanceof RexCorrelVariable) { - assert e instanceof RexFieldAccess; - final RexNode prev = - bb.mapCorrelateToRex.put(((RexCorrelVariable) e0.left).id, - (RexFieldAccess) e); - assert prev == null; - } - return e; - } - - /** - * Adjusts the type of a reference to an input field to account for nulls - * introduced by outer joins; and adjusts the offset to match the physical - * implementation. - * - * @param bb Blackboard - * @param inputRef Input ref - * @return Adjusted input ref - */ - protected RexNode adjustInputRef( - Blackboard bb, - RexInputRef inputRef) { - RelDataTypeField field = bb.getRootField(inputRef); - if (field != null) { - return rexBuilder.makeInputRef( - field.getType(), - inputRef.getIndex()); - } - return inputRef; - } - - /** - * Converts a row constructor into a relational expression. - * - * @param bb Blackboard - * @param rowConstructor Row constructor expression - * @return Relational expression which returns a single row. - */ - private RelNode convertRowConstructor( - Blackboard bb, - SqlCall rowConstructor) { - Preconditions.checkArgument(isRowConstructor(rowConstructor)); - final List operands = rowConstructor.getOperandList(); - return convertMultisets(operands, bb); - } - - private RelNode convertCursor(Blackboard bb, SubQuery subQuery) { - final SqlCall cursorCall = (SqlCall) subQuery.node; - assert cursorCall.operandCount() == 1; - SqlNode query = cursorCall.operand(0); - RelNode converted = convertQuery(query, false, false).rel; - int iCursor = bb.cursors.size(); - bb.cursors.add(converted); - subQuery.expr = - new RexInputRef( - iCursor, - converted.getRowType()); - return converted; - } - - private RelNode convertMultisets(final List operands, - Blackboard bb) { - // NOTE: Wael 2/04/05: this implementation is not the most efficient in - // terms of planning since it generates XOs that can be reduced. - final List joinList = new ArrayList<>(); - List lastList = new ArrayList<>(); - for (int i = 0; i < operands.size(); i++) { - SqlNode operand = operands.get(i); - if (!(operand instanceof SqlCall)) { - lastList.add(operand); - continue; - } - - final SqlCall call = (SqlCall) operand; - final RelNode input; - switch (call.getKind()) { - case MULTISET_VALUE_CONSTRUCTOR: - case ARRAY_VALUE_CONSTRUCTOR: - final SqlNodeList list = - new SqlNodeList(call.getOperandList(), call.getParserPosition()); - CollectNamespace nss = - (CollectNamespace) validator.getNamespace(call); - Blackboard usedBb; - if (null != nss) { - usedBb = createBlackboard(nss.getScope(), null, false); - } else { - usedBb = - createBlackboard(new ListScope(bb.scope) { - public SqlNode getNode() { - return call; - } - }, null, false); - } - RelDataType multisetType = validator.getValidatedNodeType(call); - ((SqlValidatorImpl) validator).setValidatedNodeType(list, - multisetType.getComponentType()); - input = convertQueryOrInList(usedBb, list, null); - break; - case MULTISET_QUERY_CONSTRUCTOR: - case ARRAY_QUERY_CONSTRUCTOR: - final RelRoot root = convertQuery(call.operand(0), false, true); - input = root.rel; - break; - default: - lastList.add(operand); - continue; - } - - if (lastList.size() > 0) { - joinList.add(lastList); - } - lastList = new ArrayList<>(); - Collect collect = - new Collect( - cluster, - cluster.traitSetOf(Convention.NONE), - input, - validator.deriveAlias(call, i)); - joinList.add(collect); - } - - if (joinList.size() == 0) { - joinList.add(lastList); - } - - for (int i = 0; i < joinList.size(); i++) { - Object o = joinList.get(i); - if (o instanceof List) { - @SuppressWarnings("unchecked") - List projectList = (List) o; - final List selectList = new ArrayList<>(); - final List fieldNameList = new ArrayList<>(); - for (int j = 0; j < projectList.size(); j++) { - SqlNode operand = projectList.get(j); - selectList.add(bb.convertExpression(operand)); - - // REVIEW angel 5-June-2005: Use deriveAliasFromOrdinal - // instead of deriveAlias to match field names from - // SqlRowOperator. Otherwise, get error Type - // 'RecordType(INTEGER EMPNO)' has no field 'EXPR$0' when - // doing select * from unnest( select multiset[empno] - // from sales.emps); - - fieldNameList.add(SqlUtil.deriveAliasFromOrdinal(j)); - } - - RelNode projRel = - RelOptUtil.createProject( - LogicalValues.createOneRow(cluster), - selectList, - fieldNameList); - - joinList.set(i, projRel); - } - } - - RelNode ret = (RelNode) joinList.get(0); - for (int i = 1; i < joinList.size(); i++) { - RelNode relNode = (RelNode) joinList.get(i); - ret = - RelFactories.DEFAULT_JOIN_FACTORY.createJoin( - ret, - relNode, - rexBuilder.makeLiteral(true), - ImmutableSet.of(), - JoinRelType.INNER, - false); - } - return ret; - } - - private void convertSelectList( - Blackboard bb, - SqlSelect select, - List orderList) { - SqlNodeList selectList = select.getSelectList(); - selectList = validator.expandStar(selectList, select, false); - - replaceSubQueries(bb, selectList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - - List fieldNames = new ArrayList<>(); - final List exprs = new ArrayList<>(); - final Collection aliases = new TreeSet<>(); - - // Project any system fields. (Must be done before regular select items, - // because offsets may be affected.) - final List columnMonotonicityList = new ArrayList<>(); - extraSelectItems( - bb, - select, - exprs, - fieldNames, - aliases, - columnMonotonicityList); - - // Project select clause. - int i = -1; - for (SqlNode expr : selectList) { - ++i; - exprs.add(bb.convertExpression(expr)); - fieldNames.add(deriveAlias(expr, aliases, i)); - } - - // Project extra fields for sorting. - for (SqlNode expr : orderList) { - ++i; - SqlNode expr2 = validator.expandOrderExpr(select, expr); - exprs.add(bb.convertExpression(expr2)); - fieldNames.add(deriveAlias(expr, aliases, i)); - } - - fieldNames = SqlValidatorUtil.uniquify(fieldNames, - catalogReader.nameMatcher().isCaseSensitive()); - - bb.setRoot( - RelOptUtil.createProject(bb.root, exprs, fieldNames), - false); - - assert bb.columnMonotonicities.isEmpty(); - bb.columnMonotonicities.addAll(columnMonotonicityList); - for (SqlNode selectItem : selectList) { - bb.columnMonotonicities.add( - selectItem.getMonotonicity(bb.scope)); - } - } - - /** - * Adds extra select items. The default implementation adds nothing; derived - * classes may add columns to exprList, nameList, aliasList and - * columnMonotonicityList. - * - * @param bb Blackboard - * @param select Select statement being translated - * @param exprList List of expressions in select clause - * @param nameList List of names, one per column - * @param aliasList Collection of aliases that have been used - * already - * @param columnMonotonicityList List of monotonicity, one per column - */ - protected void extraSelectItems( - Blackboard bb, - SqlSelect select, - List exprList, - List nameList, - Collection aliasList, - List columnMonotonicityList) { - } - - private String deriveAlias( - final SqlNode node, - Collection aliases, - final int ordinal) { - String alias = validator.deriveAlias(node, ordinal); - if ((alias == null) || aliases.contains(alias)) { - String aliasBase = (alias == null) ? "EXPR$" : alias; - for (int j = 0;; j++) { - alias = aliasBase + j; - if (!aliases.contains(alias)) { - break; - } - } - } - aliases.add(alias); - return alias; - } - - /** - * Converts a WITH sub-query into a relational expression. - */ - public RelRoot convertWith(SqlWith with, boolean top) { - return convertQuery(with.body, false, top); - } - - /** - * Converts a SELECT statement's parse tree into a relational expression. - */ - public RelNode convertValues( - SqlCall values, - RelDataType targetRowType) { - final SqlValidatorScope scope = validator.getOverScope(values); - assert scope != null; - final Blackboard bb = createBlackboard(scope, null, false); - convertValuesImpl(bb, values, targetRowType); - return bb.root; - } - - /** - * Converts a values clause (as in "INSERT INTO T(x,y) VALUES (1,2)") into a - * relational expression. - * - * @param bb Blackboard - * @param values Call to SQL VALUES operator - * @param targetRowType Target row type - */ - private void convertValuesImpl( - Blackboard bb, - SqlCall values, - RelDataType targetRowType) { - // Attempt direct conversion to LogicalValues; if that fails, deal with - // fancy stuff like sub-queries below. - RelNode valuesRel = - convertRowValues( - bb, - values, - values.getOperandList(), - true, - targetRowType); - if (valuesRel != null) { - bb.setRoot(valuesRel, true); - return; - } - - final List unionRels = new ArrayList<>(); - for (SqlNode rowConstructor1 : values.getOperandList()) { - SqlCall rowConstructor = (SqlCall) rowConstructor1; - Blackboard tmpBb = createBlackboard(bb.scope, null, false); - replaceSubQueries(tmpBb, rowConstructor, - RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - final List> exps = new ArrayList<>(); - for (Ord operand : Ord.zip(rowConstructor.getOperandList())) { - exps.add( - Pair.of( - tmpBb.convertExpression(operand.e), - validator.deriveAlias(operand.e, operand.i))); - } - RelNode in = - (null == tmpBb.root) - ? LogicalValues.createOneRow(cluster) - : tmpBb.root; - unionRels.add( - RelOptUtil.createProject( - in, - Pair.left(exps), - Pair.right(exps), - true)); - } - - if (unionRels.size() == 0) { - throw new AssertionError("empty values clause"); - } else if (unionRels.size() == 1) { - bb.setRoot( - unionRels.get(0), - true); - } else { - bb.setRoot( - LogicalUnion.create(unionRels, true), - true); - } - - // REVIEW jvs 22-Jan-2004: should I add - // mapScopeToLux.put(validator.getScope(values),bb.root); - // ? - } - - //~ Inner Classes ---------------------------------------------------------- - - /** - * Workspace for translating an individual SELECT statement (or sub-SELECT). - */ - protected class Blackboard implements SqlRexContext, SqlVisitor { - /** - * Collection of {@link RelNode} objects which correspond to a SELECT - * statement. - */ - public final SqlValidatorScope scope; - private final Map nameToNodeMap; - public RelNode root; - private List inputs; - private final Map mapCorrelateToRex = - new HashMap<>(); - - private boolean isPatternVarRef = false; - - final List cursors = new ArrayList<>(); - - /** - * List of IN and EXISTS nodes inside this - * SELECT statement (but not inside sub-queries). - */ - private final Set subQueryList = new LinkedHashSet<>(); - - /** - * Workspace for building aggregates. - */ - AggConverter agg; - - /** - * When converting window aggregate, we need to know if the window is - * guaranteed to be non-empty. - */ - SqlWindow window; - - /** - * Project the groupby expressions out of the root of this sub-select. - * Sub-queries can reference group by expressions projected from the - * "right" to the sub-query. - */ - private final Map> - mapRootRelToFieldProjection = new HashMap<>(); - - private final List columnMonotonicities = - new ArrayList<>(); - - private final List systemFieldList = new ArrayList<>(); - final boolean top; - - private final InitializerExpressionFactory initializerExpressionFactory = - new NullInitializerExpressionFactory(typeFactory); - - /** - * Creates a Blackboard. - * - * @param scope Name-resolution scope for expressions validated - * within this query. Can be null if this Blackboard is - * for a leaf node, say - * @param nameToNodeMap Map which translates the expression to map a - * given parameter into, if translating expressions; - * null otherwise - * @param top Whether this is the root of the query - */ - protected Blackboard(SqlValidatorScope scope, - Map nameToNodeMap, boolean top) { - this.scope = scope; - this.nameToNodeMap = nameToNodeMap; - this.top = top; - } - - public void setPatternVarRef(boolean isVarRef) { - this.isPatternVarRef = isVarRef; - } - - public RexNode register( - RelNode rel, - JoinRelType joinType) { - return register(rel, joinType, null); - } - - /** - * Registers a relational expression. - * - * @param rel Relational expression - * @param joinType Join type - * @param leftKeys LHS of IN clause, or null for expressions - * other than IN - * @return Expression with which to refer to the row (or partial row) - * coming from this relational expression's side of the join - */ - public RexNode register( - RelNode rel, - JoinRelType joinType, - List leftKeys) { - assert joinType != null; - if (root == null) { - assert leftKeys == null; - setRoot(rel, false); - return rexBuilder.makeRangeReference( - root.getRowType(), - 0, - false); - } - - final RexNode joinCond; - final int origLeftInputCount = root.getRowType().getFieldCount(); - if (leftKeys != null) { - List newLeftInputExpr = Lists.newArrayList(); - for (int i = 0; i < origLeftInputCount; i++) { - newLeftInputExpr.add(rexBuilder.makeInputRef(root, i)); - } - - final List leftJoinKeys = Lists.newArrayList(); - for (RexNode leftKey : leftKeys) { - int index = newLeftInputExpr.indexOf(leftKey); - if (index < 0 || joinType == JoinRelType.LEFT) { - index = newLeftInputExpr.size(); - newLeftInputExpr.add(leftKey); - } - leftJoinKeys.add(index); - } - - RelNode newLeftInput = - RelOptUtil.createProject( - root, - newLeftInputExpr, - null, - true); - - // maintain the group by mapping in the new LogicalProject - if (mapRootRelToFieldProjection.containsKey(root)) { - mapRootRelToFieldProjection.put( - newLeftInput, - mapRootRelToFieldProjection.get(root)); - } - - setRoot(newLeftInput, false); - - // right fields appear after the LHS fields. - final int rightOffset = root.getRowType().getFieldCount() - - newLeftInput.getRowType().getFieldCount(); - final List rightKeys = - Util.range(rightOffset, rightOffset + leftKeys.size()); - - joinCond = - RelOptUtil.createEquiJoinCondition(newLeftInput, leftJoinKeys, - rel, rightKeys, rexBuilder); - } else { - joinCond = rexBuilder.makeLiteral(true); - } - - int leftFieldCount = root.getRowType().getFieldCount(); - final RelNode join = - createJoin( - this, - root, - rel, - joinCond, - joinType); - - setRoot(join, false); - - if (leftKeys != null - && joinType == JoinRelType.LEFT) { - final int leftKeyCount = leftKeys.size(); - int rightFieldLength = rel.getRowType().getFieldCount(); - assert leftKeyCount == rightFieldLength - 1; - - final int rexRangeRefLength = leftKeyCount + rightFieldLength; - RelDataType returnType = - typeFactory.createStructType( - new AbstractList>() { - public Map.Entry get( - int index) { - return join.getRowType().getFieldList() - .get(origLeftInputCount + index); - } - - public int size() { - return rexRangeRefLength; - } - }); - - return rexBuilder.makeRangeReference( - returnType, - origLeftInputCount, - false); - } else { - return rexBuilder.makeRangeReference( - rel.getRowType(), - leftFieldCount, - joinType.generatesNullsOnRight()); - } - } - - /** - * Sets a new root relational expression, as the translation process - * backs its way further up the tree. - * - * @param root New root relational expression - * @param leaf Whether the relational expression is a leaf, that is, - * derived from an atomic relational expression such as a table - * name in the from clause, or the projection on top of a - * select-sub-query. In particular, relational expressions - * derived from JOIN operators are not leaves, but set - * expressions are. - */ - public void setRoot(RelNode root, boolean leaf) { - setRoot( - Collections.singletonList(root), root, root instanceof LogicalJoin); - if (leaf) { - leaves.add(root); - } - this.columnMonotonicities.clear(); - } - - private void setRoot( - List inputs, - RelNode root, - boolean hasSystemFields) { - this.inputs = inputs; - this.root = root; - this.systemFieldList.clear(); - if (hasSystemFields) { - this.systemFieldList.addAll(getSystemFields()); - } - } - - /** - * Notifies this Blackboard that the root just set using - * {@link #setRoot(RelNode, boolean)} was derived using dataset - * substitution. - * - *

The default implementation is not interested in such - * notifications, and does nothing. - * - * @param datasetName Dataset name - */ - public void setDataset(String datasetName) { - } - - void setRoot(List inputs) { - setRoot(inputs, null, false); - } - - /** - * Returns an expression with which to reference a from-list item. - * - * @param qualified the alias of the from item - * @return a {@link RexFieldAccess} or {@link RexRangeRef}, or null if - * not found - */ - Pair> lookupExp(SqlQualified qualified) { - if (nameToNodeMap != null && qualified.prefixLength == 1) { - RexNode node = nameToNodeMap.get(qualified.identifier.names.get(0)); - if (node == null) { - throw new AssertionError("Unknown identifier '" + qualified.identifier - + "' encountered while expanding expression"); - } - return Pair.of(node, null); - } - final SqlNameMatcher nameMatcher = - scope.getValidator().getCatalogReader().nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolve(qualified.prefix(), nameMatcher, false, resolved); - if (!(resolved.count() == 1)) { - return null; - } - final SqlValidatorScope.Resolve resolve = resolved.only(); - final RelDataType rowType = resolve.rowType(); - - // Found in current query's from list. Find which from item. - // We assume that the order of the from clause items has been - // preserved. - final SqlValidatorScope ancestorScope = resolve.scope; - boolean isParent = ancestorScope != scope; - if ((inputs != null) && !isParent) { - final LookupContext rels = - new LookupContext(this, inputs, systemFieldList.size()); - final RexNode node = lookup(resolve.path.steps().get(0).i, rels); - if (node == null) { - return null; - } else { - return Pair.of(node, null); - } - } else { - // We're referencing a relational expression which has not been - // converted yet. This occurs when from items are correlated, - // e.g. "select from emp as emp join emp.getDepts() as dept". - // Create a temporary expression. - DeferredLookup lookup = - new DeferredLookup(this, qualified.identifier.names.get(0)); - final CorrelationId correlId = cluster.createCorrel(); - mapCorrelToDeferred.put(correlId, lookup); - if (resolve.path.steps().get(0).i < 0) { - return Pair.of(rexBuilder.makeCorrel(rowType, correlId), null); - } else { - final RelDataTypeFactory.FieldInfoBuilder builder = - typeFactory.builder(); - final ListScope ancestorScope1 = (ListScope) resolve.scope; - final ImmutableMap.Builder fields = - ImmutableMap.builder(); - int i = 0; - int offset = 0; - for (SqlValidatorNamespace c : ancestorScope1.getChildren()) { - builder.addAll(c.getRowType().getFieldList()); - if (i == resolve.path.steps().get(0).i) { - for (RelDataTypeField field : c.getRowType().getFieldList()) { - fields.put(c.translate(field.getName()), - field.getIndex() + offset); - } - } - ++i; - offset += c.getRowType().getFieldCount(); - } - final RexNode c = - rexBuilder.makeCorrel(builder.uniquify().build(), correlId); - return Pair.>of(c, fields.build()); - } - } - } - - /** - * Creates an expression with which to reference the expression whose - * offset in its from-list is {@code offset}. - */ - RexNode lookup( - int offset, - LookupContext lookupContext) { - Pair pair = lookupContext.findRel(offset); - return rexBuilder.makeRangeReference( - pair.left.getRowType(), - pair.right, - false); - } - - RelDataTypeField getRootField(RexInputRef inputRef) { - int fieldOffset = inputRef.getIndex(); - for (RelNode input : inputs) { - RelDataType rowType = input.getRowType(); - if (rowType == null) { - // TODO: remove this once leastRestrictive - // is correctly implemented - return null; - } - if (fieldOffset < rowType.getFieldCount()) { - return rowType.getFieldList().get(fieldOffset); - } - fieldOffset -= rowType.getFieldCount(); - } - throw new AssertionError(); - } - - public void flatten( - List rels, - int systemFieldCount, - int[] start, - List> relOffsetList) { - for (RelNode rel : rels) { - if (leaves.contains(rel)) { - relOffsetList.add( - Pair.of(rel, start[0])); - start[0] += rel.getRowType().getFieldCount(); - } else { - if (rel instanceof LogicalJoin - || rel instanceof LogicalAggregate) { - start[0] += systemFieldCount; - } - flatten( - rel.getInputs(), - systemFieldCount, - start, - relOffsetList); - } - } - } - - void registerSubQuery(SqlNode node, RelOptUtil.Logic logic) { - for (SubQuery subQuery : subQueryList) { - if (node.equalsDeep(subQuery.node, Litmus.IGNORE)) { - return; - } - } - subQueryList.add(new SubQuery(node, logic)); - } - - SubQuery getSubQuery(SqlNode expr) { - for (SubQuery subQuery : subQueryList) { - if (expr.equalsDeep(subQuery.node, Litmus.IGNORE)) { - return subQuery; - } - } - - return null; - } - - ImmutableList retrieveCursors() { - try { - return ImmutableList.copyOf(cursors); - } finally { - cursors.clear(); - } - } - - public RexNode convertExpression(SqlNode expr) { - // If we're in aggregation mode and this is an expression in the - // GROUP BY clause, return a reference to the field. - if (agg != null) { - final SqlNode expandedGroupExpr = validator.expand(expr, scope); - final int ref = agg.lookupGroupExpr(expandedGroupExpr); - if (ref >= 0) { - return rexBuilder.makeInputRef(root, ref); - } - if (expr instanceof SqlCall) { - final RexNode rex = agg.lookupAggregates((SqlCall) expr); - if (rex != null) { - return rex; - } - } - } - - // Allow the derived class chance to override the standard - // behavior for special kinds of expressions. - RexNode rex = convertExtendedExpression(expr, this); - if (rex != null) { - return rex; - } - - // Sub-queries and OVER expressions are not like ordinary - // expressions. - final SqlKind kind = expr.getKind(); - final SubQuery subQuery; - if (!config.isExpand()) { - final SqlCall call; - final SqlNode query; - final RelRoot root; - switch (kind) { - case IN: - call = (SqlCall) expr; - query = call.operand(1); - if (!(query instanceof SqlNodeList)) { - final SqlInOperator op = (SqlInOperator) call.getOperator(); - root = convertQueryRecursive(query, false, null); - final SqlNode operand = call.operand(0); - List nodes; - switch (operand.getKind()) { - case ROW: - nodes = ((SqlCall) operand).getOperandList(); - break; - default: - nodes = ImmutableList.of(operand); - } - final ImmutableList.Builder builder = - ImmutableList.builder(); - for (SqlNode node : nodes) { - builder.add(convertExpression(node)); - } - final RexSubQuery in = RexSubQuery.in(root.rel, builder.build()); - return op.isNotIn() - ? rexBuilder.makeCall(SqlStdOperatorTable.NOT, in) - : in; - } - break; - - case EXISTS: - call = (SqlCall) expr; - query = Iterables.getOnlyElement(call.getOperandList()); - root = convertQueryRecursive(query, false, null); - RelNode rel = root.rel; - while (rel instanceof Project - || rel instanceof Sort - && ((Sort) rel).fetch == null - && ((Sort) rel).offset == null) { - rel = ((SingleRel) rel).getInput(); - } - return RexSubQuery.exists(rel); - - case SCALAR_QUERY: - call = (SqlCall) expr; - query = Iterables.getOnlyElement(call.getOperandList()); - root = convertQueryRecursive(query, false, null); - return RexSubQuery.scalar(root.rel); - } - } - - switch (kind) { - case CURSOR: - case IN: - subQuery = Preconditions.checkNotNull(getSubQuery(expr)); - rex = Preconditions.checkNotNull(subQuery.expr); - return StandardConvertletTable.castToValidatedType(expr, rex, - validator, rexBuilder); - - case SELECT: - case EXISTS: - case SCALAR_QUERY: - subQuery = getSubQuery(expr); - assert subQuery != null; - rex = subQuery.expr; - assert rex != null : "rex != null"; - - if (((kind == SqlKind.SCALAR_QUERY) - || (kind == SqlKind.EXISTS)) - && isConvertedSubq(rex)) { - // scalar sub-query or EXISTS has been converted to a - // constant - return rex; - } - - // The indicator column is the last field of the sub-query. - RexNode fieldAccess = - rexBuilder.makeFieldAccess( - rex, - rex.getType().getFieldCount() - 1); - - // The indicator column will be nullable if it comes from - // the null-generating side of the join. For EXISTS, add an - // "IS TRUE" check so that the result is "BOOLEAN NOT NULL". - if (fieldAccess.getType().isNullable() - && kind == SqlKind.EXISTS) { - fieldAccess = - rexBuilder.makeCall( - SqlStdOperatorTable.IS_NOT_NULL, - fieldAccess); - } - return fieldAccess; - - case OVER: - return convertOver(this, expr); - - default: - // fall through - } - - // Apply standard conversions. - rex = expr.accept(this); - return Preconditions.checkNotNull(rex); - } - - /** - * Converts an item in an ORDER BY clause, extracting DESC, NULLS LAST - * and NULLS FIRST flags first. - */ - public RexNode convertSortExpression(SqlNode expr, Set flags) { - switch (expr.getKind()) { - case DESCENDING: - case NULLS_LAST: - case NULLS_FIRST: - flags.add(expr.getKind()); - final SqlNode operand = ((SqlCall) expr).operand(0); - return convertSortExpression(operand, flags); - default: - return convertExpression(expr); - } - } - - /** - * Determines whether a RexNode corresponds to a sub-query that's been - * converted to a constant. - * - * @param rex the expression to be examined - * @return true if the expression is a dynamic parameter, a literal, or - * a literal that is being cast - */ - private boolean isConvertedSubq(RexNode rex) { - if ((rex instanceof RexLiteral) - || (rex instanceof RexDynamicParam)) { - return true; - } - if (rex instanceof RexCall) { - RexCall call = (RexCall) rex; - if (call.getOperator() == SqlStdOperatorTable.CAST) { - RexNode operand = call.getOperands().get(0); - if (operand instanceof RexLiteral) { - return true; - } - } - } - return false; - } - - public int getGroupCount() { - if (agg != null) { - return agg.groupExprs.size(); - } - if (window != null) { - return window.isAlwaysNonEmpty() ? 1 : 0; - } - return -1; - } - - public RexBuilder getRexBuilder() { - return rexBuilder; - } - - public RexRangeRef getSubQueryExpr(SqlCall call) { - final SubQuery subQuery = getSubQuery(call); - assert subQuery != null; - return (RexRangeRef) subQuery.expr; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - - public InitializerExpressionFactory getInitializerExpressionFactory() { - return initializerExpressionFactory; - } - - public SqlValidator getValidator() { - return validator; - } - - public RexNode convertLiteral(SqlLiteral literal) { - return exprConverter.convertLiteral(this, literal); - } - - public RexNode convertInterval(SqlIntervalQualifier intervalQualifier) { - return exprConverter.convertInterval(this, intervalQualifier); - } - - public RexNode visit(SqlLiteral literal) { - return exprConverter.convertLiteral(this, literal); - } - - public RexNode visit(SqlCall call) { - if (agg != null) { - final SqlOperator op = call.getOperator(); - if (window == null - && (op.isAggregator() || op.getKind() == SqlKind.FILTER)) { - return agg.lookupAggregates(call); - } - } - return exprConverter.convertCall(this, - new SqlCallBinding(validator, scope, call).permutedCall()); - } - - public RexNode visit(SqlNodeList nodeList) { - throw new UnsupportedOperationException(); - } - - public RexNode visit(SqlIdentifier id) { - return convertIdentifier(this, id); - } - - public RexNode visit(SqlDataTypeSpec type) { - throw new UnsupportedOperationException(); - } - - public RexNode visit(SqlDynamicParam param) { - return convertDynamicParam(param); - } - - public RexNode visit(SqlIntervalQualifier intervalQualifier) { - return convertInterval(intervalQualifier); - } - - public List getColumnMonotonicities() { - return columnMonotonicities; - } - - } - - /** Deferred lookup. */ - private static class DeferredLookup { - Blackboard bb; - String originalRelName; - - DeferredLookup( - Blackboard bb, - String originalRelName) { - this.bb = bb; - this.originalRelName = originalRelName; - } - - public RexFieldAccess getFieldAccess(CorrelationId name) { - return (RexFieldAccess) bb.mapCorrelateToRex.get(name); - } - - public String getOriginalRelName() { - return originalRelName; - } - } - - /** - * A default implementation of SubQueryConverter that does no conversion. - */ - private class NoOpSubQueryConverter implements SubQueryConverter { - public boolean canConvertSubQuery() { - return false; - } - - public RexNode convertSubQuery( - SqlCall subQuery, - SqlToRelConverter parentConverter, - boolean isExists, - boolean isExplain) { - throw new IllegalArgumentException(); - } - } - - /** - * Converts expressions to aggregates. - * - *

Consider the expression - * - *

- * {@code SELECT deptno, SUM(2 * sal) FROM emp GROUP BY deptno} - *
- * - *

Then: - * - *

    - *
  • groupExprs = {SqlIdentifier(deptno)}
  • - *
  • convertedInputExprs = {RexInputRef(deptno), 2 * - * RefInputRef(sal)}
  • - *
  • inputRefs = {RefInputRef(#0), RexInputRef(#1)}
  • - *
  • aggCalls = {AggCall(SUM, {1})}
  • - *
- */ - protected class AggConverter implements SqlVisitor { - private final Blackboard bb; - public final AggregatingSelectScope aggregatingSelectScope; - - private final Map nameMap = Maps.newHashMap(); - - /** - * The group-by expressions, in {@link SqlNode} format. - */ - private final SqlNodeList groupExprs = - new SqlNodeList(SqlParserPos.ZERO); - - /** - * The auxiliary group-by expressions. - */ - private final Map> auxiliaryGroupExprs = - new HashMap<>(); - - /** - * Input expressions for the group columns and aggregates, in - * {@link RexNode} format. The first elements of the list correspond to the - * elements in {@link #groupExprs}; the remaining elements are for - * aggregates. The right field of each pair is the name of the expression, - * where the expressions are simple mappings to input fields. - */ - private final List> convertedInputExprs = - new ArrayList<>(); - - /** Expressions to be evaluated as rows are being placed into the - * aggregate's hash table. This is when group functions such as TUMBLE - * cause rows to be expanded. */ - private final List midExprs = new ArrayList<>(); - - private final List aggCalls = new ArrayList<>(); - private final Map aggMapping = new HashMap<>(); - private final Map aggCallMapping = - new HashMap<>(); - - /** Are we directly inside a windowed aggregate? */ - private boolean inOver = false; - - /** - * Creates an AggConverter. - * - *

The select parameter provides enough context to name - * aggregate calls which are top-level select list items. - * - * @param bb Blackboard - * @param select Query being translated; provides context to give - */ - public AggConverter(Blackboard bb, SqlSelect select) { - this.bb = bb; - this.aggregatingSelectScope = - (AggregatingSelectScope) bb.getValidator().getSelectScope(select); - - // Collect all expressions used in the select list so that aggregate - // calls can be named correctly. - final SqlNodeList selectList = select.getSelectList(); - for (int i = 0; i < selectList.size(); i++) { - SqlNode selectItem = selectList.get(i); - String name = null; - if (SqlUtil.isCallTo( - selectItem, - SqlStdOperatorTable.AS)) { - final SqlCall call = (SqlCall) selectItem; - selectItem = call.operand(0); - name = call.operand(1).toString(); - } - if (name == null) { - name = validator.deriveAlias(selectItem, i); - } - nameMap.put(selectItem.toString(), name); - } - } - - public int addGroupExpr(SqlNode expr) { - int ref = lookupGroupExpr(expr); - if (ref >= 0) { - return ref; - } - final int index = groupExprs.size(); - groupExprs.add(expr); - String name = nameMap.get(expr.toString()); - RexNode convExpr = bb.convertExpression(expr); - addExpr(convExpr, name); - - if (expr instanceof SqlCall) { - SqlCall call = (SqlCall) expr; - for (Pair p - : SqlStdOperatorTable.convertGroupToAuxiliaryCalls(call)) { - addAuxiliaryGroupExpr(p.left, index, p.right); - } - } - - return index; - } - - void addAuxiliaryGroupExpr(SqlNode node, int index, - AuxiliaryConverter converter) { - for (SqlNode node2 : auxiliaryGroupExprs.keySet()) { - if (node2.equalsDeep(node, Litmus.IGNORE)) { - return; - } - } - auxiliaryGroupExprs.put(node, Ord.of(index, converter)); - } - - /** - * Adds an expression, deducing an appropriate name if possible. - * - * @param expr Expression - * @param name Suggested name - */ - private void addExpr(RexNode expr, String name) { - if ((name == null) && (expr instanceof RexInputRef)) { - final int i = ((RexInputRef) expr).getIndex(); - name = bb.root.getRowType().getFieldList().get(i).getName(); - } - if (Pair.right(convertedInputExprs).contains(name)) { - // In case like 'SELECT ... GROUP BY x, y, x', don't add - // name 'x' twice. - name = null; - } - convertedInputExprs.add(Pair.of(expr, name)); - } - - public Void visit(SqlIdentifier id) { - return null; - } - - public Void visit(SqlNodeList nodeList) { - for (int i = 0; i < nodeList.size(); i++) { - nodeList.get(i).accept(this); - } - return null; - } - - public Void visit(SqlLiteral lit) { - return null; - } - - public Void visit(SqlDataTypeSpec type) { - return null; - } - - public Void visit(SqlDynamicParam param) { - return null; - } - - public Void visit(SqlIntervalQualifier intervalQualifier) { - return null; - } - - public Void visit(SqlCall call) { - switch (call.getKind()) { - case FILTER: - translateAgg((SqlCall) call.operand(0), call.operand(1), call); - return null; - case SELECT: - // rchen 2006-10-17: - // for now do not detect aggregates in sub-queries. - return null; - } - final boolean prevInOver = inOver; - // Ignore window aggregates and ranking functions (associated with OVER - // operator). However, do not ignore nested window aggregates. - if (call.getOperator().getKind() == SqlKind.OVER) { - // Track aggregate nesting levels only within an OVER operator. - inOver = true; - } - - // Do not translate the top level window aggregate. Only do so for - // nested aggregates, if present - if (call.getOperator().isAggregator()) { - if (inOver) { - // Add the parent aggregate level before visiting its children - inOver = false; - } else { - // We're beyond the one ignored level - translateAgg(call, null, call); - return null; - } - } - for (SqlNode operand : call.getOperandList()) { - // Operands are occasionally null, e.g. switched CASE arg 0. - if (operand != null) { - operand.accept(this); - } - } - // Remove the parent aggregate level after visiting its children - inOver = prevInOver; - return null; - } - - private void translateAgg(SqlCall call, SqlNode filter, SqlCall outerCall) { - assert bb.agg == this; - final List args = new ArrayList<>(); - int filterArg = -1; - final List argTypes = - call.getOperator() instanceof SqlCountAggFunction - ? new ArrayList(call.getOperandList().size()) - : null; - try { - // switch out of agg mode - bb.agg = null; - for (SqlNode operand : call.getOperandList()) { - - // special case for COUNT(*): delete the * - if (operand instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) operand; - if (id.isStar()) { - assert call.operandCount() == 1; - assert args.isEmpty(); - break; - } - } - RexNode convertedExpr = bb.convertExpression(operand); - assert convertedExpr != null; - if (argTypes != null) { - argTypes.add(convertedExpr.getType()); - } - args.add(lookupOrCreateGroupExpr(convertedExpr)); - } - - if (filter != null) { - RexNode convertedExpr = bb.convertExpression(filter); - assert convertedExpr != null; - if (convertedExpr.getType().isNullable()) { - convertedExpr = - rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr); - } - filterArg = lookupOrCreateGroupExpr(convertedExpr); - } - } finally { - // switch back into agg mode - bb.agg = this; - } - - final SqlAggFunction aggFunction = - (SqlAggFunction) call.getOperator(); - RelDataType type = validator.deriveType(bb.scope, call); - boolean distinct = false; - SqlLiteral quantifier = call.getFunctionQuantifier(); - if ((null != quantifier) - && (quantifier.getValue() == SqlSelectKeyword.DISTINCT)) { - distinct = true; - } - final AggregateCall aggCall = - AggregateCall.create( - aggFunction, - distinct, - args, - filterArg, - type, - nameMap.get(outerCall.toString())); - final AggregatingSelectScope.Resolved r = - aggregatingSelectScope.resolved.get(); - RexNode rex = - rexBuilder.addAggCall( - aggCall, - groupExprs.size(), - r.indicator, - aggCalls, - aggCallMapping, - argTypes); - aggMapping.put(outerCall, rex); - } - - private int lookupOrCreateGroupExpr(RexNode expr) { - int index = 0; - for (RexNode convertedInputExpr : Pair.left(convertedInputExprs)) { - if (expr.toString().equals(convertedInputExpr.toString())) { - return index; - } - ++index; - } - - // not found -- add it - addExpr(expr, null); - return index; - } - - /** - * If an expression is structurally identical to one of the group-by - * expressions, returns a reference to the expression, otherwise returns - * null. - */ - public int lookupGroupExpr(SqlNode expr) { - for (int i = 0; i < groupExprs.size(); i++) { - SqlNode groupExpr = groupExprs.get(i); - if (expr.equalsDeep(groupExpr, Litmus.IGNORE)) { - return i; - } - } - return -1; - } - - public RexNode lookupAggregates(SqlCall call) { - // assert call.getOperator().isAggregator(); - assert bb.agg == this; - - switch (call.getKind()) { - case GROUPING: - case GROUP_ID: - final RelDataType type = validator.getValidatedNodeType(call); - if (!aggregatingSelectScope.resolved.get().indicator) { - return rexBuilder.makeExactLiteral( - TWO.pow(effectiveArgCount(call)).subtract(BigDecimal.ONE), type); - } else { - final List operands; - switch (call.getKind()) { - case GROUP_ID: - operands = ImmutableIntList.range(0, groupExprs.size()); - break; - default: - operands = Lists.newArrayList(); - for (SqlNode operand : call.getOperandList()) { - final int x = lookupGroupExpr(operand); - assert x >= 0; - operands.add(x); - } - } - RexNode node = null; - int shift = operands.size(); - for (int operand : operands) { - node = bitValue(node, type, operand, --shift); - } - return node; - } - } - - for (Map.Entry> e - : auxiliaryGroupExprs.entrySet()) { - if (call.equalsDeep(e.getKey(), Litmus.IGNORE)) { - AuxiliaryConverter converter = e.getValue().e; - final int groupOrdinal = e.getValue().i; - return converter.convert(rexBuilder, - convertedInputExprs.get(groupOrdinal).left, - rexBuilder.makeInputRef(bb.root, groupOrdinal)); - } - } - - return aggMapping.get(call); - } - - private int effectiveArgCount(SqlCall call) { - switch (call.getKind()) { - case GROUPING: - return call.operandCount(); - case GROUP_ID: - return groupExprs.size(); - default: - throw new AssertionError(call.getKind()); - } - } - - private RexNode bitValue(RexNode previous, RelDataType type, int x, - int shift) { - final AggregatingSelectScope.Resolved r = - aggregatingSelectScope.resolved.get(); - RexNode node = rexBuilder.makeCall(SqlStdOperatorTable.CASE, - rexBuilder.makeInputRef(bb.root, r.groupExprList.size() + x), - rexBuilder.makeExactLiteral(BigDecimal.ONE, type), - rexBuilder.makeExactLiteral(BigDecimal.ZERO, type)); - if (shift > 0) { - node = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, node, - rexBuilder.makeExactLiteral(TWO.pow(shift), type)); - } - if (previous != null) { - node = rexBuilder.makeCall(SqlStdOperatorTable.PLUS, previous, node); - } - return node; - } - - public List> getPreExprs() { - return convertedInputExprs; - } - - public List getAggCalls() { - return aggCalls; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - } - - /** - * Context to find a relational expression to a field offset. - */ - private static class LookupContext { - private final List> relOffsetList = - new ArrayList<>(); - - /** - * Creates a LookupContext with multiple input relational expressions. - * - * @param bb Context for translating this sub-query - * @param rels Relational expressions - * @param systemFieldCount Number of system fields - */ - LookupContext(Blackboard bb, List rels, int systemFieldCount) { - bb.flatten(rels, systemFieldCount, new int[]{0}, relOffsetList); - } - - /** - * Returns the relational expression with a given offset, and the - * ordinal in the combined row of its first field. - * - *

For example, in {@code Emp JOIN Dept}, findRel(1) returns the - * relational expression for {@code Dept} and offset 6 (because - * {@code Emp} has 6 fields, therefore the first field of {@code Dept} - * is field 6. - * - * @param offset Offset of relational expression in FROM clause - * @return Relational expression and the ordinal of its first field - */ - Pair findRel(int offset) { - return relOffsetList.get(offset); - } - } - - /** - * Shuttle which walks over a tree of {@link RexNode}s and applies 'over' to - * all agg functions. - * - *

This is necessary because the returned expression is not necessarily a - * call to an agg function. For example, - * - *

AVG(x)
- * - * becomes - * - *
SUM(x) / COUNT(x)
- * - *

Any aggregate functions are converted to calls to the internal - * $Histogram aggregation function and accessors such as - * $HistogramMin; for example, - * - *

MIN(x), MAX(x)
- * - * are converted to - * - *
$HistogramMin($Histogram(x)), - * $HistogramMax($Histogram(x))
- * - * Common sub-expression elmination will ensure that only one histogram is - * computed. - */ - private class HistogramShuttle extends RexShuttle { - /** - * Whether to convert calls to MIN(x) to HISTOGRAM_MIN(HISTOGRAM(x)). - * Histograms allow rolling computation, but require more space. - */ - static final boolean ENABLE_HISTOGRAM_AGG = false; - - private final List partitionKeys; - private final ImmutableList orderKeys; - private final RexWindowBound lowerBound; - private final RexWindowBound upperBound; - private final SqlWindow window; - - HistogramShuttle( - List partitionKeys, - ImmutableList orderKeys, - RexWindowBound lowerBound, RexWindowBound upperBound, - SqlWindow window) { - this.partitionKeys = partitionKeys; - this.orderKeys = orderKeys; - this.lowerBound = lowerBound; - this.upperBound = upperBound; - this.window = window; - } - - public RexNode visitCall(RexCall call) { - final SqlOperator op = call.getOperator(); - if (!(op instanceof SqlAggFunction)) { - return super.visitCall(call); - } - final SqlAggFunction aggOp = (SqlAggFunction) op; - final RelDataType type = call.getType(); - List exprs = call.getOperands(); - - SqlFunction histogramOp = !ENABLE_HISTOGRAM_AGG - ? null - : getHistogramOp(aggOp); - - if (histogramOp != null) { - final RelDataType histogramType = computeHistogramType(type); - - // For DECIMAL, since it's already represented as a bigint we - // want to do a reinterpretCast instead of a cast to avoid - // losing any precision. - boolean reinterpretCast = - type.getSqlTypeName() == SqlTypeName.DECIMAL; - - // Replace original expression with CAST of not one - // of the supported types - if (histogramType != type) { - exprs = new ArrayList<>(exprs); - exprs.set( - 0, - reinterpretCast - ? rexBuilder.makeReinterpretCast(histogramType, exprs.get(0), - rexBuilder.makeLiteral(false)) - : rexBuilder.makeCast(histogramType, exprs.get(0))); - } - - RexCallBinding bind = - new RexCallBinding( - rexBuilder.getTypeFactory(), - SqlStdOperatorTable.HISTOGRAM_AGG, - exprs, - ImmutableList.of()); - - RexNode over = - rexBuilder.makeOver( - SqlStdOperatorTable.HISTOGRAM_AGG - .inferReturnType(bind), - SqlStdOperatorTable.HISTOGRAM_AGG, - exprs, - partitionKeys, - orderKeys, - lowerBound, - upperBound, - window.isRows(), - window.isAllowPartial(), - false); - - RexNode histogramCall = - rexBuilder.makeCall( - histogramType, - histogramOp, - ImmutableList.of(over)); - - // If needed, post Cast result back to original - // type. - if (histogramType != type) { - if (reinterpretCast) { - histogramCall = - rexBuilder.makeReinterpretCast( - type, - histogramCall, - rexBuilder.makeLiteral(false)); - } else { - histogramCall = - rexBuilder.makeCast(type, histogramCall); - } - } - - return histogramCall; - } else { - boolean needSum0 = aggOp == SqlStdOperatorTable.SUM - && type.isNullable(); - SqlAggFunction aggOpToUse = - needSum0 ? SqlStdOperatorTable.SUM0 - : aggOp; - return rexBuilder.makeOver( - type, - aggOpToUse, - exprs, - partitionKeys, - orderKeys, - lowerBound, - upperBound, - window.isRows(), - window.isAllowPartial(), - needSum0); - } - } - - /** - * Returns the histogram operator corresponding to a given aggregate - * function. - * - *

For example, getHistogramOp - *({@link SqlStdOperatorTable#MIN}} returns - * {@link SqlStdOperatorTable#HISTOGRAM_MIN}. - * - * @param aggFunction An aggregate function - * @return Its histogram function, or null - */ - SqlFunction getHistogramOp(SqlAggFunction aggFunction) { - if (aggFunction == SqlStdOperatorTable.MIN) { - return SqlStdOperatorTable.HISTOGRAM_MIN; - } else if (aggFunction == SqlStdOperatorTable.MAX) { - return SqlStdOperatorTable.HISTOGRAM_MAX; - } else if (aggFunction == SqlStdOperatorTable.FIRST_VALUE) { - return SqlStdOperatorTable.HISTOGRAM_FIRST_VALUE; - } else if (aggFunction == SqlStdOperatorTable.LAST_VALUE) { - return SqlStdOperatorTable.HISTOGRAM_LAST_VALUE; - } else { - return null; - } - } - - /** - * Returns the type for a histogram function. It is either the actual - * type or an an approximation to it. - */ - private RelDataType computeHistogramType(RelDataType type) { - if (SqlTypeUtil.isExactNumeric(type) - && type.getSqlTypeName() != SqlTypeName.BIGINT) { - return typeFactory.createSqlType(SqlTypeName.BIGINT); - } else if (SqlTypeUtil.isApproximateNumeric(type) - && type.getSqlTypeName() != SqlTypeName.DOUBLE) { - return typeFactory.createSqlType(SqlTypeName.DOUBLE); - } else { - return type; - } - } - } - - /** A sub-query, whether it needs to be translated using 2- or 3-valued - * logic. */ - private static class SubQuery { - final SqlNode node; - final RelOptUtil.Logic logic; - RexNode expr; - - private SubQuery(SqlNode node, RelOptUtil.Logic logic) { - this.node = node; - this.logic = logic; - } - } - - /** - * Visitor that collects all aggregate functions in a {@link SqlNode} tree. - */ - private static class AggregateFinder extends SqlBasicVisitor { - final SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO); - - @Override public Void visit(SqlCall call) { - // ignore window aggregates and ranking functions (associated with OVER operator) - if (call.getOperator().getKind() == SqlKind.OVER) { - return null; - } - if (call.getOperator().isAggregator()) { - list.add(call); - return null; - } - - // Don't traverse into sub-queries, even if they contain aggregate - // functions. - if (call instanceof SqlSelect) { - return null; - } - - return call.getOperator().acceptCall(this, call); - } - } - - /** Use of a row as a correlating variable by a given relational - * expression. */ - private static class CorrelationUse { - private final CorrelationId id; - private final ImmutableBitSet requiredColumns; - private final RelNode r; - - CorrelationUse(CorrelationId id, ImmutableBitSet requiredColumns, - RelNode r) { - this.id = id; - this.requiredColumns = requiredColumns; - this.r = r; - } - } - - /** Creates a builder for a {@link Config}. */ - public static ConfigBuilder configBuilder() { - return new ConfigBuilder(); - } - - /** - * Interface to define the configuration for a SqlToRelConverter. - * Provides methods to set each configuration option. - * - * @see ConfigBuilder - * @see SqlToRelConverter#configBuilder() - */ - public interface Config { - /** Default configuration. */ - Config DEFAULT = configBuilder().build(); - - /** Returns the {@code convertTableAccess} option. Controls whether table - * access references are converted to physical rels immediately. The - * optimizer doesn't like leaf rels to have {@link Convention#NONE}. - * However, if we are doing further conversion passes (e.g. - * {@link RelStructuredTypeFlattener}), then we may need to defer - * conversion. */ - boolean isConvertTableAccess(); - - /** Returns the {@code decorrelationEnabled} option. Controls whether to - * disable sub-query decorrelation when needed. e.g. if outer joins are not - * supported. */ - boolean isDecorrelationEnabled(); - - /** Returns the {@code trimUnusedFields} option. Controls whether to trim - * unused fields as part of the conversion process. */ - boolean isTrimUnusedFields(); - - /** Returns the {@code createValuesRel} option. Controls whether instances - * of {@link org.apache.calcite.rel.logical.LogicalValues} are generated. - * These may not be supported by all physical implementations. */ - boolean isCreateValuesRel(); - - /** Returns the {@code explain} option. Describes whether the current - * statement is part of an EXPLAIN PLAN statement. */ - boolean isExplain(); - - /** Returns the {@code expand} option. Controls whether to expand - * sub-queries. If false, each sub-query becomes a - * {@link org.apache.calcite.rex.RexSubQuery}. */ - boolean isExpand(); - - /** Returns the {@code inSubQueryThreshold} option, - * default {@link #DEFAULT_IN_SUB_QUERY_THRESHOLD}. Controls the list size - * threshold under which {@link #convertInToOr} is used. Lists of this size - * or greater will instead be converted to use a join against an inline - * table ({@link org.apache.calcite.rel.logical.LogicalValues}) rather than - * a predicate. A threshold of 0 forces usage of an inline table in all - * cases; a threshold of {@link Integer#MAX_VALUE} forces usage of OR in all - * cases. */ - int getInSubQueryThreshold(); - } - - /** Builder for a {@link Config}. */ - public static class ConfigBuilder { - private boolean convertTableAccess = true; - private boolean decorrelationEnabled = true; - private boolean trimUnusedFields = false; - private boolean createValuesRel = true; - private boolean explain; - private boolean expand = true; - private int inSubQueryThreshold = DEFAULT_IN_SUB_QUERY_THRESHOLD; - - private ConfigBuilder() {} - - /** Sets configuration identical to a given {@link Config}. */ - public ConfigBuilder withConfig(Config config) { - this.convertTableAccess = config.isConvertTableAccess(); - this.decorrelationEnabled = config.isDecorrelationEnabled(); - this.trimUnusedFields = config.isTrimUnusedFields(); - this.createValuesRel = config.isCreateValuesRel(); - this.explain = config.isExplain(); - this.expand = config.isExpand(); - this.inSubQueryThreshold = config.getInSubQueryThreshold(); - return this; - } - - public ConfigBuilder withConvertTableAccess(boolean convertTableAccess) { - this.convertTableAccess = convertTableAccess; - return this; - } - - public ConfigBuilder withDecorrelationEnabled(boolean enabled) { - this.decorrelationEnabled = enabled; - return this; - } - - public ConfigBuilder withTrimUnusedFields(boolean trimUnusedFields) { - this.trimUnusedFields = trimUnusedFields; - return this; - } - - public ConfigBuilder withCreateValuesRel(boolean createValuesRel) { - this.createValuesRel = createValuesRel; - return this; - } - - public ConfigBuilder withExplain(boolean explain) { - this.explain = explain; - return this; - } - - public ConfigBuilder withExpand(boolean expand) { - this.expand = expand; - return this; - } - - @Deprecated // to be removed before 2.0 - public ConfigBuilder withInSubqueryThreshold(int inSubQueryThreshold) { - return withInSubQueryThreshold(inSubQueryThreshold); - } - - public ConfigBuilder withInSubQueryThreshold(int inSubQueryThreshold) { - this.inSubQueryThreshold = inSubQueryThreshold; - return this; - } - - /** Builds a {@link Config}. */ - public Config build() { - return new ConfigImpl(convertTableAccess, decorrelationEnabled, - trimUnusedFields, createValuesRel, explain, expand, - inSubQueryThreshold); - } - } - - /** Implementation of {@link Config}. - * Called by builder; all values are in private final fields. */ - private static class ConfigImpl implements Config { - private final boolean convertTableAccess; - private final boolean decorrelationEnabled; - private final boolean trimUnusedFields; - private final boolean createValuesRel; - private final boolean explain; - private final int inSubQueryThreshold; - private final boolean expand; - - private ConfigImpl(boolean convertTableAccess, boolean decorrelationEnabled, - boolean trimUnusedFields, boolean createValuesRel, boolean explain, - boolean expand, int inSubQueryThreshold) { - this.convertTableAccess = convertTableAccess; - this.decorrelationEnabled = decorrelationEnabled; - this.trimUnusedFields = trimUnusedFields; - this.createValuesRel = createValuesRel; - this.explain = explain; - this.expand = expand; - this.inSubQueryThreshold = inSubQueryThreshold; - } - - public boolean isConvertTableAccess() { - return convertTableAccess; - } - - public boolean isDecorrelationEnabled() { - return decorrelationEnabled; - } - - public boolean isTrimUnusedFields() { - return trimUnusedFields; - } - - public boolean isCreateValuesRel() { - return createValuesRel; - } - - public boolean isExplain() { - return explain; - } - - public boolean isExpand() { - return expand; - } - - public int getInSubQueryThreshold() { - return inSubQueryThreshold; - } - } -} - -// End SqlToRelConverter.java diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index 6c23b9e085c32..dbefe203e9601 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -219,7 +219,7 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp override def createSqlType(typeName: SqlTypeName, precision: Int): RelDataType = { // it might happen that inferred VARCHAR types overflow as we set them to Int.MaxValue - // always set those to default value + // Calcite will limit the length of the VARCHAR type to 65536. if (typeName == VARCHAR && precision < 0) { createSqlType(typeName, getTypeSystem.getDefaultPrecision(typeName)) } else { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala index 59352974ab1bb..3a195ed746f9f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala @@ -36,7 +36,7 @@ class FlinkTypeSystem extends RelDataTypeSystemImpl { override def getDefaultPrecision(typeName: SqlTypeName): Int = typeName match { - // by default all VARCHARs can have the Java default length + // Calcite will limit the length of the VARCHAR field to 65536 case SqlTypeName.VARCHAR => Int.MaxValue diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala index ad96e77be039a..197449ca5e6ff 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala @@ -98,6 +98,8 @@ class ExternalCatalogSchema( override def contentsHaveChangedSince(lastCheck: Long, now: Long): Boolean = true + override def snapshot(l: Long): Schema = this + /** * Registers sub-Schemas to current schema plus * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 826ba2964e0ff..63fd058b980cb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -553,6 +553,9 @@ abstract class CodeGenerator( generateInputAccess(input._1, input._2, index) } + override def visitTableInputRef(rexTableInputRef: RexTableInputRef): GeneratedExpression = + visitInputRef(rexTableInputRef) + override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = { val refExpr = rexFieldAccess.getReferenceExpr.accept(this) val index = rexFieldAccess.getField.getIndex diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala index eb4b4026b36b7..cad9ccccd6e62 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala @@ -133,6 +133,7 @@ case class OverCall( upperBound, isPhysical, true, + false, false) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala index 0a02666148e83..db2d9f98d74f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala @@ -357,26 +357,36 @@ case class TemporalOverlaps( rightT: RexNode, relBuilder: FlinkRelBuilder) : RexNode = { - // leftT = leftP + leftT if leftT is an interval - val convLeftT = if (isTimeInterval(leftTemporal.resultType)) { - relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, leftP, leftT) - } else { - leftT - } - // rightT = rightP + rightT if rightT is an interval - val convRightT = if (isTimeInterval(rightTemporal.resultType)) { - relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, rightP, rightT) - } else { - rightT - } - // leftT >= rightP - val leftPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convLeftT, rightP) - // rightT >= leftP - val rightPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convRightT, leftP) + val convLeftT = convertOverlapsEnd(relBuilder, leftP, leftT, leftTemporal.resultType) + val convRightT = convertOverlapsEnd(relBuilder, rightP, rightT, rightTemporal.resultType) + + // Sort end points into start and end, such that (s0 <= e0) and (s1 <= e1). + val (s0, e0) = buildSwap(relBuilder, leftP, convLeftT) + val (s1, e1) = buildSwap(relBuilder, rightP, convRightT) - // leftT >= rightP and rightT >= leftP + // (e0 >= s1) AND (e1 >= s0) + val leftPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, e0, s1) + val rightPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, e1, s0) relBuilder.call(SqlStdOperatorTable.AND, leftPred, rightPred) } + + private def convertOverlapsEnd( + relBuilder: FlinkRelBuilder, + start: RexNode, end: RexNode, + endType: TypeInformation[_]) = { + if (isTimeInterval(endType)) { + relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, start, end) + } else { + end + } + } + + private def buildSwap(relBuilder: FlinkRelBuilder, start: RexNode, end: RexNode) = { + val le = relBuilder.call(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, start, end) + val l = relBuilder.call(SqlStdOperatorTable.CASE, le, start, end) + val r = relBuilder.call(SqlStdOperatorTable.CASE, le, end, start) + (l, r) + } } case class DateFormat(timestamp: Expression, format: Expression) extends Expression { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala index b7d99916de23a..526ec47d03e4f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala @@ -55,6 +55,7 @@ class AggSqlFunction( // Do not need to provide a calcite aggregateFunction here. Flink aggregateion function // will be generated when translating the calcite relnode to flink runtime execution plan null, + false, requiresOver ) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 74b789cc66331..a81c7d2301a6a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.plan.rules +import org.apache.calcite.rel.core.RelFactories import org.apache.calcite.rel.rules._ import org.apache.calcite.tools.{RuleSet, RuleSets} import org.apache.flink.table.plan.rules.common._ @@ -59,7 +60,8 @@ object FlinkRuleSets { ProjectFilterTransposeRule.INSTANCE, FilterProjectTransposeRule.INSTANCE, // push a projection to the children of a join - ProjectJoinTransposeRule.INSTANCE, + // push all expressions to handle the time indicator correctly + new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), // merge projections ProjectMergeRule.INSTANCE, // remove identity project diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala index 6f4ea00741c28..a524816a2b907 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.plan.stats import java.lang.Double +import java.util import java.util.{Collections, List} -import org.apache.calcite.rel.{RelCollation, RelDistribution} +import org.apache.calcite.rel.{RelCollation, RelDistribution, RelReferentialConstraint} import org.apache.calcite.schema.Statistic import org.apache.calcite.util.ImmutableBitSet @@ -66,6 +67,8 @@ class FlinkStatistic(tableStats: Option[TableStats]) extends Statistic { override def getDistribution: RelDistribution = null + override def getReferentialConstraints: util.List[RelReferentialConstraint] = + Collections.emptyList() } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index ba8713dcf12f8..bf9a6881addef 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -149,6 +149,9 @@ class RexNodeToExpressionConverter( )) } + override def visitTableInputRef(rexTableInputRef: RexTableInputRef): Option[Expression] = + visitInputRef(rexTableInputRef) + override def visitLocalRef(localRef: RexLocalRef): Option[Expression] = { throw new TableException("Bug: RexLocalRef should have been expanded") } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/CorrelateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/CorrelateTest.scala index 5b9aa8c74aab6..a9938cb89527a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/CorrelateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/CorrelateTest.scala @@ -43,7 +43,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c)"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") @@ -63,7 +63,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c, '$')"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") @@ -89,7 +89,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c)"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "LEFT") ), term("select", "c", "f0 AS s") @@ -115,8 +115,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "func2($cor0.c)"), term("function", func2.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) f0, INTEGER f1)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) f0, INTEGER f1)"), term("joinType", "INNER") ), term("select", "c", "f0 AS name", "f1 AS len") @@ -142,8 +142,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "hierarchy($cor0.c)"), term("function", function.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," + - " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c," + + " VARCHAR(65536) f0, BOOLEAN f1, INTEGER f2)"), term("joinType", "INNER") ), term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len") @@ -169,8 +169,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "pojo($cor0.c)"), term("function", function.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," + - " INTEGER age, VARCHAR(2147483647) name)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c," + + " INTEGER age, VARCHAR(65536) name)"), term("joinType", "INNER") ), term("select", "c", "name", "age") @@ -197,8 +197,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "func2($cor0.c)"), term("function", func2.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) f0, INTEGER f1)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) f0, INTEGER f1)"), term("joinType", "INNER"), term("condition", ">($1, 2)") ), @@ -226,7 +226,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1(SUBSTRING($cor0.c, 2))"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CorrelateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CorrelateTest.scala index 63ce2674d744a..15f3def37ae3c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CorrelateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/CorrelateTest.scala @@ -43,7 +43,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "INNER") ), term("select", "c", "s") @@ -63,7 +63,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2, '$$')"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "INNER") ), term("select", "c", "s") @@ -88,7 +88,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "LEFT") ), term("select", "c", "s") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala index 52d13206dec56..9bb7bcfb60adb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/CorrelateTest.scala @@ -43,7 +43,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c)"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") @@ -63,7 +63,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c, '$')"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") @@ -89,7 +89,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1($cor0.c)"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "LEFT") ), term("select", "c", "f0 AS s") @@ -115,8 +115,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "func2($cor0.c)"), term("function", func2.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) f0, INTEGER f1)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) f0, INTEGER f1)"), term("joinType", "INNER") ), term("select", "c", "f0 AS name", "f1 AS len") @@ -142,8 +142,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "hierarchy($cor0.c)"), term("function", function.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," + - " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c," + + " VARCHAR(65536) f0, BOOLEAN f1, INTEGER f2)"), term("joinType", "INNER") ), term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len") @@ -169,8 +169,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "pojo($cor0.c)"), term("function", function.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," + - " INTEGER age, VARCHAR(2147483647) name)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c," + + " INTEGER age, VARCHAR(65536) name)"), term("joinType", "INNER") ), term("select", "c", "name", "age") @@ -197,8 +197,8 @@ class CorrelateTest extends TableTestBase { term("invocation", "func2($cor0.c)"), term("function", func2.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) f0, INTEGER f1)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) f0, INTEGER f1)"), term("joinType", "INNER"), term("condition", ">($1, 2)") ), @@ -225,7 +225,7 @@ class CorrelateTest extends TableTestBase { term("invocation", "func1(SUBSTRING($cor0.c, 2))"), term("function", func1.getClass.getCanonicalName), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)"), term("joinType", "INNER") ), term("select", "c", "f0 AS s") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/CorrelateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/CorrelateTest.scala index 8f53f4af59272..f15dea9bf60f6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/CorrelateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/CorrelateTest.scala @@ -42,7 +42,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "INNER") ), term("select", "c", "s") @@ -62,7 +62,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2, '$$')"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "INNER") ), term("select", "c", "s") @@ -87,7 +87,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "LEFT") ), term("select", "c", "s") @@ -112,8 +112,8 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) name, INTEGER len)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) name, INTEGER len)"), term("joinType", "INNER") ), term("select", "c", "name", "len") @@ -136,8 +136,8 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," + - " VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c," + + " VARCHAR(65536) name, BOOLEAN adult, INTEGER len)"), term("joinType", "INNER") ) @@ -158,8 +158,8 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "INTEGER age, VARCHAR(2147483647) name)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "INTEGER age, VARCHAR(65536) name)"), term("joinType", "INNER") ) @@ -185,8 +185,8 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}($$2)"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " + - "VARCHAR(2147483647) name, INTEGER len)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, " + + "VARCHAR(65536) name, INTEGER len)"), term("joinType", "INNER"), term("condition", ">($1, 2)") ), @@ -210,7 +210,7 @@ class CorrelateTest extends TableTestBase { term("invocation", s"${function.functionIdentifier}(SUBSTRING($$2, 2, CHAR_LENGTH($$2)))"), term("function", function), term("rowType", - "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"), + "RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) s)"), term("joinType", "INNER") ) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala index 6a21f217f2488..a5bd6f2414b4c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala @@ -1532,7 +1532,7 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { "'2011-03-10 05:02:02'.toTimestamp, '2011-03-10 05:02:01'.toTimestamp)", "(TIMESTAMP '2011-03-10 05:02:02', INTERVAL '0' SECOND) OVERLAPS " + "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')", - "false") + "true") testAllApis( temporalOverlaps("2011-03-10 02:02:02.001".toTimestamp, 0.milli, @@ -1664,7 +1664,8 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { s"TIMESTAMPADD($interval, ${data(4)._1}, ${data(4)._2})", result(4)) } - testSqlApi("TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')", "null") + testSqlApi("TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER)," + + " TIMESTAMP '2016-02-24 12:42:25')", "null") testSqlApi("TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))", "null") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala index 870025e42ae52..b17debea45367 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala @@ -162,7 +162,7 @@ class TimeIndicatorConversionTest extends TableTestBase { s"${func.functionIdentifier}(TIME_MATERIALIZATION($$0), TIME_MATERIALIZATION($$3), '')"), term("function", func), term("rowType", "RecordType(TIME ATTRIBUTE(ROWTIME) rowtime, BIGINT long, INTEGER int, " + - "TIME ATTRIBUTE(PROCTIME) proctime, VARCHAR(2147483647) s)"), + "TIME ATTRIBUTE(PROCTIME) proctime, VARCHAR(65536) s)"), term("joinType", "INNER") ), term("select",