1
0

[HUDI-3665] Support flink multiple versions (#5072)

This commit is contained in:
Danny Chan
2022-03-21 10:34:50 +08:00
committed by GitHub
parent 15d1c18625
commit 799c78e688
193 changed files with 2264 additions and 629 deletions

View File

@@ -0,0 +1,90 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hudi-flink-datasource</artifactId>
<groupId>org.apache.hudi</groupId>
<version>0.11.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hudi-flink1.13.x</artifactId>
<version>0.11.0-SNAPSHOT</version>
<packaging>jar</packaging>
<properties>
<main.basedir>${project.parent.parent.basedir}</main.basedir>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
<version>${flink1.13.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink1.13.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${flink1.13.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<version>${flink1.13.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
<phase>test-compile</phase>
</execution>
</executions>
<configuration>
<skip>false</skip>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>

View File

@@ -0,0 +1,262 @@
/*
* 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;
import org.apache.flink.table.data.binary.TypedSetters;
import org.apache.flink.table.data.vector.ArrayColumnVector;
import org.apache.flink.table.data.vector.BooleanColumnVector;
import org.apache.flink.table.data.vector.ByteColumnVector;
import org.apache.flink.table.data.vector.BytesColumnVector;
import org.apache.flink.table.data.vector.ColumnVector;
import org.apache.flink.table.data.vector.DecimalColumnVector;
import org.apache.flink.table.data.vector.DoubleColumnVector;
import org.apache.flink.table.data.vector.FloatColumnVector;
import org.apache.flink.table.data.vector.IntColumnVector;
import org.apache.flink.table.data.vector.LongColumnVector;
import org.apache.flink.table.data.vector.MapColumnVector;
import org.apache.flink.table.data.vector.RowColumnVector;
import org.apache.flink.table.data.vector.ShortColumnVector;
import org.apache.flink.table.data.vector.TimestampColumnVector;
import java.util.Arrays;
/**
* Columnar array to support access to vector column data.
*
* <p>References {@code org.apache.flink.table.data.ColumnarArrayData} to include FLINK-15390.
*/
public final class ColumnarArrayData implements ArrayData, TypedSetters {
private final ColumnVector data;
private final int offset;
private final int numElements;
public ColumnarArrayData(ColumnVector data, int offset, int numElements) {
this.data = data;
this.offset = offset;
this.numElements = numElements;
}
@Override
public int size() {
return numElements;
}
@Override
public boolean isNullAt(int pos) {
return data.isNullAt(offset + pos);
}
@Override
public void setNullAt(int pos) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public boolean getBoolean(int pos) {
return ((BooleanColumnVector) data).getBoolean(offset + pos);
}
@Override
public byte getByte(int pos) {
return ((ByteColumnVector) data).getByte(offset + pos);
}
@Override
public short getShort(int pos) {
return ((ShortColumnVector) data).getShort(offset + pos);
}
@Override
public int getInt(int pos) {
return ((IntColumnVector) data).getInt(offset + pos);
}
@Override
public long getLong(int pos) {
return ((LongColumnVector) data).getLong(offset + pos);
}
@Override
public float getFloat(int pos) {
return ((FloatColumnVector) data).getFloat(offset + pos);
}
@Override
public double getDouble(int pos) {
return ((DoubleColumnVector) data).getDouble(offset + pos);
}
@Override
public StringData getString(int pos) {
BytesColumnVector.Bytes byteArray = getByteArray(pos);
return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len);
}
@Override
public DecimalData getDecimal(int pos, int precision, int scale) {
return ((DecimalColumnVector) data).getDecimal(offset + pos, precision, scale);
}
@Override
public TimestampData getTimestamp(int pos, int precision) {
return ((TimestampColumnVector) data).getTimestamp(offset + pos, precision);
}
@Override
public <T> RawValueData<T> getRawValue(int pos) {
throw new UnsupportedOperationException("RawValueData is not supported.");
}
@Override
public byte[] getBinary(int pos) {
BytesColumnVector.Bytes byteArray = getByteArray(pos);
if (byteArray.len == byteArray.data.length) {
return byteArray.data;
} else {
return Arrays.copyOfRange(byteArray.data, byteArray.offset, byteArray.len);
}
}
@Override
public ArrayData getArray(int pos) {
return ((ArrayColumnVector) data).getArray(offset + pos);
}
@Override
public MapData getMap(int pos) {
return ((MapColumnVector) data).getMap(offset + pos);
}
@Override
public RowData getRow(int pos, int numFields) {
return ((RowColumnVector) data).getRow(offset + pos);
}
@Override
public void setBoolean(int pos, boolean value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setByte(int pos, byte value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setShort(int pos, short value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setInt(int pos, int value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setLong(int pos, long value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setFloat(int pos, float value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setDouble(int pos, double value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setDecimal(int pos, DecimalData value, int precision) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setTimestamp(int pos, TimestampData value, int precision) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public boolean[] toBooleanArray() {
boolean[] res = new boolean[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getBoolean(i);
}
return res;
}
@Override
public byte[] toByteArray() {
byte[] res = new byte[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getByte(i);
}
return res;
}
@Override
public short[] toShortArray() {
short[] res = new short[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getShort(i);
}
return res;
}
@Override
public int[] toIntArray() {
int[] res = new int[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getInt(i);
}
return res;
}
@Override
public long[] toLongArray() {
long[] res = new long[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getLong(i);
}
return res;
}
@Override
public float[] toFloatArray() {
float[] res = new float[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getFloat(i);
}
return res;
}
@Override
public double[] toDoubleArray() {
double[] res = new double[numElements];
for (int i = 0; i < numElements; i++) {
res[i] = getDouble(i);
}
return res;
}
private BytesColumnVector.Bytes getByteArray(int pos) {
return ((BytesColumnVector) data).getBytes(offset + pos);
}
}

View File

@@ -0,0 +1,71 @@
/*
* 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;
import org.apache.flink.table.data.vector.ColumnVector;
/**
* Columnar map to support access to vector column data.
*
* <p>Referenced from flink 1.14.0 {@code org.apache.flink.table.data.ColumnarMapData}.
*/
public final class ColumnarMapData implements MapData {
private final ColumnVector keyColumnVector;
private final ColumnVector valueColumnVector;
private final int offset;
private final int numElements;
public ColumnarMapData(
ColumnVector keyColumnVector,
ColumnVector valueColumnVector,
int offset,
int numElements) {
this.keyColumnVector = keyColumnVector;
this.valueColumnVector = valueColumnVector;
this.offset = offset;
this.numElements = numElements;
}
@Override
public int size() {
return numElements;
}
@Override
public ArrayData keyArray() {
return new ColumnarArrayData(keyColumnVector, offset, numElements);
}
@Override
public ArrayData valueArray() {
return new ColumnarArrayData(valueColumnVector, offset, numElements);
}
@Override
public boolean equals(Object o) {
throw new UnsupportedOperationException(
"ColumnarMapData do not support equals, please compare fields one by one!");
}
@Override
public int hashCode() {
throw new UnsupportedOperationException(
"ColumnarMapData do not support hashCode, please hash fields one by one!");
}
}

View File

@@ -0,0 +1,223 @@
/*
* 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;
import org.apache.flink.table.data.binary.TypedSetters;
import org.apache.flink.table.data.vector.BytesColumnVector.Bytes;
import org.apache.flink.table.data.vector.VectorizedColumnBatch;
import org.apache.flink.types.RowKind;
/**
* Columnar row to support access to vector column data.
* It is a row view in {@link VectorizedColumnBatch}.
*
* <p>References {@code org.apache.flink.table.data.ColumnarRowData} to include FLINK-15390.
*/
public final class ColumnarRowData implements RowData, TypedSetters {
private RowKind rowKind = RowKind.INSERT;
private VectorizedColumnBatch vectorizedColumnBatch;
private int rowId;
public ColumnarRowData() {
}
public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch) {
this(vectorizedColumnBatch, 0);
}
public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch, int rowId) {
this.vectorizedColumnBatch = vectorizedColumnBatch;
this.rowId = rowId;
}
public void setVectorizedColumnBatch(VectorizedColumnBatch vectorizedColumnBatch) {
this.vectorizedColumnBatch = vectorizedColumnBatch;
this.rowId = 0;
}
public void setRowId(int rowId) {
this.rowId = rowId;
}
@Override
public RowKind getRowKind() {
return rowKind;
}
@Override
public void setRowKind(RowKind kind) {
this.rowKind = kind;
}
@Override
public int getArity() {
return vectorizedColumnBatch.getArity();
}
@Override
public boolean isNullAt(int pos) {
return vectorizedColumnBatch.isNullAt(rowId, pos);
}
@Override
public boolean getBoolean(int pos) {
return vectorizedColumnBatch.getBoolean(rowId, pos);
}
@Override
public byte getByte(int pos) {
return vectorizedColumnBatch.getByte(rowId, pos);
}
@Override
public short getShort(int pos) {
return vectorizedColumnBatch.getShort(rowId, pos);
}
@Override
public int getInt(int pos) {
return vectorizedColumnBatch.getInt(rowId, pos);
}
@Override
public long getLong(int pos) {
return vectorizedColumnBatch.getLong(rowId, pos);
}
@Override
public float getFloat(int pos) {
return vectorizedColumnBatch.getFloat(rowId, pos);
}
@Override
public double getDouble(int pos) {
return vectorizedColumnBatch.getDouble(rowId, pos);
}
@Override
public StringData getString(int pos) {
Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos);
return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len);
}
@Override
public DecimalData getDecimal(int pos, int precision, int scale) {
return vectorizedColumnBatch.getDecimal(rowId, pos, precision, scale);
}
@Override
public TimestampData getTimestamp(int pos, int precision) {
return vectorizedColumnBatch.getTimestamp(rowId, pos, precision);
}
@Override
public <T> RawValueData<T> getRawValue(int pos) {
throw new UnsupportedOperationException("RawValueData is not supported.");
}
@Override
public byte[] getBinary(int pos) {
Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos);
if (byteArray.len == byteArray.data.length) {
return byteArray.data;
} else {
byte[] ret = new byte[byteArray.len];
System.arraycopy(byteArray.data, byteArray.offset, ret, 0, byteArray.len);
return ret;
}
}
@Override
public RowData getRow(int pos, int numFields) {
return vectorizedColumnBatch.getRow(rowId, pos);
}
@Override
public ArrayData getArray(int pos) {
return vectorizedColumnBatch.getArray(rowId, pos);
}
@Override
public MapData getMap(int pos) {
return vectorizedColumnBatch.getMap(rowId, pos);
}
@Override
public void setNullAt(int pos) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setBoolean(int pos, boolean value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setByte(int pos, byte value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setShort(int pos, short value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setInt(int pos, int value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setLong(int pos, long value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setFloat(int pos, float value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setDouble(int pos, double value) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setDecimal(int pos, DecimalData value, int precision) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public void setTimestamp(int pos, TimestampData value, int precision) {
throw new UnsupportedOperationException("Not support the operation!");
}
@Override
public boolean equals(Object o) {
throw new UnsupportedOperationException(
"ColumnarRowData do not support equals, please compare fields one by one!");
}
@Override
public int hashCode() {
throw new UnsupportedOperationException(
"ColumnarRowData do not support hashCode, please hash fields one by one!");
}
}

View File

@@ -0,0 +1,28 @@
/*
* 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.vector;
import org.apache.flink.table.data.MapData;
/**
* Map column vector.
*/
public interface MapColumnVector extends ColumnVector {
MapData getMap(int i);
}

View File

@@ -0,0 +1,28 @@
/*
* 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.vector;
import org.apache.flink.table.data.ColumnarRowData;
/**
* Row column vector.
*/
public interface RowColumnVector extends ColumnVector {
ColumnarRowData getRow(int i);
}

View File

@@ -0,0 +1,136 @@
/*
* 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.vector;
import org.apache.flink.table.data.ArrayData;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.MapData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.TimestampData;
import org.apache.flink.table.data.vector.BytesColumnVector.Bytes;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
/**
* A VectorizedColumnBatch is a set of rows, organized with each column as a vector. It is the unit
* of query execution, organized to minimize the cost per row.
*
* <p>{@code VectorizedColumnBatch}s are influenced by Apache Hive VectorizedRowBatch.
*
* <p>References {@code org.apache.flink.table.data.vector.VectorizedColumnBatch} to include FLINK-15390.
*/
public class VectorizedColumnBatch implements Serializable {
private static final long serialVersionUID = 8180323238728166155L;
/**
* This number is carefully chosen to minimize overhead and typically allows one
* VectorizedColumnBatch to fit in cache.
*/
public static final int DEFAULT_SIZE = 2048;
private int numRows;
public final ColumnVector[] columns;
public VectorizedColumnBatch(ColumnVector[] vectors) {
this.columns = vectors;
}
public void setNumRows(int numRows) {
this.numRows = numRows;
}
public int getNumRows() {
return numRows;
}
public int getArity() {
return columns.length;
}
public boolean isNullAt(int rowId, int colId) {
return columns[colId].isNullAt(rowId);
}
public boolean getBoolean(int rowId, int colId) {
return ((BooleanColumnVector) columns[colId]).getBoolean(rowId);
}
public byte getByte(int rowId, int colId) {
return ((ByteColumnVector) columns[colId]).getByte(rowId);
}
public short getShort(int rowId, int colId) {
return ((ShortColumnVector) columns[colId]).getShort(rowId);
}
public int getInt(int rowId, int colId) {
return ((IntColumnVector) columns[colId]).getInt(rowId);
}
public long getLong(int rowId, int colId) {
return ((LongColumnVector) columns[colId]).getLong(rowId);
}
public float getFloat(int rowId, int colId) {
return ((FloatColumnVector) columns[colId]).getFloat(rowId);
}
public double getDouble(int rowId, int colId) {
return ((DoubleColumnVector) columns[colId]).getDouble(rowId);
}
public Bytes getByteArray(int rowId, int colId) {
return ((BytesColumnVector) columns[colId]).getBytes(rowId);
}
private byte[] getBytes(int rowId, int colId) {
Bytes byteArray = getByteArray(rowId, colId);
if (byteArray.len == byteArray.data.length) {
return byteArray.data;
} else {
return byteArray.getBytes();
}
}
public String getString(int rowId, int colId) {
Bytes byteArray = getByteArray(rowId, colId);
return new String(byteArray.data, byteArray.offset, byteArray.len, StandardCharsets.UTF_8);
}
public DecimalData getDecimal(int rowId, int colId, int precision, int scale) {
return ((DecimalColumnVector) (columns[colId])).getDecimal(rowId, precision, scale);
}
public TimestampData getTimestamp(int rowId, int colId, int precision) {
return ((TimestampColumnVector) (columns[colId])).getTimestamp(rowId, precision);
}
public ArrayData getArray(int rowId, int colId) {
return ((ArrayColumnVector) columns[colId]).getArray(rowId);
}
public RowData getRow(int rowId, int colId) {
return ((RowColumnVector) columns[colId]).getRow(rowId);
}
public MapData getMap(int rowId, int colId) {
return ((MapColumnVector) columns[colId]).getMap(rowId);
}
}

View File

@@ -0,0 +1,35 @@
/*
* 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.hudi.adapter;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
/**
* Adapter clazz for {@code AbstractStreamOperator}.
*/
public abstract class AbstractStreamOperatorAdapter<O> extends AbstractStreamOperator<O> {
@Override
public void close() throws Exception {
super.dispose();
}
public void finish() throws Exception {
super.close();
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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.hudi.adapter;
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.MailboxExecutor;
import org.apache.flink.streaming.api.operators.YieldingOperatorFactory;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* Adapter clazz for {@link AbstractStreamOperatorFactory}.
*/
public abstract class AbstractStreamOperatorFactoryAdapter<O>
extends AbstractStreamOperatorFactory<O> implements YieldingOperatorFactory<O> {
private transient MailboxExecutor mailboxExecutor;
@Override
public void setMailboxExecutor(MailboxExecutor mailboxExecutor) {
this.mailboxExecutor = mailboxExecutor;
}
public MailboxExecutorAdapter getMailboxExecutorAdapter() {
return new MailboxExecutorAdapter(getMailboxExecutor());
}
/**
* Provides the mailbox executor iff this factory implements {@link YieldingOperatorFactory}.
*/
protected MailboxExecutor getMailboxExecutor() {
return checkNotNull(
mailboxExecutor, "Factory does not implement %s", YieldingOperatorFactory.class);
}
}

View File

@@ -0,0 +1,37 @@
/*
* 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.hudi.adapter;
import org.apache.flink.streaming.api.operators.MailboxExecutor;
import org.apache.flink.util.function.ThrowingRunnable;
/**
* Adapter clazz for {@link MailboxExecutor}.
*/
public class MailboxExecutorAdapter {
private final MailboxExecutor executor;
public MailboxExecutorAdapter(MailboxExecutor executor) {
this.executor = executor;
}
public void execute(ThrowingRunnable<? extends Exception> command, String description) {
this.executor.execute(command, description);
}
}

View File

@@ -0,0 +1,40 @@
/*
* 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.hudi.adapter;
import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.RateLimiter;
/**
* Bridge class for shaded guava clazz {@code RateLimiter}.
*/
public class RateLimiterAdapter {
private final RateLimiter rateLimiter;
private RateLimiterAdapter(double permitsPerSecond) {
this.rateLimiter = RateLimiter.create(permitsPerSecond);
}
public static RateLimiterAdapter create(double permitsPerSecond) {
return new RateLimiterAdapter(permitsPerSecond);
}
public void acquire() {
this.rateLimiter.acquire();
}
}

View File

@@ -0,0 +1,48 @@
/*
* 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.hudi.adapter;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamSourceContexts;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
/**
* Adapter utils.
*/
public class Utils {
public static <O> SourceFunction.SourceContext<O> getSourceContext(
TimeCharacteristic timeCharacteristic,
ProcessingTimeService processingTimeService,
StreamTask<?, ?> streamTask,
Output<StreamRecord<O>> output,
long watermarkInterval) {
return StreamSourceContexts.getSourceContext(
timeCharacteristic,
processingTimeService,
new Object(), // no actual locking needed
streamTask.getStreamStatusMaintainer(),
output,
watermarkInterval,
-1);
}
}

View File

@@ -0,0 +1,27 @@
/*
* 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.hudi.adapter;
import org.apache.flink.streaming.api.operators.Output;
/**
* Adapter clazz for {@link Output}.
*/
public interface OutputAdapter<O> extends Output<O> {
}

View File

@@ -0,0 +1,26 @@
/*
* 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.hudi.adapter;
import org.apache.flink.runtime.state.StateInitializationContext;
/**
* Adapter clazz for {@link StateInitializationContext}.
*/
public interface StateInitializationContextAdapter extends StateInitializationContext {
}

View File

@@ -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.hudi.adapter;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import java.util.Map;
/**
* Adapter clazz for {@link StreamingRuntimeContext}.
*/
public class StreamingRuntimeContextAdapter extends StreamingRuntimeContext {
public StreamingRuntimeContextAdapter(AbstractStreamOperator<?> operator, Environment env,
Map<String, Accumulator<?, ?>> accumulators) {
super(operator, env, accumulators);
}
@Override
public MetricGroup getMetricGroup() {
return new UnregisteredMetricsGroup();
}
}

View File

@@ -0,0 +1,34 @@
package org.apache.hudi.adapter;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
/*
* 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.
*/
/**
* TableEnv for test goals.
*/
public class TestTableEnvs {
public static TableEnvironment getBatchTableEnv() {
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
return TableEnvironmentImpl.create(settings);
}
}