Skip to content
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

[FLINK-17887][table][connector] Improve interface of ScanFormatFactory and SinkFormatFactory #12320

Merged
merged 1 commit into from
Jun 1, 2020

Conversation

wuchong
Copy link
Member

@wuchong wuchong commented May 25, 2020

What is the purpose of the change

This pull request is for improving current ScanForamtFactory and SinkFormatFactory interfaces with the following problems:

  1. ScanFormat#createScanFormat only accepts ScanTableSource.Context, which means it can’t work in lookup source.
  2. The naming of ScanFormat also indicates it is only used in scan source. But a lookup source should be able to work with format too.
  3. It’s confusing that ScanFormatFactory#createScanFormat and ScanFormat#createScanFormat (create itself?)

Brief change log

  1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext
  2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context
  3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder()
  4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder()
  5. Rename ScanFormatFactory to DecodingFormatFactory
  6. Rename SinkFormatFactory to EncodingFormatFactory

Verifying this change

This change is a trivial rework and is covered by existing tests.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

@wuchong wuchong requested a review from twalthr May 25, 2020 12:41
@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit ea7a259 (Mon May 25 12:44:15 UTC 2020)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented May 25, 2020

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

@wuchong
Copy link
Member Author

wuchong commented May 25, 2020

When I writing the code, I have a feeling that maybe DeserializationFormat is a better name than DeserializationSchemaProvider , because it is created from DeserializationFormatFactory and is more intuitive in the FactoryUtil#discoverXxxxFormat than FactoryUtil#discoverXxxxProvider. But I’m not sure what’s your opinion here @twalthr @KurtYoung , as we’re swinging in the new names again…

Copy link
Contributor

@twalthr twalthr left a comment

Choose a reason for hiding this comment

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

Thanks for the PR @wuchong. I like many of the refactorings around the naming and unification of DynamicTableSource and DynamicTableSink. However, I think we should have some hierarchy in the format interfaces. It is very likely that we will have more than one format in the future. In theory each connector could define its own kind of format interface. Those will have the same createXX(..) signature but different runtime interface. Having methods like discoverDeserializationProvider, discoverOptionalDeserializationProvider in the util are too specific (also seen by the name). Instead, we should parameterize these utility methods with the factory people are looking for. We should just have an upper SourceFormat and SinkFormat interface.

@wuchong
Copy link
Member Author

wuchong commented May 27, 2020

Hi @twalthr , I can see the benefit to have a hierarchy in the format interfaces. What do you think about to use the first proposal in our offline discussion? That DecodingFormat#createRuntimeDecoder() and EncodingFormat#createRuntimeEncoder(). I don't have a clear idea how to give a suitable method name for SourceFormat#createXxxx.

@twalthr
Copy link
Contributor

twalthr commented May 27, 2020

+1 for Encoding / DecodingFormat

Copy link
Contributor

@twalthr twalthr left a comment

Choose a reason for hiding this comment

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

+1 I just found a couple of formatting issues after the refactoring

return new Kafka011DynamicSink(
consumedDataType,
topic,
properties,
partitioner,
sinkFormat);
encodingFormat);
Copy link
Contributor

Choose a reason for hiding this comment

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

fix wrong indention here and at other locations

this.consumedDataType = Preconditions.checkNotNull(consumedDataType, "Consumed data type must not be null.");
this.topic = Preconditions.checkNotNull(topic, "Topic must not be null.");
this.properties = Preconditions.checkNotNull(properties, "Properties must not be null.");
this.partitioner = Preconditions.checkNotNull(partitioner, "Partitioner must not be null.");
this.sinkFormat = Preconditions.checkNotNull(sinkFormat, "Sink format must not be null.");
this.encodingFormat = Preconditions.checkNotNull(encodingFormat, "Sink format must not be null.");
Copy link
Contributor

Choose a reason for hiding this comment

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

update error message?

@@ -85,7 +85,7 @@
* @param outputDataType Source produced data type
* @param topic Kafka topic to consume.
* @param properties Properties for the Kafka consumer.
* @param scanFormat Scan format for decoding records from Kafka.
* @param decodingFormat Scan format for decoding records from Kafka.
Copy link
Contributor

Choose a reason for hiding this comment

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

fix wrong indention and remove "Scan" from Javadoc?

import org.apache.flink.table.types.DataType;

/**
* A {@link Format} for a {@link ScanTableSource}.
* A {@link Format} for a {@link DynamicTableSource} to reading rows.
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: for reading rows

@@ -34,15 +34,15 @@
*
* <p>Formats can be distinguished along two dimensions:
* <ul>
* <li>Context in which the format is applied (e.g. {@link ScanTableSource} or {@link DynamicTableSink}).
* <li>Context in which the format is applied (e.g. {@link DynamicTableSource} or {@link DynamicTableSink}).
Copy link
Contributor

Choose a reason for hiding this comment

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

we can remove the e.g. it is only for those two locations now

public final @Nullable ScanFormat<DeserializationSchema<RowData>> sourceKeyFormat;
public final ScanFormat<DeserializationSchema<RowData>> sourceValueFormat;
public final @Nullable
DecodingFormat<DeserializationSchema<RowData>> sourceKeyFormat;
Copy link
Contributor

Choose a reason for hiding this comment

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

fix formatting here and below

…y and SinkFormatFactory

1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext
2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context
3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder()
4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder()
5. Rename ScanFormatFactory to DecodingFormatFactory
6. Rename SinkFormatFactory to EncodingFormatFactory
@wuchong wuchong merged commit 64de78e into apache:master Jun 1, 2020
@wuchong wuchong deleted the improve-format-interface branch June 1, 2020 02:32
wuchong added a commit to wuchong/flink that referenced this pull request Jun 1, 2020
…y and SinkFormatFactory


We improved the interfaces with the following changes:
1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext
2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context
3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder()
4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder()
5. Rename ScanFormatFactory to DecodingFormatFactory
6. Rename SinkFormatFactory to EncodingFormatFactory

This closes apache#12320
zhangjun0x01 pushed a commit to zhangjun0x01/flink that referenced this pull request Jul 8, 2020
…y and SinkFormatFactory


We improved the interfaces with the following changes:
1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext
2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context
3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder()
4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder()
5. Rename ScanFormatFactory to DecodingFormatFactory
6. Rename SinkFormatFactory to EncodingFormatFactory

This closes apache#12320
jnh5y pushed a commit to jnh5y/flink that referenced this pull request Dec 18, 2023
…y and SinkFormatFactory


We improved the interfaces with the following changes:
1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext
2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context
3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder()
4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder()
5. Rename ScanFormatFactory to DecodingFormatFactory
6. Rename SinkFormatFactory to EncodingFormatFactory

This closes apache#12320
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
4 participants