-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Adds an abstract class BeamSqlUdaf for defining Calcite SQL UDAFs. - Updates built-in COUNT/SUM/AVG/MAX/MIN accumulators to use this new class.
- Loading branch information
Showing
6 changed files
with
633 additions
and
523 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
72 changes: 72 additions & 0 deletions
72
dsls/sql/src/main/java/org/apache/beam/dsls/sql/schema/BeamSqlUdaf.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* 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.beam.dsls.sql.schema; | ||
|
||
import java.io.Serializable; | ||
import java.lang.reflect.ParameterizedType; | ||
import org.apache.beam.sdk.coders.CannotProvideCoderException; | ||
import org.apache.beam.sdk.coders.Coder; | ||
import org.apache.beam.sdk.coders.CoderRegistry; | ||
import org.apache.beam.sdk.transforms.Combine.CombineFn; | ||
|
||
/** | ||
* abstract class of aggregation functions in Beam SQL. | ||
* | ||
* <p>There're several constrains for a UDAF:<br> | ||
* 1. A constructor with an empty argument list is required;<br> | ||
* 2. The type of {@code InputT} and {@code OutputT} can only be Interger/Long/Short/Byte/Double | ||
* /Float/Date/BigDecimal, mapping as SQL type INTEGER/BIGINT/SMALLINT/TINYINE/DOUBLE/FLOAT | ||
* /TIMESTAMP/DECIMAL;<br> | ||
* 3. Keep intermediate data in {@code AccumT}, and do not rely on elements in class;<br> | ||
*/ | ||
public abstract class BeamSqlUdaf<InputT, AccumT, OutputT> implements Serializable { | ||
public BeamSqlUdaf(){} | ||
|
||
/** | ||
* create an initial aggregation object, equals to {@link CombineFn#createAccumulator()}. | ||
*/ | ||
public abstract AccumT init(); | ||
|
||
/** | ||
* add an input value, equals to {@link CombineFn#addInput(Object, Object)}. | ||
*/ | ||
public abstract AccumT add(AccumT accumulator, InputT input); | ||
|
||
/** | ||
* merge aggregation objects from parallel tasks, equals to | ||
* {@link CombineFn#mergeAccumulators(Iterable)}. | ||
*/ | ||
public abstract AccumT merge(Iterable<AccumT> accumulators); | ||
|
||
/** | ||
* extract output value from aggregation object, equals to | ||
* {@link CombineFn#extractOutput(Object)}. | ||
*/ | ||
public abstract OutputT result(AccumT accumulator); | ||
|
||
/** | ||
* get the coder for AccumT which stores the intermediate result. | ||
* By default it's fetched from {@link CoderRegistry}. | ||
*/ | ||
public Coder<AccumT> getAccumulatorCoder(CoderRegistry registry) | ||
throws CannotProvideCoderException { | ||
return registry.getCoder( | ||
(Class<AccumT>) ((ParameterizedType) getClass() | ||
.getGenericSuperclass()).getActualTypeArguments()[1]); | ||
} | ||
} |
Oops, something went wrong.