[ https://issues.apache.org/jira/browse/BEAM-9279?focusedWorklogId=386935&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-386935 ]
ASF GitHub Bot logged work on BEAM-9279: ---------------------------------------- Author: ASF GitHub Bot Created on: 13/Feb/20 22:47 Start Date: 13/Feb/20 22:47 Worklog Time Spent: 10m Work Description: iemejia commented on pull request #10815: [BEAM-9279] Make HBase.ReadAll based on Reads instead of HBaseQuery URL: https://github.com/apache/beam/pull/10815#discussion_r379162358 ########## File path: sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java ########## @@ -240,63 +245,109 @@ private Read( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add(DisplayData.item("configuration", serializableConfiguration.get().toString())); + builder.add(DisplayData.item("configuration", configuration.toString())); builder.add(DisplayData.item("tableId", tableId)); - builder.addIfNotNull(DisplayData.item("scan", serializableScan.get().toString())); + builder.addIfNotNull(DisplayData.item("scan", scan.toString())); } public Configuration getConfiguration() { - return serializableConfiguration.get(); + return configuration; } public String getTableId() { return tableId; } public Scan getScan() { - return serializableScan.get(); + return scan; } /** Returns the range of keys that will be read from the table. */ public ByteKeyRange getKeyRange() { - byte[] startRow = serializableScan.get().getStartRow(); - byte[] stopRow = serializableScan.get().getStopRow(); + byte[] startRow = scan.getStartRow(); + byte[] stopRow = scan.getStopRow(); return ByteKeyRange.of(ByteKey.copyFrom(startRow), ByteKey.copyFrom(stopRow)); } - private final SerializableConfiguration serializableConfiguration; + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Read read = (Read) o; + return configuration.toString().equals(read.configuration.toString()) + && Objects.equals(tableId, read.tableId) + && scan.toString().equals(read.scan.toString()); + } + + @Override + public int hashCode() { + return Objects.hash(configuration, tableId, scan); + } + + private Object writeReplace() { + return new SerializationProxy(this); + } + + private static class SerializationProxy implements Serializable { + public SerializationProxy() {} + + public SerializationProxy(Read read) { + configuration = read.configuration; + tableId = read.tableId; + scan = read.scan; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + SerializableCoder.of(SerializableConfiguration.class) + .encode(new SerializableConfiguration(this.configuration), out); + StringUtf8Coder.of().encode(this.tableId, out); + ProtobufUtil.toScan(this.scan).writeDelimitedTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + this.configuration = SerializableCoder.of(SerializableConfiguration.class).decode(in).get(); + this.tableId = StringUtf8Coder.of().decode(in); + this.scan = ProtobufUtil.toScan(ClientProtos.Scan.parseDelimitedFrom(in)); + } + + Object readResolve() { + return HBaseIO.read().withConfiguration(configuration).withTableId(tableId).withScan(scan); + } + + private Configuration configuration; + private String tableId; + private Scan scan; + } + + @SuppressFBWarnings("SE_BAD_FIELD") + private final Configuration configuration; + private final String tableId; - private final SerializableScan serializableScan; + + @SuppressFBWarnings("SE_BAD_FIELD") + private final Scan scan; } /** * A {@link PTransform} that works like {@link #read}, but executes read operations coming from a - * {@link PCollection} of {@link HBaseQuery}. + * {@link PCollection} of {@link Read}. */ public static ReadAll readAll() { - return new ReadAll(null); + return new ReadAll(); } /** Implementation of {@link #readAll}. */ - public static class ReadAll extends PTransform<PCollection<HBaseQuery>, PCollection<Result>> { - - private ReadAll(SerializableConfiguration serializableConfiguration) { - this.serializableConfiguration = serializableConfiguration; - } - - /** Reads from the HBase instance indicated by the* given configuration. */ - public ReadAll withConfiguration(Configuration configuration) { - checkArgument(configuration != null, "configuration can not be null"); - return new ReadAll(new SerializableConfiguration(configuration)); - } + public static class ReadAll extends PTransform<PCollection<Read>, PCollection<Result>> { Review comment: Yes and it is intended, notice however that the only breaking change is for the SDF based read transform `ReadAll`. Also let's not forget that HBaseIO is still Experimental and Reads based on SDF for HBaseIO are Experimental^2. However after this change I seriously doubt the API will change in the future. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 386935) Time Spent: 1h 10m (was: 1h) > Make HBase.ReadAll based on Reads instead of HBaseQuery > ------------------------------------------------------- > > Key: BEAM-9279 > URL: https://issues.apache.org/jira/browse/BEAM-9279 > Project: Beam > Issue Type: Improvement > Components: io-java-hbase > Reporter: Ismaël Mejía > Assignee: Ismaël Mejía > Priority: Minor > Time Spent: 1h 10m > Remaining Estimate: 0h > > HBaseIO support for SplittableDoFn introduced a new request type HBaseQuery, > however the attributes defined in that class are already available in > HBase.Read. Allowing users to define pipelines based on HBaseIO.Read allows > to create pipelines that can read from multiple clusters because the > Configuration now is part of the request object. -- This message was sent by Atlassian Jira (v8.3.4#803005)