Skip to content

Latest commit

 

History

History
487 lines (467 loc) · 121 KB

velox-backend-support-progress.md

File metadata and controls

487 lines (467 loc) · 121 KB
layout title nav_order
page
Velox Backend's Supported Operators & Functions
4

The Operators and Functions Support Progress

Gluten is still under active development. Here is a list of supported operators and functions.

Since the same function may have different semantics between Presto and Spark, Velox implement the functions in Presto category, if we note a different semantics from Spark, then the function is implemented in Spark category. So Gluten will first try to find function in Velox's spark category, if a function isn't implemented then refer to Presto category.

The total number of functions in Spark3.3 is 387, Gluten supports 189 of them.

We use some notations to describe the supporting status of operators/functions in the tables below, they are:

Value Description
S Supported. Gluten or Velox supports fully.
S* Mark for foldable expression that will be converted to alias after spark's optimization.
[Blank Cell] Not applicable case or needs to confirm.
PS Partial Support. Velox only partially supports it.
NS Not Supported. Velox backend does not support it.

And also some notations for the function implementation's restrictions:

Value Description
Mismatched Some functions are implemented by Velox, but have different semantics from Apache Spark, we mark them as "Mismatched".
ANSI OFF Gluten doesn't support ANSI mode. If it is enabled, Gluten will fall back to Vanilla Spark.

Operator Map

Gluten supports 28 operators (Drag to right to see all data types)

Executor Description Gluten Name Velox Name BOOLEAN BYTE SHORT INT LONG FLOAT DOUBLE STRING NULL BINARY ARRAY MAP STRUCT(ROW) DATE TIMESTAMP DECIMAL CALENDAR UDT
FileSourceScanExec Reading data from files, often from Hive tables FileSourceScanExecTransformer TableScanNode S S S S S S S S S S NS NS NS S NS NS NS NS
BatchScanExec The backend for most file input BatchScanExecTransformer TableScanNode S S S S S S S S S S NS NS NS S NS NS NS NS
FilterExec The backend for most filter statements FilterExecTransformer FilterNode S S S S S S S S S S NS NS NS S NS NS NS NS
ProjectExec The backend for most select, withColumn and dropColumn statements ProjectExecTransformer ProjectNode S S S S S S S S S S NS NS NS S NS NS NS NS
HashAggregateExec The backend for hash based aggregations HashAggregateBaseTransformer AggregationNode S S S S S S S S S S NS NS NS S NS NS NS NS
BroadcastHashJoinExec Implementation of join using broadcast data BroadcastHashJoinExecTransformer HashJoinNode S S S S S S S S S S NS NS NS S NS NS NS NS
ShuffledHashJoinExec Implementation of join using hashed shuffled data ShuffleHashJoinExecTransformer HashJoinNode S S S S S S S S S S NS NS NS S NS NS NS NS
SortExec The backend for the sort operator SortExecTransformer OrderByNode S S S S S S S S S S NS NS NS S NS NS NS NS
SortMergeJoinExec Sort merge join, replacing with shuffled hash join SortMergeJoinExecTransformer MergeJoinNode S S S S S S S S S S NS NS NS S NS NS NS NS
WindowExec Window operator backend WindowExecTransformer WindowNode S S S S S S S S S S NS NS NS S NS NS NS NS
GlobalLimitExec Limiting of results across partitions LimitTransformer LimitNode S S S S S S S S S S NS NS NS S NS NS NS NS
LocalLimitExec Per-partition limiting of results LimitTransformer LimitNode S S S S S S S S S S NS NS NS S NS NS NS NS
ExpandExec The backend for the expand operator ExpandExecTransformer GroupIdNode S S S S S S S S S S NS NS NS S NS NS NS NS
UnionExec The backend for the union operator UnionExecTransformer N S S S S S S S S S S NS NS NS S NS NS NS NS
DataWritingCommandExec Writing data Y TableWriteNode S S S S S S S S S S S NS S S NS S NS NS
CartesianProductExec Implementation of join using brute force CartesianProductExecTransformer NestedLoopJoinNode S S S S S S S S S S NS NS NS S NS NS NS NS
ShuffleExchangeExec The backend for most data being exchanged between processes ColumnarShuffleExchangeExec ExchangeNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The unnest operation expands arrays and maps into separate columns N UnnestNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The top-n operation reorders a dataset based on one or more identified sort fields as well as a sorting order N TopNNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The partitioned output operation redistributes data based on zero or more distribution fields N PartitionedOutputNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The values operation returns specified data N ValuesNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
A receiving operation that merges multiple ordered streams to maintain orderedness N MergeExchangeNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
An operation that merges multiple ordered streams to maintain orderedness N LocalMergeNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
Partitions input data into multiple streams or combines data from multiple streams into a single stream N LocalPartitionNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The enforce single row operation checks that input contains at most one row and returns that row unmodified N EnforceSingleRowNode NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS NS
The assign unique id operation adds one column at the end of the input columns with unique value per row N AssignUniqueIdNode NS NS NS NS NS NS NS NS NS NS NS NS NS S S S S S
ReusedExchangeExec A wrapper for reused exchange to have different output ReusedExchangeExec N
CollectLimitExec Reduce to single partition and apply limit N N
BroadcastExchangeExec The backend for broadcast exchange of data Y Y S S S S S S S S S S NS NS NS S NS S NS NS
ObjectHashAggregateExec The backend for hash based aggregations supporting TypedImperativeAggregate functions N N
SortAggregateExec The backend for sort based aggregations N N
CoalesceExec Reduce the partition numbers CoalesceExecTransformer N
GenerateExec The backend for operations that generate more output rows than input rows like explode GenerateExecTransformer UnnestNode
RangeExec The backend for range operator N N
SampleExec The backend for the sample operator N N
SubqueryBroadcastExec Plan to collect and transform the broadcast key values Y Y S S S S S S S S S S NS NS NS S NS S NS NS
TakeOrderedAndProjectExec Take the first limit elements as defined by the sortOrder, and do projection if needed Y Y S S S S S S S S S S NS NS NS S NS S NS NS
CustomShuffleReaderExec A wrapper of shuffle query stage N N
InMemoryTableScanExec Implementation of InMemory Table Scan Y Y
BroadcastNestedLoopJoinExec Implementation of join using brute force. Full outer joins and joins where the broadcast side matches the join side (e.g.: LeftOuter with left broadcast) are not supported BroadcastNestedLoopJoinExecTransformer NestedLoopJoinNode S S S S S S S S S S NS NS NS S NS NS NS NS
AggregateInPandasExec The backend for an Aggregation Pandas UDF, this accelerates the data transfer between the Java process and the Python process N N
ArrowEvalPythonExec The backend of the Scalar Pandas UDFs. Accelerates the data transfer between the Java process and the Python process N N
FlatMapGroupsInPandasExec The backend for Flat Map Groups Pandas UDF, Accelerates the data transfer between the Java process and the Python process N N
MapInPandasExec The backend for Map Pandas Iterator UDF. Accelerates the data transfer between the Java process and the Python process N N
WindowInPandasExec The backend for Window Aggregation Pandas UDF, Accelerates the data transfer between the Java process and the Python process N N
HiveTableScanExec The Hive table scan operator. Column and partition pruning are both handled Y Y
InsertIntoHiveTable Command for writing data out to a Hive table Y Y
Velox2Row Convert Velox format to Row format Y Y S S S S S S S S NS S NS NS NS S S NS NS NS
Velox2Arrow Convert Velox format to Arrow format Y Y S S S S S S S S NS S S S S S NS S NS NS

