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

SingleStoreIO #23535

Merged
merged 49 commits into from
Nov 18, 2022
Merged

SingleStoreIO #23535

merged 49 commits into from
Nov 18, 2022

Conversation

AdalbertMemSQL
Copy link
Contributor

@AdalbertMemSQL AdalbertMemSQL commented Oct 7, 2022

Implemented SingleStoreIO according to this design doc:
https://docs.google.com/document/d/1WU-hkoZ93SaGXyOz_UtX0jXzIRl194hCId_IdmEV9jw/edit?usp=sharing
addresses #22617

In future pull requests, it is planned to add the possibility to use Read, ReadWithPartitions, and Write PTransforms without setting RowMapper and UserDataMapper and implement SchemaTransforms for them.

@codecov
Copy link

codecov bot commented Oct 7, 2022

Codecov Report

Merging #23535 (ca842ff) into master (48c70cc) will increase coverage by 9.70%.
The diff coverage is n/a.

@@            Coverage Diff             @@
##           master   #23535      +/-   ##
==========================================
+ Coverage   73.47%   83.18%   +9.70%     
==========================================
  Files         714      473     -241     
  Lines       96403    66952   -29451     
==========================================
- Hits        70828    55691   -15137     
+ Misses      24252    11261   -12991     
+ Partials     1323        0    -1323     
Flag Coverage Δ
go ?
python 83.18% <ø> (-0.03%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
...hon/apache_beam/runners/worker/worker_pool_main.py 56.32% <0.00%> (-2.94%) ⬇️
.../python/apache_beam/transforms/periodicsequence.py 97.01% <0.00%> (-1.50%) ⬇️
...apache_beam/typehints/native_type_compatibility.py 85.16% <0.00%> (-0.37%) ⬇️
sdks/python/apache_beam/typehints/typehints.py 93.05% <0.00%> (-0.33%) ⬇️
...hon/apache_beam/runners/worker/bundle_processor.py 93.54% <0.00%> (-0.13%) ⬇️
sdks/python/apache_beam/ml/inference/__init__.py 100.00% <0.00%> (ø)
...thon/apache_beam/ml/inference/pytorch_inference.py 0.00% <0.00%> (ø)
...pache_beam/typehints/pytorch_type_compatibility.py 0.00% <0.00%> (ø)
sdks/go/pkg/beam/core/graph/coder/bool.go
sdks/go/pkg/beam/runners/direct/buffer.go
... and 251 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@AdalbertMemSQL AdalbertMemSQL marked this pull request as ready for review October 26, 2022 14:24
@github-actions
Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kileys for label java.
R: @Abacn for label build.
R: @pabloem for label io.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@AdalbertMemSQL
Copy link
Contributor Author

R: @kennknowles

@github-actions
Copy link
Contributor

Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control

@AdalbertMemSQL
Copy link
Contributor Author

R: @johnjcasey

@AdalbertMemSQL
Copy link
Contributor Author

R: @TheNeuralBit

@AdalbertMemSQL
Copy link
Contributor Author

retest this please

@AdalbertMemSQL
Copy link
Contributor Author

Run CommunityMetrics PreCommit

@AdalbertMemSQL
Copy link
Contributor Author

Run Python PreCommit

@AdalbertMemSQL
Copy link
Contributor Author

Run Java PreCommit

1 similar comment
@AdalbertMemSQL
Copy link
Contributor Author

Run Java PreCommit

Copy link
Contributor

@johnjcasey johnjcasey left a comment

Choose a reason for hiding this comment

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

Reviewed the core code, Looks really solid so far.

implementation project(path: ":sdks:java:core", configuration: "shadow")
implementation group: 'com.singlestore', name: 'singlestore-jdbc-client', version: '1.1.4'
implementation library.java.slf4j_api
implementation "org.apache.commons:commons-dbcp2:2.8.0"
Copy link
Contributor

Choose a reason for hiding this comment

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

can we move these dependencies to BeamModulePlugin.groovy?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Do I understand correctly that your proposition is to add these libs to this map

    // A map of maps containing common libraries used per language. To use:
    // dependencies {
    //   compile library.java.slf4j_api
    // }
    project.ext.library = [
      java : [

and use it here?

DataSource dataSource = dataSourceConfiguration.getDataSource();
Connection conn = dataSource.getConnection();
try {
for (long partition = tracker.currentRestriction().getFrom();
Copy link
Contributor

Choose a reason for hiding this comment

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

this should probably not have partition++. In principle, this will always cause an error, because at some point the tryClaim will attempt to claim an invalid partition.

Copy link
Contributor

Choose a reason for hiding this comment

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

either because it is outside of the tracker range, or because it is beyond the total partition count.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The same loop is presented in this blog https://beam.apache.org/blog/splittable-do-fn/

public void process(ProcessContext c, OffsetRangeTracker tracker) {
    for (long i = tracker.currentRestriction().getFrom(); tracker.tryClaim(i); ++i) {
      c.output(KV.of(c.element().getKey(), i));
    }
  }

I thought that tryClaim will just return false when an invalid partition is provided.


final Exception[] writeException = new Exception[1];

Thread dataWritingThread =
Copy link
Contributor

Choose a reason for hiding this comment

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

Avoid creating threads inside of dofns. In Beam, asynchronous and multithreading behavior is best handled by the Beam framework itself.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmm....
I don't know how to implement this asynchronous task using Beam framework.
Here we have two parts - statement execution and writing data to the stream.
They are tightly coupled (writing will get stuck if the statement is not executed and the buffer is full and the statement execution won't finish until writing is finished).
And also these parts share a common PipedOutputStream which is not serializable.

Does the Beam framework have features that will allow something like this?

Copy link
Contributor

Choose a reason for hiding this comment

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

This is somewhat unusual for me as well. @lukecwik do we have a pattern for something like this?

@johnjcasey
Copy link
Contributor

This looks good to me outside of the threading, which I'm unsure about.

@chamikaramj and @Abacn can you take a second look, as this is an entire IO I'd like more pairs of eyes

@Abacn
Copy link
Contributor

Abacn commented Nov 11, 2022

Thanks @AdalbertMemSQL! Will take a look. For now, could you please integrate Read.java and Write.java into SingleStoreIO.java, as the patterns used for other IO connectors (e.g. JdbcIO). users may not able to do something like

import org.apache.beam.sdk.io.singlestore.Read;
import org.apache.beam.sdk.io.singlestore.Write;

since there may be naming conflictions.

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

run seed job

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Run SQL PreCommit

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Run Seed Job

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Hi @AdalbertMemSQL would you mind rebase the branch onto latest master and I can run a seedjob (make beam_PerformanceTests_SingleStoreIO and Java_SingleStore_IO_Direct work), thanks

also please consider adding an updates to https://github.com/apache/beam/blob/master/CHANGES.md

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Run Java_Spark3_Versions PreCommit

@AdalbertMemSQL
Copy link
Contributor Author

I thought that SingleStore_IO_Direct will run unit tests.
But after your comment, I realized that it is running the same integration test.
Deleted it for now.

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Run Java SingleStoreIO Performance Test

@Abacn
Copy link
Contributor

Abacn commented Nov 14, 2022

Notice that SingleStoreIO Performance Test actually runs on direct runner. Direct runner is only for local debug use. It adds some redundant calculations for validations and is not performant. Could you please consider running it on Dataflow runner (there is an switches("-DintegrationTestRunner=dataflow") could refer to other performance tests)

@AdalbertMemSQL
Copy link
Contributor Author

Sure :)

@AdalbertMemSQL
Copy link
Contributor Author

@Abacn Can you now run the perf test?

@Abacn
Copy link
Contributor

Abacn commented Nov 15, 2022

Run Seed Job

@Abacn
Copy link
Contributor

Abacn commented Nov 15, 2022

Run Java SingleStoreIO Performance Test

Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

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

Besides the performance test, left some other comments here.

import org.slf4j.Logger;

/** Provides utility functions for working with {@link SingleStoreIO}. */
public class SingleStoreUtil {
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
public class SingleStoreUtil {
final class SingleStoreUtil {

We may not expose this as public API (as the practice of other IOs)

CHANGES.md Outdated
@@ -60,6 +60,7 @@

* Support for Bigtable sink (Write and WriteBatch) added (Go) ([#23324](https://github.com/apache/beam/issues/23324)).
* S3 implementation of the Beam filesystem (Go) ([#23991](https://github.com/apache/beam/issues/23991)).
* Support for SingleStoreDB source added (Java) ([#22617](https://github.com/apache/beam/issues/22617)).
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mean "SingleStoreDB Source and Sink"?

singleStoreUsername : "admin",
singleStorePassword : "secretpass",
singleStorePort: "3306",
numberOfRecords: "100000",
Copy link
Contributor

Choose a reason for hiding this comment

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

Looks like the performance test is passing. The Write then Read steps finishing within seconds. To get a more meaningful metric, ideally the pipeline runs several minutes. How about setting numberOfRecords to 10000000?

Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry I meant 10M records (100 times of "100000").

The metrics shown in https://ci-beam.apache.org/view/PerformanceTests/job/beam_PerformanceTests_SingleStoreIO/6/console

13:54:32     Load test results for test (ID): 403b18ab-b3dc-475a-b4d9-0a77866159c8 and timestamp: 2022-11-15T18:54:32.477000000Z:
13:54:32                      Metric:                    Value:
13:54:32                   write_time                     0.628
13:54:32     Load test results for test (ID): 403b18ab-b3dc-475a-b4d9-0a77866159c8 and timestamp: 2022-11-15T18:54:32.477000000Z:
13:54:33                      Metric:                    Value:
13:54:33                    read_time                     4.433
13:54:33     Load test results for test (ID): 403b18ab-b3dc-475a-b4d9-0a77866159c8 and timestamp: 2022-11-15T18:54:32.477000000Z:
13:54:33                      Metric:                    Value:
13:54:33     read_with_partitions_time                      1.87

If performance is linear 1M record still costs seconds to execute write.

SingleStoreDB is pretty performant!

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Wow, it is really pretty fast :)
Increased the number of rows to 10M.

Copy link
Contributor

Choose a reason for hiding this comment

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

ah sorry. The test row pre-defined some hash for validation. 10000000 does not work but 5000000 will do:

  private static final ImmutableMap<Integer, String> EXPECTED_HASHES =
      ImmutableMap.of(
          1000, "7d94d63a41164be058a9680002914358",
          100_000, "c7cbddb319209e200f1c5eebef8fe960",
          600_000, "e2add2f680de9024e9bc46cd3912545e",
          5_000_000, "c44f8a5648cd9207c9c6f77395a998dc");

Changed SingleStoreUtil to be final class.
Changed CHANGES.md file.
Increased number of rows in the integration test.
Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

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

One last comment from my side. Thanks!

PipelineResult writeResult = runWrite();
writeResult.waitUntilFinish();
PipelineResult readResult = runRead();
readResult.waitUntilFinish();
Copy link
Contributor

Choose a reason for hiding this comment

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

We should assert the pipeline status for write and read is DONE; we may also assert the number of record we read is consistent with the number of record we have written (some basic validation). Looks like currently if the pipeline fails remotely the test will still pass.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

number of records is checked here

  private void testReadResult(PCollection<TestRow> namesAndIds) {
    PAssert.thatSingleton(namesAndIds.apply("Count All", Count.globally()))
        .isEqualTo((long) numberOfRows);

(lines 248-250)

Will add asserts for pipeline status. Thanks!

@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Run Java SingleStoreIO Performance Test

@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Run Java_Examples_Dataflow_Java17 PreCommit

@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Run Java SingleStoreIO Performance Test

1 similar comment
@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Run Java SingleStoreIO Performance Test

@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Succeeded: https://ci-beam.apache.org/view/PerformanceTests/job/beam_PerformanceTests_SingleStoreIO/9/console

metrics:

13:10:34 org.apache.beam.sdk.io.singlestore.SingleStoreIOIT > testWriteThenRead STANDARD_OUT
13:10:34     Load test results for test (ID): fef7d33b-660b-4c77-a441-22fd9d6d548a and timestamp: 2022-11-16T18:10:34.119000000Z:
13:10:34                      Metric:                    Value:
13:10:34                   write_time                    25.008
13:10:34     Load test results for test (ID): fef7d33b-660b-4c77-a441-22fd9d6d548a and timestamp: 2022-11-16T18:10:34.119000000Z:
13:10:34                      Metric:                    Value:
13:10:34                    read_time                    74.979
13:10:34     Load test results for test (ID): fef7d33b-660b-4c77-a441-22fd9d6d548a and timestamp: 2022-11-16T18:10:34.119000000Z:
13:10:34                      Metric:                    Value:
13:10:34     read_with_partitions_time                    28.704

* );
* }</pre>
*/
final class SingleStoreIO {
Copy link
Contributor

@Abacn Abacn Nov 16, 2022

Choose a reason for hiding this comment

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

Oops I believe this is a mistake. should be final class SingleStoreUtil and public class SingleStoreIO

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yea, sorry

Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

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

LGTM once the last typo gets resolved (see above)

Some followups about tests (in separate PRs)

@Abacn
Copy link
Contributor

Abacn commented Nov 16, 2022

Let us pin @lukecwik and @chamikaramj again who got mentioned in this thread and see if they have any inputs. Will be a nice feature for Beam 2.44.0

@Abacn
Copy link
Contributor

Abacn commented Nov 18, 2022

ok let's get this in so tests are continuously exercised

@Abacn Abacn merged commit 0265634 into apache:master Nov 18, 2022
prodriguezdefino pushed a commit to prodriguezdefino/beam-pabs that referenced this pull request Nov 18, 2022
Summary:
Implemented SingleStoreIO according to this design doc:
https://docs.google.com/document/d/1WU-hkoZ93SaGXyOz_UtX0jXzIRl194hCId_IdmEV9jw/edit?usp=sharing

Changed SingleStoreUtil to be final class.
Changed CHANGES.md file.
Increased number of rows in the integration test.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants