You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@paimon.apache.org by "JingsongLi (via GitHub)" <gi...@apache.org> on 2023/04/25 09:10:35 UTC

[GitHub] [incubator-paimon] JingsongLi opened a new pull request, #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

JingsongLi opened a new pull request, #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022

   ### Purpose
   
   This PR is intended to introduce a consume mode, where when a `consume-id` is specified, scan reads the file and writes the progress of the consume to the file system, which has several advantages:
   1. When the job is killed, the newly started job can continue to consume from the previous progress without resuming from the state.
   2. When deleting expired snapshots, it ensures that the snapshot being consumed will not be deleted, so that the consumer will not read the expired snapshot and get hung.
   
   This PR also introduces a new Source implementation:
   - MonitorFunction
   - ReadOperator
   This is to avoid Flink's Source interface forcing a checkpoint on Splits, which may only read part of the snapshot data when consuming, the progress of consuming cannot be recorded.
   
   ### Documentation
   
   TODO
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177440749


##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.paimon.flink.source.operator;
+
+import org.apache.paimon.flink.utils.JavaTypeInfo;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.EndOfScanException;
+import org.apache.paimon.table.source.ReadBuilder;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.StreamTableScan;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+
+/**
+ * This is the single (non-parallel) monitoring task, it is responsible for:
+ *
+ * <ol>
+ *   <li>Monitoring snapshots of the Paimon table.
+ *   <li>Creating the {@link Split splits} corresponding to the incremental files
+ *   <li>Assigning them to downstream tasks for further processing.
+ * </ol>
+ *
+ * <p>The splits to be read are forwarded to the downstream {@link ReadOperator} which can have
+ * parallelism greater than one.
+ */
+public class MonitorFunction extends RichSourceFunction<Split>
+        implements CheckpointedFunction, CheckpointListener {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class);
+
+    private final ReadBuilder readBuilder;
+    private final long monitorInterval;
+
+    private volatile boolean isRunning = true;
+
+    private transient StreamTableScan scan;
+    private transient SourceContext<Split> ctx;
+
+    private transient ListState<Long> checkpointState;
+    private transient ListState<Tuple2<Long, Long>> nextSnapshotState;
+    private transient TreeMap<Long, Long> nextSnapshotPerCheckpoint;
+
+    public MonitorFunction(ReadBuilder readBuilder, long monitorInterval) {
+        this.readBuilder = readBuilder;
+        this.monitorInterval = monitorInterval;
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        this.scan = readBuilder.newStreamScan();
+
+        this.checkpointState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "next-snapshot", LongSerializer.INSTANCE));
+
+        @SuppressWarnings("unchecked")
+        final Class<Tuple2<Long, Long>> typedTuple =
+                (Class<Tuple2<Long, Long>>) (Class<?>) Tuple2.class;
+        this.nextSnapshotState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "next-snapshot-per-checkpoint",
+                                        new TupleSerializer<>(
+                                                typedTuple,
+                                                new TypeSerializer[] {
+                                                    LongSerializer.INSTANCE, LongSerializer.INSTANCE
+                                                })));
+
+        this.nextSnapshotPerCheckpoint = new TreeMap<>();
+
+        if (context.isRestored()) {
+            LOG.info("Restoring state for the {}.", getClass().getSimpleName());
+
+            List<Long> retrievedStates = new ArrayList<>();
+            for (Long entry : this.checkpointState.get()) {
+                retrievedStates.add(entry);
+            }
+
+            // given that the parallelism of the function is 1, we can only have 1 retrieved items.
+
+            Preconditions.checkArgument(
+                    retrievedStates.size() <= 1,
+                    getClass().getSimpleName() + " retrieved invalid state.");
+
+            if (retrievedStates.size() == 1) {
+                this.scan.restore(retrievedStates.get(0));

Review Comment:
   In `AbstractInnerTableScan`, it will read consume-id file.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1522807943

   > maybe each consumer has an inner UUID store in state?and use UUID +consumer-id to update the record file and then we can decide which semantics to use. or compare the snapshot in record file with current snapshot -1,and then to decide whether to update? if not equals,it means that some other consumer has finished reading the snapshot.something like java CAS to avoid ABA ? or use the feature in storage,such as hdfs,it provide lease to keep single writer,keep only one consumer writing the record file
   
   The discussion is based on the premise that we need to support a third requirement:
   
   3. the same consumer-id supports multiple simultaneous reads, and the same snapshot needs to be guaranteed to be read by only one reader. It should behave just like the groupId of the message queue.
   
   I think it's an interesting topic, but I'm not sure if the current users have such a need, and I propose that I can keep it for now.
   The current information is enough, for example I know which snapshot I read and I know how I need to update the consumer file, similar to CAS (compare and swap), but the hard part is the subsequent processing, really to avoid it, I guess we can only have pre-commit and commit phases.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] lilyzhoupeijie commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "lilyzhoupeijie (via GitHub)" <gi...@apache.org>.
lilyzhoupeijie commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177292969


##########
paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java:
##########
@@ -65,6 +69,16 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
             return new ContinuousCompactorStartingScanner();
         }
 
