Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.flink.table.data.utils;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.TableException;

import javax.annotation.Nullable;

/**
* Interface to model a function that performs the casting of a value from one type to another.
*
* @param <IN> Input internal type
* @param <OUT> Output internal type
*/
@Internal
@FunctionalInterface
public interface CastExecutor<IN, OUT> {
/**
* Cast the input value. The output is null only and only if the input is null. The method
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Imho we should add a TODO and link to https://issues.apache.org/jira/browse/FLINK-24385, since only then normal cast would throw errors.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we should, because TRY_CAST is the user interface. This interface is just a detailed explanation of how internally we need to implement the casting logic, and every new implementation should be compliant to such interface (that is, when porting to these new interfaces, we need to take care of this difference)

* throws an exception if something goes wrong when casting.
*
* @param value Input value.
*/
@Nullable
OUT cast(@Nullable IN value) throws TableException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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.flink.table.planner.functions.casting;

import org.apache.flink.annotation.Internal;

import java.util.Objects;

/**
* Generated cast code block result. This POJO contains the Java code of the block performing the
* cast, the output is null term and the output variable containing the cast result. It is
* guaranteed that the {@code returnTerm} and {@code isNullTerm} can be accessed within the external
* scope of the returned {@code code}. Both {@code returnTerm} and {@code isNullTerm} can be either
* variable names or expressions.
*/
@Internal
public class CastCodeBlock {

private final String code;
private final String returnTerm;
private final String isNullTerm;

public CastCodeBlock(String code, String returnTerm, String isNullTerm) {
this.code = code;
this.returnTerm = returnTerm;
this.isNullTerm = isNullTerm;
}

public String getCode() {
return code;
}

public String getReturnTerm() {
return returnTerm;
}

public String getIsNullTerm() {
return isNullTerm;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
CastCodeBlock that = (CastCodeBlock) o;
return Objects.equals(code, that.code)
&& Objects.equals(returnTerm, that.returnTerm)
&& Objects.equals(isNullTerm, that.isNullTerm);
}

@Override
public int hashCode() {
return Objects.hash(code, returnTerm, isNullTerm);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.flink.table.planner.functions.casting;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.data.utils.CastExecutor;
import org.apache.flink.table.types.logical.LogicalType;

import java.time.ZoneId;

/**
* A {@link CastRule} provides the logic to create a {@link CastExecutor} starting from the input
* and the target types. A rule is matched using {@link CastRulePredicate}.
*
* @param <IN> Input internal type
* @param <OUT> Output internal type
*/
@Internal
public interface CastRule<IN, OUT> {

/** @see CastRulePredicate for more details about a cast rule predicate definition */
CastRulePredicate getPredicateDefinition();

/**
* Create a {@link CastExecutor} starting from the provided input type. The returned {@link
* CastExecutor} assumes the input value is using the internal data type, and it's a valid value
* for the provided {@code targetLogicalType}.
*/
CastExecutor<IN, OUT> create(
Context context, LogicalType inputLogicalType, LogicalType targetLogicalType);

/** Casting context. */
interface Context {
ZoneId getSessionZoneId();

ClassLoader getClassLoader();

/** Create a casting context. */
static Context create(ZoneId zoneId, ClassLoader classLoader) {
return new Context() {
@Override
public ZoneId getSessionZoneId() {
return zoneId;
}

@Override
public ClassLoader getClassLoader() {
return classLoader;
}
};
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
* 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.flink.table.planner.functions.casting;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.LogicalTypeFamily;
import org.apache.flink.table.types.logical.LogicalTypeRoot;

import javax.annotation.Nullable;

import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.function.BiPredicate;

/**
* In order to apply a {@link CastRule}, the runtime checks if a particular rule matches the tuple
* of input and target type using this class. In particular, a rule is applied if:
*
* <ol>
* <li>{@link #getTargetTypes()} includes the {@link LogicalTypeRoot} of target type and either
* <ol>
* <li>{@link #getInputTypes()} includes the {@link LogicalTypeRoot} of input type or
* <li>{@link #getInputTypeFamilies()} includes one of the {@link LogicalTypeFamily} of
* input type
* </ol>
* <li>Or {@link #getTargetTypeFamilies()} includes one of the {@link LogicalTypeFamily} of target
* type and either
* <ol>
* <li>{@link #getInputTypes()} includes the {@link LogicalTypeRoot} of input type or
* <li>{@link #getInputTypeFamilies()} includes one of the {@link LogicalTypeFamily} of
* input type
* </ol>
* <li>Or, if {@link #getCustomPredicate()} is not null, the input {@link LogicalType} and target
* {@link LogicalType} matches the predicate.
* </ol>
*
* <p>The {@code customPredicate} should be used in cases where {@link LogicalTypeRoot} and {@link
* LogicalTypeFamily} are not enough to identify whether a rule is applicable or not, for example
* when the matching depends on a field of the provided input {@link LogicalType} instance.
*/
@Internal
public class CastRulePredicate {

private final Set<LogicalTypeRoot> inputTypes;
private final Set<LogicalTypeRoot> targetTypes;

private final Set<LogicalTypeFamily> inputTypeFamilies;
private final Set<LogicalTypeFamily> targetTypeFamilies;

private final BiPredicate<LogicalType, LogicalType> customPredicate;

private CastRulePredicate(
Set<LogicalTypeRoot> inputTypes,
Set<LogicalTypeRoot> targetTypes,
Set<LogicalTypeFamily> inputTypeFamilies,
Set<LogicalTypeFamily> targetTypeFamilies,
BiPredicate<LogicalType, LogicalType> customPredicate) {
this.inputTypes = inputTypes;
this.targetTypes = targetTypes;
this.inputTypeFamilies = inputTypeFamilies;
this.targetTypeFamilies = targetTypeFamilies;
this.customPredicate = customPredicate;
}

public Set<LogicalTypeRoot> getInputTypes() {
return inputTypes;
}

public Set<LogicalTypeRoot> getTargetTypes() {
return targetTypes;
}

public Set<LogicalTypeFamily> getInputTypeFamilies() {
return inputTypeFamilies;
}

public Set<LogicalTypeFamily> getTargetTypeFamilies() {
return targetTypeFamilies;
}

public @Nullable BiPredicate<LogicalType, LogicalType> getCustomPredicate() {
return customPredicate;
}

public static Builder builder() {
return new Builder();
}

/** Builder for the {@link CastRulePredicate}. */
public static class Builder {
private final Set<LogicalTypeRoot> inputTypes = new HashSet<>();
private final Set<LogicalTypeRoot> targetTypes = new HashSet<>();

private final Set<LogicalTypeFamily> inputTypeFamilies = new HashSet<>();
private final Set<LogicalTypeFamily> targetTypeFamilies = new HashSet<>();

private BiPredicate<LogicalType, LogicalType> customPredicate;

public Builder input(LogicalTypeRoot inputType) {
inputTypes.add(inputType);
return this;
}

public Builder target(LogicalTypeRoot outputType) {
targetTypes.add(outputType);
return this;
}

public Builder input(LogicalTypeFamily inputTypeFamily) {
inputTypeFamilies.add(inputTypeFamily);
return this;
}

public Builder target(LogicalTypeFamily outputTypeFamily) {
targetTypeFamilies.add(outputTypeFamily);
return this;
}

public Builder predicate(BiPredicate<LogicalType, LogicalType> customPredicate) {
this.customPredicate = customPredicate;
return this;
}

public CastRulePredicate build() {
return new CastRulePredicate(
Collections.unmodifiableSet(inputTypes),
Collections.unmodifiableSet(targetTypes),
Collections.unmodifiableSet(inputTypeFamilies),
Collections.unmodifiableSet(targetTypeFamilies),
customPredicate);
}
}
}
Loading