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
Expand Up @@ -18,13 +18,15 @@
package org.apache.fluss.lake.lakestorage;

import org.apache.fluss.annotation.PublicEvolving;
import org.apache.fluss.exception.InvalidConfigException;
import org.apache.fluss.exception.TableAlreadyExistException;
import org.apache.fluss.exception.TableNotExistException;
import org.apache.fluss.metadata.TableChange;
import org.apache.fluss.metadata.TableDescriptor;
import org.apache.fluss.metadata.TablePath;

import java.util.List;
import java.util.Map;

/**
* A catalog interface to modify metadata in external datalake.
Expand All @@ -34,6 +36,14 @@
@PublicEvolving
public interface LakeCatalog extends AutoCloseable {

/**
* Get a table in lake.
*
* @param tablePath path of the table to be created
* @throws TableNotExistException if the table not exists
*/
TableDescriptor getTable(TablePath tablePath) throws TableNotExistException;

/**
* Create a new table in lake.
*
Expand All @@ -54,6 +64,26 @@ void createTable(TablePath tablePath, TableDescriptor tableDescriptor)
void alterTable(TablePath tablePath, List<TableChange> tableChanges)
throws TableNotExistException;

/**
* Check whether the properties of the existing lake table are compatible with the properties of
* the table want to be created.
*
* @param properties The properties to be validated
*/
default void validateTablePropertyCompatibility(
Map<String, String> existingProperties, Map<String, String> properties) {
if (!existingProperties.equals(properties)) {
throw new InvalidConfigException(
"The properties of the existing lake table are not compatible with "
+ "the properties of the table want to be created. "
+ "Properties of existing lake table: "
+ existingProperties
+ ", "
+ "properties of lake table to be created: "
+ properties);
}
}