Function support

Gluten supports 199 functions. (Drag to right to see all data types)

Cast function's support status

  • S: supported.
  • NS: not supported.
  • -: not accepted by Spark.
  • N/A: not applicable case, e.g., from type is as same as to type, where cast will not actually happen.
From \ To BOOLEAN BYTE SHORT INT LONG FLOAT DOUBLE DECIMAL DATE TIMESTAMP STRING BINARY ARRAY MAP STRUCT NULL
BOOLEAN N/A S S S S S S S - NS S - - - - -
BYTE S N/A S S S S S S - NS S S - - - -
SHORT S S N/A S S S S S - NS S S - - - -
INT S S S N/A S S S S - NS S S - - - -
LONG S S S S N/A S S S - NS S S - - - -
FLOAT S S S S S N/A S S - NS S - - - - -
DOUBLE S S S S S S N/A S - NS S - - - - -
DECIMAL S S S S S S S N/A - NS S - - - - -
DATE NS NS NS NS NS NS NS NS N/A NS NS - - - - -
TIMESTAMP NS NS NS NS NS NS NS NS NS N/A NS - - - - -
STRING S S S S S S S S NS NS N/A - - - - -
BINARY S S S S S S S S NS NS S N/A - - - -
ARRAY - - - - - - - - - - NS - N/A - - -
Map - - - - - - - - - - NS - - N/A - -
STRUCT - - - - - - - - - - NS - - - N/A -
NULL S S S S S S S S S NS S S S S S N/A

Other functions' support status

