-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Implement Flink InputFormat and integrate it to FlinkCatalog #1293
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
ad42994
dfb6825
1194d43
c826b03
9bd4248
02bc024
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,51 @@ | ||
| /* | ||
| * 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.iceberg.flink; | ||
|
|
||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.types.FixupTypes; | ||
| import org.apache.iceberg.types.Type; | ||
| import org.apache.iceberg.types.TypeUtil; | ||
| import org.apache.iceberg.types.Types; | ||
|
|
||
| /** | ||
| * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, | ||
| * which may not be correct. | ||
| */ | ||
| class FlinkFixupTypes extends FixupTypes { | ||
|
|
||
| private FlinkFixupTypes(Schema referenceSchema) { | ||
| super(referenceSchema); | ||
| } | ||
|
|
||
| static Schema fixup(Schema schema, Schema referenceSchema) { | ||
| return new Schema(TypeUtil.visit(schema, | ||
| new FlinkFixupTypes(referenceSchema)).asStructType().fields()); | ||
| } | ||
|
|
||
| @Override | ||
| protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { | ||
| if (type instanceof Types.FixedType) { | ||
| int length = ((Types.FixedType) type).length(); | ||
| return source.typeId() == Type.TypeID.UUID && length == 16; | ||
| } | ||
| return false; | ||
| } | ||
| } |
| 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.iceberg.flink; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Map; | ||
| import org.apache.flink.table.api.TableSchema; | ||
| import org.apache.flink.table.api.ValidationException; | ||
| import org.apache.flink.table.catalog.ObjectPath; | ||
| import org.apache.flink.table.catalog.exceptions.TableNotExistException; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.factories.TableSourceFactory; | ||
| import org.apache.flink.table.sources.TableSource; | ||
| import org.apache.flink.table.utils.TableSchemaUtils; | ||
| import org.apache.iceberg.flink.source.FlinkTableSource; | ||
|
|
||
| /** | ||
| * Flink Iceberg table factory to create table source and sink. | ||
| * Only works for catalog, can not be loaded from Java SPI(Service Provider Interface). | ||
| */ | ||
| class FlinkTableFactory implements TableSourceFactory<RowData> { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We may put the tableSink creator in this class too. https://github.com/apache/iceberg/pull/1348/files#diff-0ad7dfff9cfa32fbb760796d976fd650R34
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, should be in this class too. |
||
|
|
||
| private final FlinkCatalog catalog; | ||
|
|
||
| FlinkTableFactory(FlinkCatalog catalog) { | ||
| this.catalog = catalog; | ||
| } | ||
|
|
||
| @Override | ||
| public Map<String, String> requiredContext() { | ||
| throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI"); | ||
| } | ||
|
|
||
| @Override | ||
| public List<String> supportedProperties() { | ||
| throw new UnsupportedOperationException("Iceberg Table Factory can not be loaded from Java SPI"); | ||
| } | ||
|
|
||
| @Override | ||
| public TableSource<RowData> createTableSource(Context context) { | ||
| ObjectPath objectPath = context.getObjectIdentifier().toObjectPath(); | ||
| TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getTable().getSchema()); | ||
| try { | ||
| return new FlinkTableSource( | ||
| catalog.getIcebergTable(objectPath), createTableLoader(objectPath), catalog.getHadoopConf(), tableSchema, | ||
| context.getTable().getOptions()); | ||
| } catch (TableNotExistException e) { | ||
| throw new ValidationException(String.format("Iceberg Table(%s) not exist.", objectPath), e); | ||
| } | ||
| } | ||
|
|
||
| private TableLoader createTableLoader(ObjectPath objectPath) { | ||
| return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Q: The javadoc says it's deprecated now, What's the time for us to use the
getFactoryin future ?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Until the new API is really ready...
In the 1.11 and master, FLIP-95 interfaces still lack many things. I am not sure about Flink 1.12, maybe 1.13 is the time.