@Override
default void close() throws Exception {
// default do nothing
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.fluss.utils.WrappingProxy;

import java.util.List;
import java.util.Map;

/**
* A wrapper around {@link LakeStoragePlugin} that ensures the plugin classloader is used for all
Expand Down Expand Up @@ -74,6 +75,13 @@ private ClassLoaderFixingLakeCatalog(final LakeCatalog inner, final ClassLoader
this.loader = loader;
}

@Override
public TableDescriptor getTable(TablePath tablePath) throws TableNotExistException {
try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(loader)) {
return inner.getTable(tablePath);
}
}

@Override
public void createTable(TablePath tablePath, TableDescriptor tableDescriptor)
throws TableAlreadyExistException {
Expand All @@ -90,6 +98,14 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges)
}
}

@Override
public void validateTablePropertyCompatibility(
Map<String, String> existingProperties, Map<String, String> properties) {
try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(loader)) {
inner.validateTablePropertyCompatibility(existingProperties, properties);
}
}

@Override
public void close() throws Exception {
try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(loader)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,14 @@ public String asSummaryString() {

public abstract <R> R accept(DataTypeVisitor<R> visitor);

public DataType notNull() {
return copy(false);
}

public DataType nullable() {
return copy(true);
}

@Override
public String toString() {
return asSummaryString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,11 @@ public LakeSource<?> createLakeSource(TablePath tablePath) {

private static class TestPaimonLakeCatalog implements LakeCatalog {

@Override
public TableDescriptor getTable(TablePath tablePath) throws TableNotExistException {
return null;
}

@Override
public void createTable(TablePath tablePath, TableDescriptor tableDescriptor)
throws TableAlreadyExistException {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -308,13 +308,8 @@ void testCreateAlreadyExistsLakeTable() throws Exception {
assertThat(catalog.tableExists(lakeTablePath)).isTrue();
// drop fluss table
catalog.dropTable(lakeTablePath, false);
// create the table again, should throw exception with ignore if exist = false
assertThatThrownBy(() -> catalog.createTable(lakeTablePath, table, false))
.isInstanceOf(CatalogException.class)
.hasMessage(
String.format(
"The table %s already exists in %s catalog, please first drop the table in %s catalog or use a new table name.",
lakeTablePath, "paimon", "paimon"));
// create the table again, should be ok if lake table schema match the fluss table schema
catalog.createTable(lakeTablePath, table, false);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,12 @@ protected Catalog getIcebergCatalog() {
return icebergCatalog;
}

@Override
public TableDescriptor getTable(TablePath tablePath) throws TableNotExistException {
throw new UnsupportedOperationException(
"Get table is not supported for Iceberg at the moment");
}

@Override
public void createTable(TablePath tablePath, TableDescriptor tableDescriptor)
throws TableAlreadyExistException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,12 @@ public LanceLakeCatalog(Configuration config) {
this.options = config;
}

@Override
public TableDescriptor getTable(TablePath tablePath) throws TableNotExistException {
throw new UnsupportedOperationException(
"Get table is not supported for Lance at the moment");
}

@Override
public void createTable(TablePath tablePath, TableDescriptor tableDescriptor) {
// currently, we don't support primary key table for lance
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,178 @@
/*
* 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.fluss.lake.paimon;

import org.apache.fluss.types.DataType;
import org.apache.fluss.types.DataTypes;

import org.apache.paimon.types.ArrayType;
import org.apache.paimon.types.BigIntType;
import org.apache.paimon.types.BinaryType;
import org.apache.paimon.types.BooleanType;
import org.apache.paimon.types.CharType;
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataTypeVisitor;
import org.apache.paimon.types.DateType;
import org.apache.paimon.types.DecimalType;
import org.apache.paimon.types.DoubleType;
import org.apache.paimon.types.FloatType;
import org.apache.paimon.types.IntType;
import org.apache.paimon.types.LocalZonedTimestampType;
import org.apache.paimon.types.MapType;
import org.apache.paimon.types.MultisetType;
import org.apache.paimon.types.RowType;
import org.apache.paimon.types.SmallIntType;
import org.apache.paimon.types.TimeType;
import org.apache.paimon.types.TimestampType;
import org.apache.paimon.types.TinyIntType;
import org.apache.paimon.types.VarBinaryType;
import org.apache.paimon.types.VarCharType;
import org.apache.paimon.types.VariantType;

/** Convert from Paimon's data type to Fluss's data type. */
public class PaimonDataTypeToFlussDataType implements DataTypeVisitor<DataType> {

public static final PaimonDataTypeToFlussDataType INSTANCE =
new PaimonDataTypeToFlussDataType();

@Override
public DataType visit(CharType charType) {
return withNullability(DataTypes.CHAR(charType.getLength()), charType.isNullable());
}

@Override
public DataType visit(VarCharType varCharType) {
return withNullability(DataTypes.STRING(), varCharType.isNullable());
}

@Override
public DataType visit(BooleanType booleanType) {
return withNullability(DataTypes.BOOLEAN(), booleanType.isNullable());
}

@Override
public DataType visit(BinaryType binaryType) {
return withNullability(DataTypes.BINARY(binaryType.getLength()), binaryType.isNullable());
}

@Override
public DataType visit(VarBinaryType varBinaryType) {
return withNullability(DataTypes.BYTES(), varBinaryType.isNullable());
}

@Override
public DataType visit(DecimalType decimalType) {
return withNullability(
DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale()),
decimalType.isNullable());
}

@Override
public DataType visit(TinyIntType tinyIntType) {
return withNullability(DataTypes.TINYINT(), tinyIntType.isNullable());
}

@Override
public DataType visit(SmallIntType smallIntType) {
return withNullability(DataTypes.SMALLINT(), smallIntType.isNullable());
}

@Override
public DataType visit(IntType intType) {
return withNullability(DataTypes.INT(), intType.isNullable());
}

@Override
public DataType visit(BigIntType bigIntType) {
return withNullability(DataTypes.BIGINT(), bigIntType.isNullable());
}

@Override
public DataType visit(FloatType floatType) {
return withNullability(DataTypes.FLOAT(), floatType.isNullable());
}

@Override
public DataType visit(DoubleType doubleType) {
return withNullability(DataTypes.DOUBLE(), doubleType.isNullable());
}

@Override
public DataType visit(DateType dateType) {
return withNullability(DataTypes.DATE(), dateType.isNullable());
}

@Override
public DataType visit(TimeType timeType) {
return withNullability(DataTypes.TIME(), timeType.isNullable());
}

@Override
public DataType visit(TimestampType timestampType) {
return withNullability(
DataTypes.TIMESTAMP(timestampType.getPrecision()), timestampType.isNullable());
}

@Override
public DataType visit(LocalZonedTimestampType localZonedTimestampType) {
return withNullability(
DataTypes.TIMESTAMP_LTZ(localZonedTimestampType.getPrecision()),
localZonedTimestampType.isNullable());
}

@Override
public DataType visit(VariantType variantType) {
throw new UnsupportedOperationException("VariantType is not supported for Fluss.");
}

@Override
public DataType visit(ArrayType arrayType) {
return withNullability(
DataTypes.ARRAY(arrayType.getElementType().accept(this)), arrayType.isNullable());
}

@Override
public DataType visit(MultisetType multisetType) {
throw new UnsupportedOperationException("MultisetType is not supported for Fluss.");
}

@Override
public DataType visit(MapType mapType) {
return withNullability(
DataTypes.MAP(
mapType.getKeyType().accept(this), mapType.getValueType().accept(this)),
mapType.isNullable());
}

@Override
public DataType visit(RowType rowType) {
org.apache.fluss.types.RowType.Builder rowTypeBuilder =
org.apache.fluss.types.RowType.builder();
for (DataField field : rowType.getFields()) {
rowTypeBuilder.field(field.name(), field.type().accept(this), field.description());
}
return withNullability(rowTypeBuilder.build(), rowType.isNullable());
}

private DataType withNullability(DataType fluss, boolean nullable) {
if (fluss.isNullable() != nullable) {
return nullable ? fluss.nullable() : fluss.notNull();
}
return fluss;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,15 +34,19 @@
import org.apache.paimon.options.Options;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.SchemaChange;
import org.apache.paimon.table.Table;
import org.apache.paimon.types.DataType;
import org.apache.paimon.types.DataTypes;

import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;

import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toFlussTableDescriptor;
import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimon;
import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimonSchema;
import static org.apache.fluss.lake.paimon.utils.PaimonConversions.toPaimonSchemaChanges;
import static org.apache.fluss.lake.paimon.utils.PaimonConversions.validatePaimonTableOptions;
import static org.apache.fluss.metadata.TableDescriptor.BUCKET_COLUMN_NAME;
import static org.apache.fluss.metadata.TableDescriptor.OFFSET_COLUMN_NAME;
import static org.apache.fluss.metadata.TableDescriptor.TIMESTAMP_COLUMN_NAME;
Expand Down Expand Up @@ -76,6 +80,17 @@ protected Catalog getPaimonCatalog() {
return paimonCatalog;
}

@Override
public TableDescriptor getTable(TablePath tablePath) throws TableNotExistException {
try {
Identifier paimonPath = toPaimon(tablePath);
Table table = paimonCatalog.getTable(paimonPath);
return toFlussTableDescriptor(table);
} catch (Catalog.TableNotExistException e) {
throw new TableNotExistException("Table " + tablePath + " not exists.");
}
}

@Override
public void createTable(TablePath tablePath, TableDescriptor tableDescriptor)
throws TableAlreadyExistException {
Expand Down Expand Up @@ -114,6 +129,12 @@ public void alterTable(TablePath tablePath, List<TableChange> tableChanges)
}
}

@Override
public void validateTablePropertyCompatibility(
Map<String, String> existingProperties, Map<String, String> properties) {
validatePaimonTableOptions(existingProperties, properties);
}

private void createTable(Identifier tablePath, Schema schema)
throws Catalog.DatabaseNotExistException {
try {
Expand Down
Loading
Loading