Spark Functions Velox/Presto Functions Velox/Spark functions Gluten Restrictions BOOLEAN BYTE SHORT INT LONG FLOAT DOUBLE DATE TIMESTAMP STRING DECIMAL NULL BINARY CALENDAR ARRAY MAP STRUCT UDT
! not S S S S S S S S S
!= neq S S S S S S S S S
% mod remainder S ANSI OFF S S S S S
& bitwise_and bitwise_and S
* multiply multiply S ANSI OFF S S S S S
+ plus add S ANSI OFF S S S S S
- minus subtract S ANSI OFF S S S S S
/ divide divide S ANSI OFF S S S S S
< lt lessthan S S S S S S S S S
<= lte lessthanorequa S S S S S S S S S
<=> equalnullsafe S
<> neq notequalto S S S S S S S S S
= S
== eq equalto S S S S S S S S S
> gt greaterthan S S S S S S S S S
>= gte greaterthanorequal S S S S S S S S S
^ bitwise_xor S S S S
| bitwise_or bitwise_or S S S S
|| S
~ bitwise_not S S S S
and S S S S S S S S S
between between between S S S S S S S S S S
bit_and bitwise_and_agg S S S S S S
bit_count bit_count bit_count S S S S S S
bit_get bit_get S S S S S
bit_or S
bit_xor bit_xor S
case S
div
getbit
if S
ifnull
in in S S S S S S S S S
isnan is_nan isnan S S S
isnotnull isnotnull S S S S S S S S S
isnull is_null isnull S S S S S S S S S
mod mod remainder S
negative negate unaryminus
not not S S S S S S S S S
nullif S
or S
positive S
when S
ascii ascii S S
base64
bin bin
bit_length S
btrim S
char, chr chr chr S S
char_length/character_length length length S S
character_lengt/char_length length length S S
chr, char chr chr S S
concat concat concat S S
concat_ws
contains contains
decode
elt
encode
endswith endsWith
find_in_set S S
format_number
format_string
initcap
instr instr S S
lcase, lower lower lower S S
left S S
length length length S S
levenshtein levenshtein S
locate strpos S Mismatched S
lower lower lower S S
lpad lpad S S
ltrim ltrim ltrim S S
octet_length
overlay overlay S
parse_url
position strpos
printf
repeat repeat S
replace replace replace S S
reverse reverse S S
right
rpad rpad S S
rtrim rtrim rtrim S S
sentences
soundex soundex S
space
split split split S
split_part split_part Mismatched
startswith startsWith
substr, substring substr substring S S
substring, substr substr substring S S
substring_index substring_index S
translate
trim trim trim S S
ucase, upper upper upper S S
unbase64
unhex S
upper, ucase upper upper S S
xpath
xpath_boolean
xpath_double
xpath_float
xpath_int
xpath_long
xpath_number
xpath_short
xpath_string
like like S S
regexp rlike S Lookaround unsupported S
regexp_extract regexp_extract regexp_extract S Lookaround unsupported S
regexp_extract_all regexp_extract_all S Lookaround unsupported S
regexp_like regexp_like rlike S Lookaround unsupported S
regexp_replace regexp_replace S Lookaround unsupported S
rlike rlike S Lookaround unsupported S
abs abs abs S ANSI OFF S S S S S S
acos acos S S S S S S S
acosh acosh S S S S S S S
asin asin S S S S S S S
asinh asinh S S S S S S S
atan atan S S S S S S S
atan2 atan2 S S S S S S S
atanh atanh S S S S S S S
bround
cbrt cbrt S S S S S S S
ceil ceil ceil S S S S S S S
ceiling ceiling S S S S S S S
conv
cos cos S S S S S S S
cosh cosh S S S S S S S
cot
degrees degrees S S S S S S S
e e S S S S S S S
exp exp exp S S S S S S S
expm1 S
factorial
floor floor floor S S S S S S S
hex S S S S
hypot
ln ln S S S S S S S
log ln log S S S S S S S
log10 log10 S S S S S S S
log1p
log2 log2 S S S S S S S
pi pi S S S S S S S
pmod pmod S ANSI OFF S S S S S
pow, power pow,power power S S S S S
power, pow power,pow power S S S S S S S
radians radians S S S S S S S
rand rand rand S
rand rand rand
random random S
rint rint S
round round round S S S S S S S
shiftleft bitwise_left_shift shiftleft S S S S S S S
shiftright bitwise_right_shift shiftright S S S S S S S
shiftrightunsigned
sign, signum sign S S S S S S S
signum, sign sign S S S S S S S
sin sin S S S S S S S
sinh sinh
sqrt sqrt S S S S S S S
tan tan S S S S S S S
tanh tanh S S S S S S S
width_bucket width_bucket width_bucket S
array array S S
aggregate aggregate reduce S S
array_contains array_contains S
array_distinct array_distinct S S
array_except array_except S S
array_intersect array_intersect array_intersect S
array_join array_join S
array_max array_max S
array_min array_min S
array_position array_position S S
array_remove array_remove S
array_repeat S S S S S S S S S S S S
array_sort array_sort array_sort S
array_union
arrays_overlap array_overlap S
arrays_zip zip S
cardinality cardinality
element_at element_at element_at S S S
exists any_match S
explode, explode_outer
explode_outer, explode
filter filter filter PS
forall all_match S
flatten flatten flatten S
map map map S
map_concat map_concat
map_entries map_entries
map_filter map_filter map_filter
get_map_value element_at S S
map_from_arrays map_from_arrays S S
map_from_entries map_from_entries
map_keys map_keys map_keys S
map_values map_values map_values S S
map_zip_with map_zip_with S S
mask mask S
named_struct,struct row_construct named_struct S S
posexplode_outer,posexplode
sequence
shuffle shuffle shuffle S
size size S
array_size S
slice slice
sort_array sort_array S
str_to_map S
transform transform transofrm
transform_keys transform_keys PS
transform_values transform_values PS
zip_with zip_with S
add_months S
current_date S*
current_timestamp S*
current_timezone S*
date date S
date_add date_add date_add S S S S S S
date_format date_format S S S
date_from_unix_date S
date_part
date_sub S
date_trunc date_trunc
datediff date_diff S S S
day day S S S
dayofmonth day_of_month S S S
dayofweek day_of_week,dow S S S
dayofyear day_of_year,doy S S S
extract S S
from_unixtime from_unixtime S S
from_utc_timestamp from_utc_timestamp S
hour hour S S
last_day last_day S
make_date make_date S
make_dt_interval
make_interval
make_timestamp
make_ym_interval
minute minute S
month month S S S
months_between
next_day S S
now S S S
quarter quarter S S S
second second S
session_window
timestamp
timestamp_micros timestamp_micros S
timestamp_millis timestamp_millis S
timestamp_seconds
to_date S S S
to_timestamp
to_unix_timestamp to_unixtime to_unix_timestamp S
to_utc_timestamp
trunc
unix_timestamp unix_timestamp
unix_seconds unix_seconds S
unix_millis unix_millis S
unix_micros unix_micros S
weekday S S
weekofyear week,week_of_year S
window
year year year S S S
aggregate aggregate S
any
approx_count_distinct approx_distinct S S S S S S S S S S
approx_percentile
avg avg S ANSI OFF S S S S S
bool_and
bool_or
collect_list S
collect_set S
corr corr S S S S S S
count count S S S S S S
count_if count_if S S S S S
count_min_sketch
covar_pop covar_pop S S S S S S
covar_samp covar_samp S S S S S S
every
first first S
first_value first_value S
grouping
grouping_id
kurtosis kurtosis kurtosis S S S S S S
last last S
last_value last_value S
max max S S S S S S
max_by S
mean avg S ANSI OFF
min min S S S S S S
min_by S
regr_avgx regr_avgx regr_avgx S S S S S S
regr_avgy regr_avgy regr_avgy S S S S S S
regr_count regr_count regr_count S S S S S S
regr_r2 regr_r2 regr_r2 S S S S S S
regr_intercept regr_intercept regr_intercept S S S S S S
regr_slope regr_slope regr_slope S S S S S S
regr_sxy regr_sxy regr_sxy S S S S S S
regr_sxx regr_sxx regr_sxx S S S S S S
regr_syy regr_syy regr_syy S S S S S S
skewness skewness skewness S S S S S S
some
std,stddev stddev S S S S S S
stddev,std stddev S S S S S S
stddev_pop stddev_pop S S S S S S
stddev_samp stddev_samp S S S S S S
sum sum S ANSI OFF S S S S S
var_pop var_pop S S S S S S
var_samp var_samp S S S S S S
variance variance S S S S S S
cume_dist cume_dist S
dense_rank dense_rank S
lag S
lead S
nth_value nth_value nth_value PS
ntile ntile ntile S
percent_rank percent_rank S
rank rank S
row_number row_number S S S S
from_csv
from_json
get_json_object json_extract_scalar get_json_object S S
json_array_length json_array_length S S
json_tuple
schema_of_csv
schema_of_json
to_csv
to_json
assert_true S
coalesce PS
crc32 crc32 S S
current_user S* S
current_catalog S
current_database S
greatest greatest greatest S S S S S S
hash hash hash S S S S S S S S
inline
inline_outer
input_file_name S
input_file_block_length S
input_file_block_start S
java_method
least least least S S S S S S
md5 md5 S S
monotonically_increasing_id S
nanvl S
nvl
nvl2
raise_error raise_error S
reflect
sha S S
sha1 sha1 sha1 S S
sha2 sha2 S S
spark_partition_id S
stack S S S S S S S S S S S S S S S S S S S
xxhash64 xxhash64 xxhash64
uuid uuid uuid S
rand rand rand S
try_add S
try_substract S
try_multiply S
try_divide S