+        // read from consumer id
+        String consumerId = options.consumerId();

Review Comment:
   do we need to compare the snapshot from consumer with snapshot from state? 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] lilyzhoupeijie commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "lilyzhoupeijie (via GitHub)" <gi...@apache.org>.
lilyzhoupeijie commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1522746673

   maybe each consumer has an inner UUID store in state?and use UUID +consumer-id to update the record file and then we can decide which semantics  to use.
   or compare the snapshot in record file with current snapshot -1,and then to decide whether to update? if not equals,it means that some other consumer has finished reading the snapshot.something like java CAS to avoid ABA ?
   or use the feature in storage,such as hdfs,it provide lease to keep single writer,keep only one consumer writing the record file


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177300266


##########
paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java:
##########
@@ -194,4 +195,18 @@ public Long checkpoint() {
     public void restore(@Nullable Long nextSnapshotId) {
         this.nextSnapshotId = nextSnapshotId;
     }
+
+    @Override
+    public void notifyCheckpointComplete(@Nullable Long nextSnapshot) {
+        if (nextSnapshot == null) {
+            return;
+        }
+
+        String consumerId = options.consumerId();
+        if (consumerId != null) {
+            snapshotSplitReader
+                    .consumerManager()
+                    .recordConsumer(consumerId, new Consumer(nextSnapshot));

Review Comment:
   Good point!
   I'm not sure what kind of behavior it should be, so I'll let them each play it out here, not sure who will record success.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1534098973

   Thanks @lilyzhoupeijie @yuzelin for your review, merged...


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177299759


##########
paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java:
##########
@@ -65,6 +69,16 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
             return new ContinuousCompactorStartingScanner();
         }
 
+        // read from consumer id
+        String consumerId = options.consumerId();

Review Comment:
   I think if the relevant content is recorded inside the state, it should not be read from the consumer file and should be trusted in the state.
   Do you think something bad will happen?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi closed pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi closed pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot
URL: https://github.com/apache/incubator-paimon/pull/1022


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] lilyzhoupeijie commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "lilyzhoupeijie (via GitHub)" <gi...@apache.org>.
lilyzhoupeijie commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177294016


##########
paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java:
##########
@@ -194,4 +195,18 @@ public Long checkpoint() {
     public void restore(@Nullable Long nextSnapshotId) {
         this.nextSnapshotId = nextSnapshotId;
     }
+
+    @Override
+    public void notifyCheckpointComplete(@Nullable Long nextSnapshot) {
+        if (nextSnapshot == null) {
+            return;
+        }
+
+        String consumerId = options.consumerId();
+        if (consumerId != null) {
+            snapshotSplitReader
+                    .consumerManager()
+                    .recordConsumer(consumerId, new Consumer(nextSnapshot));

Review Comment:
   If two flink programs with the same consumer-id exist simultaneously, what semantics do we guarantee? or  it is necessary we try to  guarantee? cause there is no lock on the file,and anyone with the consumer-id can change it.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1522728406

   Thanks @lilyzhoupeijie for the review, do you have any thoughts on the solution? Does the current solution work to your satisfaction?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi closed pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi closed pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot
URL: https://github.com/apache/incubator-paimon/pull/1022


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi merged pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi merged PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] FangYongs commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "FangYongs (via GitHub)" <gi...@apache.org>.
FangYongs commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1177407410


##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.paimon.flink.source.operator;
+
+import org.apache.paimon.flink.utils.JavaTypeInfo;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.EndOfScanException;
+import org.apache.paimon.table.source.ReadBuilder;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.StreamTableScan;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+
+/**
+ * This is the single (non-parallel) monitoring task, it is responsible for:
+ *
+ * <ol>
+ *   <li>Monitoring snapshots of the Paimon table.
+ *   <li>Creating the {@link Split splits} corresponding to the incremental files
+ *   <li>Assigning them to downstream tasks for further processing.
+ * </ol>
+ *
+ * <p>The splits to be read are forwarded to the downstream {@link ReadOperator} which can have
+ * parallelism greater than one.
+ */
+public class MonitorFunction extends RichSourceFunction<Split>
+        implements CheckpointedFunction, CheckpointListener {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class);
+
+    private final ReadBuilder readBuilder;
+    private final long monitorInterval;
+
+    private volatile boolean isRunning = true;
+
+    private transient StreamTableScan scan;
+    private transient SourceContext<Split> ctx;
+
+    private transient ListState<Long> checkpointState;
+    private transient ListState<Tuple2<Long, Long>> nextSnapshotState;
+    private transient TreeMap<Long, Long> nextSnapshotPerCheckpoint;
+
+    public MonitorFunction(ReadBuilder readBuilder, long monitorInterval) {
+        this.readBuilder = readBuilder;
+        this.monitorInterval = monitorInterval;
+    }
+
+    @Override
+    public void initializeState(FunctionInitializationContext context) throws Exception {
+        this.scan = readBuilder.newStreamScan();
+
+        this.checkpointState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "next-snapshot", LongSerializer.INSTANCE));
+
+        @SuppressWarnings("unchecked")
+        final Class<Tuple2<Long, Long>> typedTuple =
+                (Class<Tuple2<Long, Long>>) (Class<?>) Tuple2.class;
+        this.nextSnapshotState =
+                context.getOperatorStateStore()
+                        .getListState(
+                                new ListStateDescriptor<>(
+                                        "next-snapshot-per-checkpoint",
+                                        new TupleSerializer<>(
+                                                typedTuple,
+                                                new TypeSerializer[] {
+                                                    LongSerializer.INSTANCE, LongSerializer.INSTANCE
+                                                })));
+
+        this.nextSnapshotPerCheckpoint = new TreeMap<>();
+
+        if (context.isRestored()) {
+            LOG.info("Restoring state for the {}.", getClass().getSimpleName());
+
+            List<Long> retrievedStates = new ArrayList<>();
+            for (Long entry : this.checkpointState.get()) {
+                retrievedStates.add(entry);
+            }
+
+            // given that the parallelism of the function is 1, we can only have 1 retrieved items.
+
+            Preconditions.checkArgument(
+                    retrievedStates.size() <= 1,
+                    getClass().getSimpleName() + " retrieved invalid state.");
+
+            if (retrievedStates.size() == 1) {
+                this.scan.restore(retrievedStates.get(0));

Review Comment:
   Should we need to restore scan with the snapshot id in state or in consume-id file here?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] lilyzhoupeijie commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "lilyzhoupeijie (via GitHub)" <gi...@apache.org>.
lilyzhoupeijie commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1522854008

   > > maybe each consumer has an inner UUID store in state?and use UUID +consumer-id to update the record file and then we can decide which semantics to use. or compare the snapshot in record file with current snapshot -1,and then to decide whether to update? if not equals,it means that some other consumer has finished reading the snapshot.something like java CAS to avoid ABA ? or use the feature in storage,such as hdfs,it provide lease to keep single writer,keep only one consumer writing the record file
   > 
   > The discussion is based on the premise that we need to support a third requirement:
   > 
   > 3. the same consumer-id supports multiple simultaneous reads, and the same snapshot needs to be guaranteed to be read by only one reader. It should behave just like the groupId of the message queue.
   > 
   > I think it's an interesting topic, but I'm not sure if the current users have such a need, and I propose that I can keep it for now. The current information is enough, for example I know which snapshot I read and I know how I need to update the consumer file, similar to CAS (compare and swap), but the hard part is the subsequent processing, really to avoid it, I guess we can only have pre-commit and commit phases.
   
   Unless we have a global lock, maybe we cannot prevent data from being consumed multiple times.But is there some way to avoid sending overconsumed data to downstream operators?so that we still can keep an end-to-end eos.
   similar with:
   https://github.com/apache/incubator-paimon/issues/881
   is this still needed?maybe global lock or pre-allocation based on snapshot.if not needed,this one can be closed.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] JingsongLi commented on pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "JingsongLi (via GitHub)" <gi...@apache.org>.
JingsongLi commented on PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#issuecomment-1522902481

   > > > maybe each consumer has an inner UUID store in state?and use UUID +consumer-id to update the record file and then we can decide which semantics to use. or compare the snapshot in record file with current snapshot -1,and then to decide whether to update? if not equals,it means that some other consumer has finished reading the snapshot.something like java CAS to avoid ABA ? or use the feature in storage,such as hdfs,it provide lease to keep single writer,keep only one consumer writing the record file
   > > 
   > > 
   > > The discussion is based on the premise that we need to support a third requirement:
   > > 
   > > 3. the same consumer-id supports multiple simultaneous reads, and the same snapshot needs to be guaranteed to be read by only one reader. It should behave just like the groupId of the message queue.
   > > 
   > > I think it's an interesting topic, but I'm not sure if the current users have such a need, and I propose that I can keep it for now. The current information is enough, for example I know which snapshot I read and I know how I need to update the consumer file, similar to CAS (compare and swap), but the hard part is the subsequent processing, really to avoid it, I guess we can only have pre-commit and commit phases.
   > 
   > Unless we have a global lock, maybe we cannot prevent data from being consumed multiple times.But is there some way to avoid sending overconsumed data to downstream operators?so that we still can keep an end-to-end eos. similar with: #881 is this still needed?maybe global lock or pre-allocation based on snapshot.if not needed,this one can be closed.
   
   We can disregard it for now. In fact, a simple case oriented to multiple reads is to report an error.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-paimon] yuzelin commented on a diff in pull request #1022: [core] Introduce consume-id for retore consuming and reserve snapshot

Posted by "yuzelin (via GitHub)" <gi...@apache.org>.
yuzelin commented on code in PR #1022:
URL: https://github.com/apache/incubator-paimon/pull/1022#discussion_r1178891642


##########
paimon-core/src/main/java/org/apache/paimon/consumer/ConsumerManager.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.paimon.consumer;
+
+import org.apache.paimon.fs.FileIO;
+import org.apache.paimon.fs.FileStatus;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.PositionOutputStream;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Serializable;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.OptionalLong;
+
+/** Manage consumer groups. */
+public class ConsumerManager implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String CONSUMER_PREFIX = "consumer-";
+
+    private final FileIO fileIO;
+    private final Path tablePath;
+
+    public ConsumerManager(FileIO fileIO, Path tablePath) {
+        this.fileIO = fileIO;
+        this.tablePath = tablePath;
+    }
+
+    public Optional<Consumer> consumer(String consumerId) {
+        return Consumer.fromPath(fileIO, consumerPath(consumerId));
+    }
+
+    public void recordConsumer(String consumerId, Consumer consumer) {
+        try (PositionOutputStream out = fileIO.newOutputStream(consumerPath(consumerId), true)) {
+            OutputStreamWriter writer = new OutputStreamWriter(out, StandardCharsets.UTF_8);
+            writer.write(consumer.toJson());
+            writer.flush();
+        } catch (IOException e) {
+            throw new UncheckedIOException(e);
+        }
+    }
+
+    public OptionalLong minNextSnapshot() {
+        try {
+            Path directory = consumerDirectory();
+            if (!fileIO.exists(directory)) {
+                return OptionalLong.empty();
+            }
+
+            FileStatus[] statuses = fileIO.listStatus(directory);
+
+            if (statuses == null) {
+                throw new RuntimeException(
+                        String.format(
+                                "The return value is null of the listStatus for the '%s' directory.",
+                                directory));
+            }
+
+            return Arrays.stream(statuses)
+                    .map(FileStatus::getPath)
+                    .filter(path -> path.getName().startsWith(CONSUMER_PREFIX))
+                    .map(path -> Consumer.fromPath(fileIO, path))
+                    .map(Optional::get)

Review Comment:
   filter before get?
   `.filter(Optional::isPresent)`



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@paimon.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org