fresh-borzoni commented on code in PR #2976: URL: https://github.com/apache/fluss/pull/2976#discussion_r3027954615
########## fluss-common/src/main/java/org/apache/fluss/exception/InvalidScanRequestException.java: ########## @@ -0,0 +1,31 @@ +/* + * 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.fluss.exception; + +/** + * Thrown when a ScanKv request is malformed, for example when both {@code scanner_id} and {@code + * bucket_scan_req} are set simultaneously, or when neither is set, or when the {@code call_seq_id} + * is out of order. + */ +public class InvalidScanRequestException extends ApiException { Review Comment: @PublicEvolving in all error classes ########## fluss-rpc/src/main/proto/FlussApi.proto: ########## @@ -300,6 +300,43 @@ message LimitScanResponse{ } +// Full KV scan request and response. +// A new scan is initiated with bucket_scan_req; subsequent batches use scanner_id. +message PbScanReqForBucket { + required int64 table_id = 1; + optional int64 partition_id = 2; + required int32 bucket_id = 3; + // If set, stops returning rows after this many records. + optional int64 limit = 4; +} + +message ScanKvRequest { + // Mutually exclusive: either scanner_id (continuation) or bucket_scan_req (new scan). + optional bytes scanner_id = 1; + optional PbScanReqForBucket bucket_scan_req = 2; + // Monotonically increasing sequence number for in-order delivery validation. + optional int32 call_seq_id = 3; + // Maximum number of bytes of record data to return in this batch. + required int32 batch_size_bytes = 4; Review Comment: Wouldn't required blow up serialization in close-scanner request? ########## fluss-rpc/src/main/proto/FlussApi.proto: ########## @@ -300,6 +300,43 @@ message LimitScanResponse{ } +// Full KV scan request and response. +// A new scan is initiated with bucket_scan_req; subsequent batches use scanner_id. +message PbScanReqForBucket { Review Comment: nit: Mb PbBucketScanRequest reads better? ########## fluss-common/src/main/java/org/apache/fluss/exception/ScannerNotFoundException.java: ########## @@ -0,0 +1,27 @@ +/* + * 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.fluss.exception; + +/** Exception thrown when a scanner is not found. */ +public class ScannerNotFoundException extends ApiException { Review Comment: I don't see it wired in Errors.java, what is the difference between this guy and ScannerNotFound and ScannerExpiredException? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
