forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-28416][table] Add (Async)LookupFunction and providers in repla…
…ce of (Async)TableFunction as the API for lookup table (apache#20177)
- Loading branch information
1 parent
aef75be
commit 3e2620b
Showing
7 changed files
with
210 additions
and
4 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
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
36 changes: 36 additions & 0 deletions
36
...main/java/org/apache/flink/table/connector/source/lookup/AsyncLookupFunctionProvider.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,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 | ||
* | ||
* https://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.connector.source.lookup; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.table.connector.source.LookupTableSource; | ||
import org.apache.flink.table.functions.AsyncLookupFunction; | ||
|
||
/** A provider for creating {@link AsyncLookupFunction}. */ | ||
@PublicEvolving | ||
public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider { | ||
|
||
/** Helper function for creating a static provider. */ | ||
static AsyncLookupFunctionProvider of(AsyncLookupFunction asyncLookupFunction) { | ||
return () -> asyncLookupFunction; | ||
} | ||
|
||
/** Creates an {@link AsyncLookupFunction} instance. */ | ||
AsyncLookupFunction createAsyncLookupFunction(); | ||
} |
36 changes: 36 additions & 0 deletions
36
.../src/main/java/org/apache/flink/table/connector/source/lookup/LookupFunctionProvider.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,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 | ||
* | ||
* https://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.connector.source.lookup; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.table.connector.source.LookupTableSource; | ||
import org.apache.flink.table.functions.LookupFunction; | ||
|
||
/** A provider for creating {@link LookupFunction}. */ | ||
@PublicEvolving | ||
public interface LookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider { | ||
|
||
/** Helper function for creating a static provider. */ | ||
static LookupFunctionProvider of(LookupFunction lookupFunction) { | ||
return () -> lookupFunction; | ||
} | ||
|
||
/** Creates an {@link LookupFunction} instance. */ | ||
LookupFunction createLookupFunction(); | ||
} |
64 changes: 64 additions & 0 deletions
64
...link-table-common/src/main/java/org/apache/flink/table/functions/AsyncLookupFunction.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,64 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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.functions; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.table.api.TableException; | ||
import org.apache.flink.table.data.GenericRowData; | ||
import org.apache.flink.table.data.RowData; | ||
|
||
import java.util.Collection; | ||
import java.util.concurrent.CompletableFuture; | ||
|
||
/** | ||
* A wrapper class of {@link AsyncTableFunction} for asynchronously lookup rows matching the lookup | ||
* keys from external system. | ||
* | ||
* <p>The output type of this table function is fixed as {@link RowData}. | ||
*/ | ||
@PublicEvolving | ||
public abstract class AsyncLookupFunction extends AsyncTableFunction<RowData> { | ||
|
||
/** | ||
* Asynchronously lookup rows matching the lookup keys. | ||
* | ||
* @param keyRow - A {@link RowData} that wraps lookup keys. | ||
* @return A collection of all matching rows in the lookup table. | ||
*/ | ||
public abstract CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow); | ||
|
||
/** Invokes {@link #asyncLookup} and chains futures. */ | ||
public final void eval(CompletableFuture<Collection<RowData>> future, Object... keys) { | ||
GenericRowData keyRow = GenericRowData.of(keys); | ||
asyncLookup(keyRow) | ||
.whenCompleteAsync( | ||
(result, exception) -> { | ||
if (exception != null) { | ||
future.completeExceptionally( | ||
new TableException( | ||
String.format( | ||
"Failed to asynchronously lookup entries with key '%s'", | ||
keyRow), | ||
exception)); | ||
return; | ||
} | ||
future.complete(result); | ||
}); | ||
} | ||
} |
59 changes: 59 additions & 0 deletions
59
...ble/flink-table-common/src/main/java/org/apache/flink/table/functions/LookupFunction.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,59 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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.functions; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.table.data.GenericRowData; | ||
import org.apache.flink.table.data.RowData; | ||
|
||
import java.io.IOException; | ||
import java.util.Collection; | ||
|
||
/** | ||
* A wrapper class of {@link TableFunction} for synchronously lookup rows matching the lookup keys | ||
* from external system. | ||
* | ||
* <p>The output type of this table function is fixed as {@link RowData}. | ||
*/ | ||
@PublicEvolving | ||
public abstract class LookupFunction extends TableFunction<RowData> { | ||
|
||
/** | ||
* Synchronously lookup rows matching the lookup keys. | ||
* | ||
* @param keyRow - A {@link RowData} that wraps lookup keys. | ||
* @return A collection of all matching rows in the lookup table. | ||
*/ | ||
public abstract Collection<RowData> lookup(RowData keyRow) throws IOException; | ||
|
||
/** Invoke {@link #lookup} and handle exceptions. */ | ||
public final void eval(Object... keys) { | ||
GenericRowData keyRow = GenericRowData.of(keys); | ||
try { | ||
Collection<RowData> lookup = lookup(keyRow); | ||
if (lookup == null) { | ||
return; | ||
} | ||
lookup.forEach(this::collect); | ||
} catch (IOException e) { | ||
throw new RuntimeException( | ||
String.format("Failed to lookup values with given key row '%s'", keyRow), e); | ||
} | ||
} | ||
} |