mirror of https://github.com/apache/druid.git
Add IPv4 SQL functions (#8223)
* Add IPv4 SQL functions New SQL functions for filtering IPv4 addresses: - IPV4_MATCH: Check if IP address belongs to a subnet - IPV4_PARSE: Convert string IP address to integer - IPV4_STRINGIFY: Convert integer IP address to string These are the SQL analogs of the druid expressions with the same name. Filtering is more efficient when operating on IP addresses as integers instead of strings. * Refactor operator conversions into named constants
This commit is contained in:
parent
5528dec4a4
commit
4bd3bad8ba
|
@ -25,7 +25,7 @@ title: "Multi-value dimensions"
|
|||
# Multi-value dimensions
|
||||
|
||||
Apache Druid (incubating) supports "multi-value" string dimensions. These are generated when an input field contains an
|
||||
array of values instead of a single value (e.e. JSON arrays, or a TSV field containing one or more `listDelimiter`
|
||||
array of values instead of a single value (e.g. JSON arrays, or a TSV field containing one or more `listDelimiter`
|
||||
characters).
|
||||
|
||||
This document describes the behavior of groupBy (topN has similar behavior) queries on multi-value dimensions when they
|
||||
|
|
|
@ -77,7 +77,7 @@ The following table describes how SQL types map onto Druid types during query ru
|
|||
that have the same Druid runtime type will have no effect, other than exceptions noted in the table. Casts between two
|
||||
SQL types that have different Druid runtime types will generate a runtime cast in Druid. If a value cannot be properly
|
||||
cast to another value, as in `CAST('foo' AS BIGINT)`, the runtime will substitute a default value. NULL values cast
|
||||
to non-nullable types will also be substitued with a default value (for example, nulls cast to numbers will be
|
||||
to non-nullable types will also be substituted with a default value (for example, nulls cast to numbers will be
|
||||
converted to zeroes).
|
||||
|
||||
|SQL type|Druid runtime type|Default value|Notes|
|
||||
|
@ -183,7 +183,7 @@ Only the COUNT aggregation can accept DISTINCT.
|
|||
|`APPROX_QUANTILE(expr, probability, [resolution])`|Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.html#approximate-histogram-aggregator) exprs. The "probability" should be between 0 and 1 (exclusive). The "resolution" is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|
||||
|`APPROX_QUANTILE_DS(expr, probability, [k])`|Computes approximate quantiles on numeric or [Quantiles sketch](../development/extensions-core/datasketches-quantiles.html) exprs. The "probability" should be between 0 and 1 (exclusive). The `k` parameter is described in the Quantiles sketch documentation. The [DataSketches extension](../development/extensions-core/datasketches-extension.html) must be loaded to use this function.|
|
||||
|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.html#fixed-buckets-histogram) exprs. The "probability" should be between 0 and 1 (exclusive). The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. The [approximate histogram extension](../development/extensions-core/approximate-histograms.html) must be loaded to use this function.|
|
||||
|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positve rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
|
||||
|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positive rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.html) documentation for additional details.|
|
||||
|`VAR_POP(expr)`|Computes variance population of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|
||||
|`VAR_SAMP(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|
||||
|`VARIANCE(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|
||||
|
@ -233,7 +233,7 @@ String functions accept strings, and return a type appropriate to the function.
|
|||
|
||||
|Function|Notes|
|
||||
|--------|-----|
|
||||
|`<code>x || y</code>`|Concat strings x and y.|
|
||||
|<code>x || y</code>|Concat strings x and y.|
|
||||
|`CONCAT(expr, expr...)`|Concats a list of expressions.|
|
||||
|`TEXTCAT(expr, expr)`|Two argument version of CONCAT.|
|
||||
|`STRING_FORMAT(pattern[, args...])`|Returns a string formatted in the manner of Java's [String.format](https://docs.oracle.com/javase/8/docs/api/java/lang/String.html#format-java.lang.String-java.lang.Object...-).|
|
||||
|
@ -252,7 +252,7 @@ String functions accept strings, and return a type appropriate to the function.
|
|||
|`RIGHT(expr, [length])`|Returns the rightmost length characters from expr.|
|
||||
|`LEFT(expr, [length])`|Returns the leftmost length characters from expr.|
|
||||
|`SUBSTR(expr, index, [length])`|Synonym for SUBSTRING.|
|
||||
|`TRIM([BOTH | LEADING | TRAILING] [<chars> FROM] expr)`|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".|
|
||||
|<code>TRIM([BOTH | LEADING | TRAILING] [<chars> FROM] expr)</code>|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".|
|
||||
|`BTRIM(expr[, chars])`|Alternate form of `TRIM(BOTH <chars> FROM <expr>`).|
|
||||
|`LTRIM(expr[, chars])`|Alternate form of `TRIM(LEADING <chars> FROM <expr>`).|
|
||||
|`RTRIM(expr[, chars])`|Alternate form of `TRIM(TRAILING <chars> FROM <expr>`).|
|
||||
|
@ -281,7 +281,7 @@ simplest way to write literal timestamps in other time zones is to use TIME_PARS
|
|||
|--------|-----|
|
||||
|`CURRENT_TIMESTAMP`|Current timestamp in the connection's time zone.|
|
||||
|`CURRENT_DATE`|Current date in the connection's time zone.|
|
||||
|`DATE_TRUNC(<unit>, <timestamp_expr>)`|Rounds down a timestamp, returning it as a new timestamp. Unit can be 'milliseconds', 'second', 'minute', 'hour', 'day', 'week', 'month', 'quarter', 'year', 'decade', 'century', or 'millenium'.|
|
||||
|`DATE_TRUNC(<unit>, <timestamp_expr>)`|Rounds down a timestamp, returning it as a new timestamp. Unit can be 'milliseconds', 'second', 'minute', 'hour', 'day', 'week', 'month', 'quarter', 'year', 'decade', 'century', or 'millennium'.|
|
||||
|`TIME_CEIL(<timestamp_expr>, <period>, [<origin>, [<timezone>]])`|Rounds up a timestamp, returning it as a new timestamp. Period can be any ISO8601 period, like P3M (quarters) or PT12H (half-days). The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00". This function is similar to `CEIL` but is more flexible.|
|
||||
|`TIME_FLOOR(<timestamp_expr>, <period>, [<origin>, [<timezone>]])`|Rounds down a timestamp, returning it as a new timestamp. Period can be any ISO8601 period, like P3M (quarters) or PT12H (half-days). The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00". This function is similar to `FLOOR` but is more flexible.|
|
||||
|`TIME_SHIFT(<timestamp_expr>, <period>, <step>, [<timezone>])`|Shifts a timestamp by a period (step times), returning it as a new timestamp. Period can be any ISO8601 period. Step may be negative. The time zone, if provided, should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|
|
||||
|
@ -295,7 +295,22 @@ simplest way to write literal timestamps in other time zones is to use TIME_PARS
|
|||
|`CEIL(timestamp_expr TO <unit>)`|Rounds up a timestamp, returning it as a new timestamp. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
|
||||
|`TIMESTAMPADD(<unit>, <count>, <timestamp>)`|Equivalent to `timestamp + count * INTERVAL '1' UNIT`.|
|
||||
|`TIMESTAMPDIFF(<unit>, <timestamp1>, <timestamp2>)`|Returns the (signed) number of `unit` between `timestamp1` and `timestamp2`. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
|
||||
|`timestamp_expr { + | - } <interval_expr>`|Add or subtract an amount of time from a timestamp. interval_expr can include interval literals like `INTERVAL '2' HOUR`, and may include interval arithmetic as well. This operator treats days as uniformly 86400 seconds long, and does not take into account daylight savings time. To account for daylight savings time, use TIME_SHIFT instead.|
|
||||
|<code>timestamp_expr { + | - } <interval_expr><code>|Add or subtract an amount of time from a timestamp. interval_expr can include interval literals like `INTERVAL '2' HOUR`, and may include interval arithmetic as well. This operator treats days as uniformly 86400 seconds long, and does not take into account daylight savings time. To account for daylight savings time, use TIME_SHIFT instead.|
|
||||
|
||||
|
||||
### IP address functions
|
||||
|
||||
For the IPv4 address functions, the `address` argument can either be an IPv4 dotted-decimal string
|
||||
(e.g., '192.168.0.1') or an IP address represented as an integer (e.g., 3232235521). The `subnet`
|
||||
argument should be a string formatted as an IPv4 address subnet in CIDR notation (e.g.,
|
||||
'192.168.0.0/16').
|
||||
|
||||
|Function|Notes|
|
||||
|---|---|
|
||||
|`IPV4_MATCH(address, subnet)`|Returns true if the `address` belongs to the `subnet` literal, else false. If `address` is not a valid IPv4 address, then false is returned. This function is more efficient if `address` is an integer instead of a string.|
|
||||
|`IPV4_PARSE(address)`|Parses `address` into an IPv4 address stored as an integer . If `address` is an integer that is a valid IPv4 address, then it is passed through. Returns null if `address` cannot be represented as an IPv4 address.|
|
||||
|`IPV4_STRINGIFY(address)`|Converts `address` into an IPv4 address dotted-decimal string. If `address` is a string that is a valid IPv4 address, then it is passed through. Returns null if `address` cannot be represented as an IPv4 address.|
|
||||
|
||||
|
||||
### Comparison operators
|
||||
|
||||
|
@ -566,12 +581,12 @@ Connection context can be specified as JDBC connection properties or as a "conte
|
|||
|---------|-----------|-------------|
|
||||
|`sqlQueryId`|Unique identifier given to this SQL query. For HTTP client, it will be returned in `X-Druid-SQL-Query-Id` header.|auto-generated|
|
||||
|`sqlTimeZone`|Sets the time zone for this connection, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|druid.sql.planner.sqlTimeZone on the Broker (default: UTC)|
|
||||
|`useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|druid.sql.planner.useApproximateCountDistinct on the Broker (default: true)|
|
||||
|`useApproximateCountDistinct`|Whether to use an approximate cardinality algorithm for `COUNT(DISTINCT foo)`.|druid.sql.planner.useApproximateCountDistinct on the Broker (default: true)|
|
||||
|`useApproximateTopN`|Whether to use approximate [TopN queries](topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](groupbyquery.html) will be used instead.|druid.sql.planner.useApproximateTopN on the Broker (default: true)|
|
||||
|
||||
### Retrieving metadata
|
||||
|
||||
Druid Brokers infer table and column metadata for each dataSource from segments loaded in the cluster, and use this to
|
||||
Druid Brokers infer table and column metadata for each datasource from segments loaded in the cluster, and use this to
|
||||
plan SQL queries. This metadata is cached on Broker startup and also updated periodically in the background through
|
||||
[SegmentMetadata queries](segmentmetadataquery.html). Background metadata refreshing is triggered by
|
||||
segments entering and exiting the cluster, and can also be throttled through configuration.
|
||||
|
@ -640,7 +655,7 @@ The "sys" schema provides visibility into Druid segments, servers and tasks.
|
|||
Segments table provides details on all Druid segments, whether they are published yet or not.
|
||||
|
||||
#### CAVEAT
|
||||
Note that a segment can be served by more than one stream ingestion tasks or Historical processes, in that case it would have multiple replicas. These replicas are weakly consistent with each other when served by multiple ingestion tasks, until a segment is eventually served by a Historical, at that point the segment is immutable. Broker prefers to query a segment from Historical over an ingestion task. But if a segment has multiple realtime replicas, for eg. kafka index tasks, and one task is slower than other, then the sys.segments query results can vary for the duration of the tasks because only one of the ingestion tasks is queried by the Broker and it is not gauranteed that the same task gets picked everytime. The `num_rows` column of segments table can have inconsistent values during this period. There is an open [issue](https://github.com/apache/incubator-druid/issues/5915) about this inconsistency with stream ingestion tasks.
|
||||
Note that a segment can be served by more than one stream ingestion tasks or Historical processes, in that case it would have multiple replicas. These replicas are weakly consistent with each other when served by multiple ingestion tasks, until a segment is eventually served by a Historical, at that point the segment is immutable. Broker prefers to query a segment from Historical over an ingestion task. But if a segment has multiple realtime replicas, for eg. kafka index tasks, and one task is slower than other, then the sys.segments query results can vary for the duration of the tasks because only one of the ingestion tasks is queried by the Broker and it is not guaranteed that the same task gets picked every time. The `num_rows` column of segments table can have inconsistent values during this period. There is an open [issue](https://github.com/apache/incubator-druid/issues/5915) about this inconsistency with stream ingestion tasks.
|
||||
|
||||
|Column|Type|Notes|
|
||||
|------|-----|-----|
|
||||
|
@ -652,7 +667,7 @@ Note that a segment can be served by more than one stream ingestion tasks or His
|
|||
|version|STRING|Version string (generally an ISO8601 timestamp corresponding to when the segment set was first started). Higher version means the more recently created segment. Version comparing is based on string comparison.|
|
||||
|partition_num|LONG|Partition number (an integer, unique within a datasource+interval+version; may not necessarily be contiguous)|
|
||||
|num_replicas|LONG|Number of replicas of this segment currently being served|
|
||||
|num_rows|LONG|Number of rows in current segment, this value could be null if unkown to Broker at query time|
|
||||
|num_rows|LONG|Number of rows in current segment, this value could be null if unknown to Broker at query time|
|
||||
|is_published|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 represents this segment has been published to the metadata store with `used=1`|
|
||||
|is_available|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is currently being served by any process(Historical or realtime)|
|
||||
|is_realtime|LONG|Boolean is represented as long type where 1 = true, 0 = false. 1 if this segment is _only_ served by realtime tasks, and 0 if any historical process is serving this segment|
|
||||
|
@ -769,9 +784,9 @@ The Druid SQL server is configured through the following properties on the Broke
|
|||
|`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M|
|
||||
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|
||||
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|
||||
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|
||||
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries without filter condition on __time column will fail|false|
|
||||
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|
||||
|`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST api will be invoked when broker needs published segments info.|false|
|
||||
|`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST API will be invoked when broker needs published segments info.|false|
|
||||
|`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000|
|
||||
|
||||
## SQL Metrics
|
||||
|
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression.builtin;
|
||||
|
||||
import org.apache.calcite.sql.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.expression.IPv4AddressMatchExprMacro;
|
||||
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
|
||||
public class IPv4AddressMatchOperatorConversion extends DirectOperatorConversion
|
||||
{
|
||||
private static final SqlSingleOperandTypeChecker ADDRESS_OPERAND = OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.INTEGER)
|
||||
);
|
||||
|
||||
private static final SqlSingleOperandTypeChecker SUBNET_OPERAND = OperandTypes.family(SqlTypeFamily.STRING);
|
||||
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(IPv4AddressMatchExprMacro.NAME))
|
||||
.operandTypeChecker(OperandTypes.sequence("(expr,string)", ADDRESS_OPERAND, SUBNET_OPERAND))
|
||||
.returnTypeInference(ReturnTypes.BOOLEAN_NULLABLE)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
||||
public IPv4AddressMatchOperatorConversion()
|
||||
{
|
||||
super(SQL_FUNCTION, IPv4AddressMatchExprMacro.NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression.builtin;
|
||||
|
||||
import org.apache.calcite.sql.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.ReturnTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.expression.IPv4AddressParseExprMacro;
|
||||
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
|
||||
public class IPv4AddressParseOperatorConversion extends DirectOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(IPv4AddressParseExprMacro.NAME))
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.INTEGER)
|
||||
))
|
||||
.returnTypeInference(ReturnTypes.INTEGER_NULLABLE)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
||||
public IPv4AddressParseOperatorConversion()
|
||||
{
|
||||
super(SQL_FUNCTION, IPv4AddressParseExprMacro.NAME);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,57 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression.builtin;
|
||||
|
||||
import org.apache.calcite.sql.SqlFunction;
|
||||
import org.apache.calcite.sql.SqlFunctionCategory;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.OperandTypes;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.expression.IPv4AddressStringifyExprMacro;
|
||||
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.OperatorConversions;
|
||||
|
||||
public class IPv4AddressStringifyOperatorConversion extends DirectOperatorConversion
|
||||
{
|
||||
private static final SqlFunction SQL_FUNCTION = OperatorConversions
|
||||
.operatorBuilder(StringUtils.toUpperCase(IPv4AddressStringifyExprMacro.NAME))
|
||||
.operandTypeChecker(
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.INTEGER),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
))
|
||||
.nullableReturnType(SqlTypeName.CHAR)
|
||||
.functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION)
|
||||
.build();
|
||||
|
||||
public IPv4AddressStringifyOperatorConversion()
|
||||
{
|
||||
super(SQL_FUNCTION, IPv4AddressStringifyExprMacro.NAME);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public SqlOperator calciteOperator()
|
||||
{
|
||||
return SQL_FUNCTION;
|
||||
}
|
||||
}
|
|
@ -62,6 +62,9 @@ import org.apache.druid.sql.calcite.expression.builtin.ConcatOperatorConversion;
|
|||
import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.ExtractOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.FloorOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressMatchOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressParseOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressStringifyOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.LPadOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.LTrimOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.LeftOperatorConversion;
|
||||
|
@ -127,6 +130,93 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
"strlen"
|
||||
);
|
||||
|
||||
private static final List<SqlOperatorConversion> TIME_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new CeilOperatorConversion())
|
||||
.add(new DateTruncOperatorConversion())
|
||||
.add(new ExtractOperatorConversion())
|
||||
.add(new FloorOperatorConversion())
|
||||
.add(new MillisToTimestampOperatorConversion())
|
||||
.add(new TimeArithmeticOperatorConversion.TimeMinusIntervalOperatorConversion())
|
||||
.add(new TimeArithmeticOperatorConversion.TimePlusIntervalOperatorConversion())
|
||||
.add(new TimeExtractOperatorConversion())
|
||||
.add(new TimeCeilOperatorConversion())
|
||||
.add(new TimeFloorOperatorConversion())
|
||||
.add(new TimeFormatOperatorConversion())
|
||||
.add(new TimeParseOperatorConversion())
|
||||
.add(new TimeShiftOperatorConversion())
|
||||
.add(new TimestampToMillisOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> STRING_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new BTrimOperatorConversion())
|
||||
.add(new LikeOperatorConversion())
|
||||
.add(new LTrimOperatorConversion())
|
||||
.add(new PositionOperatorConversion())
|
||||
.add(new RegexpExtractOperatorConversion())
|
||||
.add(new RTrimOperatorConversion())
|
||||
.add(new ParseLongOperatorConversion())
|
||||
.add(new StringFormatOperatorConversion())
|
||||
.add(new StrposOperatorConversion())
|
||||
.add(new SubstringOperatorConversion())
|
||||
.add(new RightOperatorConversion())
|
||||
.add(new LeftOperatorConversion())
|
||||
.add(new ReverseOperatorConversion())
|
||||
.add(new RepeatOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new SubstringOperatorConversion(), "SUBSTR"))
|
||||
.add(new ConcatOperatorConversion())
|
||||
.add(new TextcatOperatorConversion())
|
||||
.add(new TrimOperatorConversion())
|
||||
.add(new TruncateOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new TruncateOperatorConversion(), "TRUNC"))
|
||||
.add(new LPadOperatorConversion())
|
||||
.add(new RPadOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> VALUE_COERCION_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new CastOperatorConversion())
|
||||
.add(new ReinterpretOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> ARRAY_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new ArrayConstructorOperatorConversion())
|
||||
.add(new ArrayContainsOperatorConversion())
|
||||
.add(new ArrayOverlapOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayContainsOperatorConversion(), "MV_CONTAINS"))
|
||||
.add(new AliasedOperatorConversion(new ArrayOverlapOperatorConversion(), "MV_OVERLAP"))
|
||||
.add(new ArrayLengthOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayLengthOperatorConversion(), "MV_LENGTH"))
|
||||
.add(new ArrayOffsetOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOperatorConversion(), "MV_OFFSET"))
|
||||
.add(new ArrayOrdinalOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOperatorConversion(), "MV_ORDINAL"))
|
||||
.add(new ArrayOffsetOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOfOperatorConversion(), "MV_OFFSET_OF"))
|
||||
.add(new ArrayOrdinalOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOfOperatorConversion(), "MV_ORDINAL_OF"))
|
||||
.add(new ArrayToStringOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayToStringOperatorConversion(), "MV_TO_STRING"))
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> MULTIVALUE_STRING_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new MultiValueStringAppendOperatorConversion())
|
||||
.add(new MultiValueStringConcatOperatorConversion())
|
||||
.add(new MultiValueStringPrependOperatorConversion())
|
||||
.add(new MultiValueStringSliceOperatorConversion())
|
||||
.add(new StringToMultiValueStringOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> IPV4ADDRESS_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new IPv4AddressMatchOperatorConversion())
|
||||
.add(new IPv4AddressParseOperatorConversion())
|
||||
.add(new IPv4AddressStringifyOperatorConversion())
|
||||
.build();
|
||||
|
||||
private static final List<SqlOperatorConversion> STANDARD_OPERATOR_CONVERSIONS =
|
||||
ImmutableList.<SqlOperatorConversion>builder()
|
||||
.add(new DirectOperatorConversion(SqlStdOperatorTable.ABS, "abs"))
|
||||
|
@ -178,71 +268,12 @@ public class DruidOperatorTable implements SqlOperatorTable
|
|||
.add(new BinaryOperatorConversion(SqlStdOperatorTable.AND, "&&"))
|
||||
.add(new BinaryOperatorConversion(SqlStdOperatorTable.OR, "||"))
|
||||
.add(new RoundOperatorConversion())
|
||||
// time operators
|
||||
.add(new CeilOperatorConversion())
|
||||
.add(new DateTruncOperatorConversion())
|
||||
.add(new ExtractOperatorConversion())
|
||||
.add(new FloorOperatorConversion())
|
||||
.add(new MillisToTimestampOperatorConversion())
|
||||
.add(new TimeArithmeticOperatorConversion.TimeMinusIntervalOperatorConversion())
|
||||
.add(new TimeArithmeticOperatorConversion.TimePlusIntervalOperatorConversion())
|
||||
.add(new TimeExtractOperatorConversion())
|
||||
.add(new TimeCeilOperatorConversion())
|
||||
.add(new TimeFloorOperatorConversion())
|
||||
.add(new TimeFormatOperatorConversion())
|
||||
.add(new TimeParseOperatorConversion())
|
||||
.add(new TimeShiftOperatorConversion())
|
||||
.add(new TimestampToMillisOperatorConversion())
|
||||
// string operators
|
||||
.add(new BTrimOperatorConversion())
|
||||
.add(new LikeOperatorConversion())
|
||||
.add(new LTrimOperatorConversion())
|
||||
.add(new PositionOperatorConversion())
|
||||
.add(new RegexpExtractOperatorConversion())
|
||||
.add(new RTrimOperatorConversion())
|
||||
.add(new ParseLongOperatorConversion())
|
||||
.add(new StringFormatOperatorConversion())
|
||||
.add(new StrposOperatorConversion())
|
||||
.add(new SubstringOperatorConversion())
|
||||
.add(new RightOperatorConversion())
|
||||
.add(new LeftOperatorConversion())
|
||||
.add(new ReverseOperatorConversion())
|
||||
.add(new RepeatOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new SubstringOperatorConversion(), "SUBSTR"))
|
||||
.add(new ConcatOperatorConversion())
|
||||
.add(new TextcatOperatorConversion())
|
||||
.add(new TrimOperatorConversion())
|
||||
.add(new TruncateOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new TruncateOperatorConversion(), "TRUNC"))
|
||||
.add(new LPadOperatorConversion())
|
||||
.add(new RPadOperatorConversion())
|
||||
// value coercion operators
|
||||
.add(new CastOperatorConversion())
|
||||
.add(new ReinterpretOperatorConversion())
|
||||
// array and multi-value string operators
|
||||
.add(new ArrayConstructorOperatorConversion())
|
||||
.add(new ArrayContainsOperatorConversion())
|
||||
.add(new ArrayOverlapOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayContainsOperatorConversion(), "MV_CONTAINS"))
|
||||
.add(new AliasedOperatorConversion(new ArrayOverlapOperatorConversion(), "MV_OVERLAP"))
|
||||
.add(new ArrayLengthOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayLengthOperatorConversion(), "MV_LENGTH"))
|
||||
.add(new ArrayOffsetOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOperatorConversion(), "MV_OFFSET"))
|
||||
.add(new ArrayOrdinalOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOperatorConversion(), "MV_ORDINAL"))
|
||||
.add(new ArrayOffsetOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOffsetOfOperatorConversion(), "MV_OFFSET_OF"))
|
||||
.add(new ArrayOrdinalOfOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayOrdinalOfOperatorConversion(), "MV_ORDINAL_OF"))
|
||||
.add(new ArrayToStringOperatorConversion())
|
||||
.add(new AliasedOperatorConversion(new ArrayToStringOperatorConversion(), "MV_TO_STRING"))
|
||||
// multi-value string operators
|
||||
.add(new MultiValueStringAppendOperatorConversion())
|
||||
.add(new MultiValueStringConcatOperatorConversion())
|
||||
.add(new MultiValueStringPrependOperatorConversion())
|
||||
.add(new MultiValueStringSliceOperatorConversion())
|
||||
.add(new StringToMultiValueStringOperatorConversion())
|
||||
.addAll(TIME_OPERATOR_CONVERSIONS)
|
||||
.addAll(STRING_OPERATOR_CONVERSIONS)
|
||||
.addAll(VALUE_COERCION_OPERATOR_CONVERSIONS)
|
||||
.addAll(ARRAY_OPERATOR_CONVERSIONS)
|
||||
.addAll(MULTIVALUE_STRING_OPERATOR_CONVERSIONS)
|
||||
.addAll(IPV4ADDRESS_OPERATOR_CONVERSIONS)
|
||||
.build();
|
||||
|
||||
// Operators that have no conversion, but are handled in the convertlet table, so they still need to exist.
|
||||
|
|
|
@ -0,0 +1,36 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression;
|
||||
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.junit.Rule;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
public abstract class ExpressionTestBase extends CalciteTestBase
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
void expectException(Class<? extends Throwable> type, String message)
|
||||
{
|
||||
expectedException.expect(type);
|
||||
expectedException.expectMessage(message);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,236 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlIntervalQualifier;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.Assert;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
class ExpressionTestHelper
|
||||
{
|
||||
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
|
||||
CalciteTests.createOperatorTable(),
|
||||
CalciteTests.createExprMacroTable(),
|
||||
new PlannerConfig(),
|
||||
ImmutableMap.of(),
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT
|
||||
);
|
||||
|
||||
private final RowSignature rowSignature;
|
||||
private final Map<String, Object> bindings;
|
||||
private final RelDataTypeFactory typeFactory;
|
||||
private final RexBuilder rexBuilder;
|
||||
private final RelDataType relDataType;
|
||||
|
||||
ExpressionTestHelper(RowSignature rowSignature, Map<String, Object> bindings)
|
||||
{
|
||||
this.rowSignature = rowSignature;
|
||||
this.bindings = bindings;
|
||||
|
||||
this.typeFactory = new JavaTypeFactoryImpl();
|
||||
this.rexBuilder = new RexBuilder(typeFactory);
|
||||
this.relDataType = rowSignature.getRelDataType(typeFactory);
|
||||
}
|
||||
|
||||
RelDataType createSqlType(SqlTypeName sqlTypeName)
|
||||
{
|
||||
return typeFactory.createSqlType(sqlTypeName);
|
||||
}
|
||||
|
||||
RexNode makeInputRef(String columnName)
|
||||
{
|
||||
int columnNumber = rowSignature.getRowOrder().indexOf(columnName);
|
||||
return rexBuilder.makeInputRef(relDataType.getFieldList().get(columnNumber).getType(), columnNumber);
|
||||
}
|
||||
|
||||
RexNode getConstantNull()
|
||||
{
|
||||
return rexBuilder.constantNull();
|
||||
}
|
||||
|
||||
RexLiteral makeFlag(Enum flag)
|
||||
{
|
||||
return rexBuilder.makeFlag(flag);
|
||||
}
|
||||
|
||||
RexLiteral makeNullLiteral(SqlTypeName sqlTypeName)
|
||||
{
|
||||
return rexBuilder.makeNullLiteral(createSqlType(sqlTypeName));
|
||||
}
|
||||
|
||||
RexLiteral makeLiteral(String s)
|
||||
{
|
||||
return rexBuilder.makeLiteral(s);
|
||||
}
|
||||
|
||||
RexNode makeLiteral(DateTime timestamp)
|
||||
{
|
||||
return rexBuilder.makeTimestampLiteral(Calcites.jodaToCalciteTimestampString(timestamp, DateTimeZone.UTC), 0);
|
||||
}
|
||||
|
||||
RexNode makeLiteral(Integer integer)
|
||||
{
|
||||
return rexBuilder.makeLiteral(new BigDecimal(integer), createSqlType(SqlTypeName.INTEGER), true);
|
||||
}
|
||||
|
||||
RexNode makeLiteral(Long bigint)
|
||||
{
|
||||
return rexBuilder.makeLiteral(new BigDecimal(bigint), createSqlType(SqlTypeName.BIGINT), true);
|
||||
}
|
||||
|
||||
RexNode makeLiteral(BigDecimal bigDecimal)
|
||||
{
|
||||
return rexBuilder.makeExactLiteral(bigDecimal);
|
||||
}
|
||||
|
||||
RexNode makeLiteral(BigDecimal v, SqlIntervalQualifier intervalQualifier)
|
||||
{
|
||||
return rexBuilder.makeIntervalLiteral(v, intervalQualifier);
|
||||
}
|
||||
|
||||
RexNode makeCall(SqlOperator op, RexNode... exprs)
|
||||
{
|
||||
return rexBuilder.makeCall(op, exprs);
|
||||
}
|
||||
|
||||
RexNode makeAbstractCast(RelDataType type, RexNode exp)
|
||||
{
|
||||
return rexBuilder.makeAbstractCast(type, exp);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Representation of variable that is bound in an expression. Intended use is as one of
|
||||
* the args to {@link #buildExpectedExpression(String, Object...)}.
|
||||
*/
|
||||
Variable makeVariable(String name)
|
||||
{
|
||||
return new Variable(name);
|
||||
}
|
||||
|
||||
private static class Variable
|
||||
{
|
||||
private final String name;
|
||||
|
||||
Variable(String name)
|
||||
{
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "\"" + name + "\"";
|
||||
}
|
||||
}
|
||||
|
||||
DruidExpression buildExpectedExpression(String functionName, Object... args)
|
||||
{
|
||||
String noDelimiter = "";
|
||||
String argsString = Arrays.stream(args)
|
||||
.map(ExpressionTestHelper::quoteIfNeeded)
|
||||
.collect(Collectors.joining(","));
|
||||
List<String> elements = Arrays.asList(functionName, "(", argsString, ")");
|
||||
return DruidExpression.fromExpression(String.join(noDelimiter, elements));
|
||||
}
|
||||
|
||||
private static String quoteIfNeeded(@Nullable Object arg)
|
||||
{
|
||||
if (arg == null) {
|
||||
return "null";
|
||||
} else if (arg instanceof String) {
|
||||
return "'" + arg + "'";
|
||||
} else if (arg instanceof Boolean) {
|
||||
return (Boolean) arg ? "1" : "0";
|
||||
} else {
|
||||
return arg.toString();
|
||||
}
|
||||
}
|
||||
|
||||
void testExpression(
|
||||
SqlTypeName sqlTypeName,
|
||||
SqlOperator op,
|
||||
List<RexNode> exprs,
|
||||
DruidExpression expectedExpression,
|
||||
Object expectedResult
|
||||
)
|
||||
{
|
||||
RelDataType returnType = createSqlType(sqlTypeName);
|
||||
testExpression(rexBuilder.makeCall(returnType, op, exprs), expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
void testExpression(
|
||||
SqlOperator op,
|
||||
RexNode expr,
|
||||
DruidExpression expectedExpression,
|
||||
Object expectedResult
|
||||
)
|
||||
{
|
||||
testExpression(op, Collections.singletonList(expr), expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
void testExpression(
|
||||
SqlOperator op,
|
||||
List<? extends RexNode> exprs,
|
||||
DruidExpression expectedExpression,
|
||||
Object expectedResult
|
||||
)
|
||||
{
|
||||
testExpression(rexBuilder.makeCall(op, exprs), expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
void testExpression(
|
||||
RexNode rexNode,
|
||||
DruidExpression expectedExpression,
|
||||
Object expectedResult
|
||||
)
|
||||
{
|
||||
DruidExpression expression = Expressions.toDruidExpression(PLANNER_CONTEXT, rowSignature, rexNode);
|
||||
Assert.assertEquals("Expression for: " + rexNode, expectedExpression, expression);
|
||||
|
||||
ExprEval result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable())
|
||||
.eval(Parser.withMap(bindings));
|
||||
Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value());
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,269 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressMatchOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class IPv4AddressMatchExpressionTest extends ExpressionTestBase
|
||||
{
|
||||
private static final String IPV4 = "192.168.0.1";
|
||||
private static final long IPV4_LONG = 3232235521L;
|
||||
private static final String IPV4_UINT = "3232235521";
|
||||
private static final String IPV4_NETWORK = "192.168.0.0";
|
||||
private static final String IPV4_BROADCAST = "192.168.255.255";
|
||||
private static final String IPV6_COMPATIBLE = "::192.168.0.1";
|
||||
private static final String IPV6_MAPPED = "::ffff:192.168.0.1";
|
||||
private static final String SUBNET_192_168 = "192.168.0.0/16";
|
||||
private static final String SUBNET_10 = "10.0.0.0/8";
|
||||
private static final Object IGNORE_EXPECTED_RESULT = null;
|
||||
private static final long MATCH = 1L;
|
||||
private static final long NO_MATCH = 0L;
|
||||
|
||||
private static final String VAR = "s";
|
||||
private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.STRING).build();
|
||||
private static final Map<String, Object> BINDINGS = ImmutableMap.of(VAR, "foo");
|
||||
|
||||
private IPv4AddressMatchOperatorConversion target;
|
||||
private ExpressionTestHelper testHelper;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new IPv4AddressMatchOperatorConversion();
|
||||
testHelper = new ExpressionTestHelper(ROW_SIGNATURE, BINDINGS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooFewArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 2 arguments");
|
||||
|
||||
testExpression(
|
||||
Collections.emptyList(),
|
||||
buildExpectedExpression(),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooManyArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 2 arguments");
|
||||
|
||||
String address = IPV4;
|
||||
String subnet = SUBNET_192_168;
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(address),
|
||||
testHelper.makeLiteral(subnet),
|
||||
testHelper.makeLiteral(address)
|
||||
),
|
||||
buildExpectedExpression(address, subnet, address),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubnetArgNotLiteral()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "subnet arg must be a literal");
|
||||
|
||||
String address = IPV4;
|
||||
String variableName = VAR;
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(address),
|
||||
testHelper.makeInputRef(variableName)
|
||||
),
|
||||
buildExpectedExpression(address, testHelper.makeVariable(variableName)),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubnetArgInvalid()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "subnet arg has an invalid format");
|
||||
|
||||
String address = IPV4;
|
||||
String invalidSubnet = "192.168.0.1/invalid";
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(address),
|
||||
testHelper.makeLiteral(invalidSubnet)
|
||||
),
|
||||
buildExpectedExpression(address, invalidSubnet),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullArg()
|
||||
{
|
||||
String subnet = SUBNET_192_168;
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.getConstantNull(),
|
||||
testHelper.makeLiteral(subnet)
|
||||
),
|
||||
buildExpectedExpression(null, subnet),
|
||||
NO_MATCH
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidArgType()
|
||||
{
|
||||
String variableNameWithInvalidType = VAR;
|
||||
String subnet = SUBNET_192_168;
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeInputRef(variableNameWithInvalidType),
|
||||
testHelper.makeLiteral(subnet)
|
||||
),
|
||||
buildExpectedExpression(testHelper.makeVariable(variableNameWithInvalidType), subnet),
|
||||
NO_MATCH
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingStringArgIPv4()
|
||||
{
|
||||
testExpression(IPV4, SUBNET_192_168, MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotMatchingStringArgIPv4()
|
||||
{
|
||||
testExpression(IPV4, SUBNET_10, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingStringArgIPv6Mapped()
|
||||
{
|
||||
testExpression(IPV6_MAPPED, SUBNET_192_168, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotMatchingStringArgIPv6Mapped()
|
||||
{
|
||||
testExpression(IPV6_MAPPED, SUBNET_10, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingStringArgIPv6Compatible()
|
||||
{
|
||||
testExpression(IPV6_COMPATIBLE, SUBNET_192_168, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotMatchingStringArgIPv6Compatible()
|
||||
{
|
||||
testExpression(IPV6_COMPATIBLE, SUBNET_10, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotIpAddress()
|
||||
{
|
||||
testExpression("druid.apache.org", SUBNET_192_168, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingLongArg()
|
||||
{
|
||||
testExpression(IPV4_LONG, SUBNET_192_168, MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotMatchingLongArg()
|
||||
{
|
||||
testExpression(IPV4_LONG, SUBNET_10, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchingStringArgUnsignedInt()
|
||||
{
|
||||
testExpression(IPV4_UINT, SUBNET_192_168, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNotMatchingStringArgUnsignedInt()
|
||||
{
|
||||
testExpression(IPV4_UINT, SUBNET_10, NO_MATCH);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInclusive()
|
||||
{
|
||||
String subnet = SUBNET_192_168;
|
||||
testExpression(IPV4_NETWORK, subnet, MATCH);
|
||||
testExpression(IPV4, subnet, MATCH);
|
||||
testExpression(IPV4_BROADCAST, subnet, MATCH);
|
||||
}
|
||||
|
||||
private void testExpression(String address, String subnet, long match)
|
||||
{
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(address),
|
||||
testHelper.makeLiteral(subnet)
|
||||
),
|
||||
buildExpectedExpression(address, subnet),
|
||||
match
|
||||
);
|
||||
}
|
||||
|
||||
private void testExpression(long address, String subnet, long match)
|
||||
{
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(address),
|
||||
testHelper.makeLiteral(subnet)
|
||||
),
|
||||
buildExpectedExpression(address, subnet),
|
||||
match
|
||||
);
|
||||
}
|
||||
|
||||
private void testExpression(
|
||||
List<? extends RexNode> exprs,
|
||||
final DruidExpression expectedExpression,
|
||||
final Object expectedResult
|
||||
)
|
||||
{
|
||||
testHelper.testExpression(target.calciteOperator(), exprs, expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
private DruidExpression buildExpectedExpression(Object... args)
|
||||
{
|
||||
return testHelper.buildExpectedExpression(target.getDruidFunctionName(), args);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,235 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressParseOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class IPv4AddressParseExpressionTest extends ExpressionTestBase
|
||||
{
|
||||
private static final String VALID = "192.168.0.1";
|
||||
private static final long EXPECTED = 3232235521L;
|
||||
private static final Object IGNORE_EXPECTED_RESULT = null;
|
||||
private static final Long NULL = NullHandling.replaceWithDefault() ? NullHandling.ZERO_LONG : null;
|
||||
|
||||
private static final String VAR = "f";
|
||||
private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.FLOAT).build();
|
||||
private static final Map<String, Object> BINDINGS = ImmutableMap.of(VAR, 3.14);
|
||||
|
||||
private IPv4AddressParseOperatorConversion target;
|
||||
private ExpressionTestHelper testHelper;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new IPv4AddressParseOperatorConversion();
|
||||
testHelper = new ExpressionTestHelper(ROW_SIGNATURE, BINDINGS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooFewArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 1 argument");
|
||||
|
||||
testExpression(
|
||||
Collections.emptyList(),
|
||||
buildExpectedExpression(),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooManyArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 1 argument");
|
||||
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.getConstantNull(),
|
||||
testHelper.getConstantNull()
|
||||
),
|
||||
buildExpectedExpression(null, null),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullArg()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.getConstantNull(),
|
||||
buildExpectedExpression((String) null),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidArgType()
|
||||
{
|
||||
String variableNameWithInvalidType = VAR;
|
||||
testExpression(
|
||||
testHelper.makeInputRef(variableNameWithInvalidType),
|
||||
buildExpectedExpression(testHelper.makeVariable(variableNameWithInvalidType)),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidStringArgNotIPAddress()
|
||||
{
|
||||
String notIpAddress = "druid.apache.org";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(notIpAddress),
|
||||
buildExpectedExpression(notIpAddress),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidStringArgIPv6Compatible()
|
||||
{
|
||||
String ipv6Compatible = "::192.168.0.1";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(ipv6Compatible),
|
||||
buildExpectedExpression(ipv6Compatible),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgIPv6Mapped()
|
||||
{
|
||||
String ipv6Mapped = "::ffff:192.168.0.1";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(ipv6Mapped),
|
||||
buildExpectedExpression(ipv6Mapped),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgIPv4()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.makeLiteral(VALID),
|
||||
buildExpectedExpression(VALID),
|
||||
EXPECTED
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgUnsignedInt()
|
||||
{
|
||||
String unsignedInt = "3232235521";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(unsignedInt),
|
||||
buildExpectedExpression(unsignedInt),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidIntegerArgTooLow()
|
||||
{
|
||||
long tooLow = -1L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(tooLow),
|
||||
buildExpectedExpression(tooLow),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArgLowest()
|
||||
{
|
||||
long lowest = 0L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(lowest),
|
||||
buildExpectedExpression(lowest),
|
||||
lowest
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArg()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.makeLiteral(EXPECTED),
|
||||
buildExpectedExpression(EXPECTED),
|
||||
EXPECTED
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArgHighest()
|
||||
{
|
||||
long highest = 0xff_ff_ff_ffL;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(highest),
|
||||
buildExpectedExpression(highest),
|
||||
highest
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidIntegerArgTooHigh()
|
||||
{
|
||||
long tooHigh = 0x1_00_00_00_00L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(tooHigh),
|
||||
buildExpectedExpression(tooHigh),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
private void testExpression(
|
||||
RexNode expr,
|
||||
final DruidExpression expectedExpression,
|
||||
final Object expectedResult
|
||||
)
|
||||
{
|
||||
testExpression(Collections.singletonList(expr), expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
private void testExpression(
|
||||
List<? extends RexNode> exprs,
|
||||
final DruidExpression expectedExpression,
|
||||
final Object expectedResult
|
||||
)
|
||||
{
|
||||
testHelper.testExpression(target.calciteOperator(), exprs, expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
private DruidExpression buildExpectedExpression(Object... args)
|
||||
{
|
||||
return testHelper.buildExpectedExpression(target.getDruidFunctionName(), args);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,234 @@
|
|||
/*
|
||||
* 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.druid.sql.calcite.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressStringifyOperatorConversion;
|
||||
import org.apache.druid.sql.calcite.table.RowSignature;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class IPv4AddressStringifyExpressionTest extends ExpressionTestBase
|
||||
{
|
||||
private static final long VALID = 3232235521L;
|
||||
private static final String EXPECTED = "192.168.0.1";
|
||||
private static final Object IGNORE_EXPECTED_RESULT = null;
|
||||
private static final String NULL = null;
|
||||
|
||||
private static final String VAR = "f";
|
||||
private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.FLOAT).build();
|
||||
private static final Map<String, Object> BINDINGS = ImmutableMap.of(VAR, 3.14);
|
||||
|
||||
private IPv4AddressStringifyOperatorConversion target;
|
||||
private ExpressionTestHelper testHelper;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
target = new IPv4AddressStringifyOperatorConversion();
|
||||
testHelper = new ExpressionTestHelper(ROW_SIGNATURE, BINDINGS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooFewArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 1 argument");
|
||||
|
||||
testExpression(
|
||||
Collections.emptyList(),
|
||||
buildExpectedExpression(),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooManyArgs()
|
||||
{
|
||||
expectException(IllegalArgumentException.class, "must have 1 argument");
|
||||
|
||||
testExpression(
|
||||
Arrays.asList(
|
||||
testHelper.makeLiteral(VALID),
|
||||
testHelper.makeLiteral(VALID)
|
||||
),
|
||||
buildExpectedExpression(VALID, VALID),
|
||||
IGNORE_EXPECTED_RESULT
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNullArg()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.getConstantNull(),
|
||||
buildExpectedExpression((String) null),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidArgType()
|
||||
{
|
||||
String variableNameWithInvalidType = VAR;
|
||||
testExpression(
|
||||
testHelper.makeInputRef(variableNameWithInvalidType),
|
||||
buildExpectedExpression(testHelper.makeVariable(variableNameWithInvalidType)),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidIntegerArgTooLow()
|
||||
{
|
||||
long tooLow = -1L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(tooLow),
|
||||
buildExpectedExpression(tooLow),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArgLowest()
|
||||
{
|
||||
long lowest = 0L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(lowest),
|
||||
buildExpectedExpression(lowest),
|
||||
"0.0.0.0"
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArgHighest()
|
||||
{
|
||||
long highest = 0xff_ff_ff_ffL;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(highest),
|
||||
buildExpectedExpression(highest),
|
||||
"255.255.255.255"
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidIntegerArgTooHigh()
|
||||
{
|
||||
long tooHigh = 0x1_00_00_00_00L;
|
||||
testExpression(
|
||||
testHelper.makeLiteral(tooHigh),
|
||||
buildExpectedExpression(tooHigh),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidIntegerArg()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.makeLiteral(VALID),
|
||||
buildExpectedExpression(VALID),
|
||||
EXPECTED
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidStringArgNotIPAddress()
|
||||
{
|
||||
String notIpAddress = "druid.apache.org";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(notIpAddress),
|
||||
buildExpectedExpression(notIpAddress),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidStringArgIPv6Compatible()
|
||||
{
|
||||
String ipv6Compatible = "::192.168.0.1";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(ipv6Compatible),
|
||||
buildExpectedExpression(ipv6Compatible),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgIPv6Mapped()
|
||||
{
|
||||
String ipv6Mapped = "::ffff:192.168.0.1";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(ipv6Mapped),
|
||||
buildExpectedExpression(ipv6Mapped),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgIPv4()
|
||||
{
|
||||
testExpression(
|
||||
testHelper.makeLiteral(EXPECTED),
|
||||
buildExpectedExpression(EXPECTED),
|
||||
EXPECTED
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidStringArgUnsignedInt()
|
||||
{
|
||||
String unsignedInt = "3232235521";
|
||||
testExpression(
|
||||
testHelper.makeLiteral(unsignedInt),
|
||||
buildExpectedExpression(unsignedInt),
|
||||
NULL
|
||||
);
|
||||
}
|
||||
|
||||
private void testExpression(
|
||||
RexNode expr,
|
||||
final DruidExpression expectedExpression,
|
||||
final Object expectedResult
|
||||
)
|
||||
{
|
||||
testExpression(Collections.singletonList(expr), expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
private void testExpression(
|
||||
List<? extends RexNode> exprs,
|
||||
final DruidExpression expectedExpression,
|
||||
final Object expectedResult
|
||||
)
|
||||
{
|
||||
testHelper.testExpression(target.calciteOperator(), exprs, expectedExpression, expectedResult);
|
||||
}
|
||||
|
||||
private DruidExpression buildExpectedExpression(Object... args)
|
||||
{
|
||||
return testHelper.buildExpectedExpression(target.getDruidFunctionName(), args);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue