|
26 | 26 | import org.apache.fluss.cluster.BucketLocation; |
27 | 27 | import org.apache.fluss.config.ConfigOptions; |
28 | 28 | import org.apache.fluss.config.Configuration; |
| 29 | +import org.apache.fluss.exception.ApiException; |
29 | 30 | import org.apache.fluss.exception.InvalidMetadataException; |
30 | 31 | import org.apache.fluss.exception.LeaderNotAvailableException; |
| 32 | +import org.apache.fluss.exception.PartitionNotExistException; |
31 | 33 | import org.apache.fluss.fs.FsPath; |
32 | 34 | import org.apache.fluss.metadata.PhysicalTablePath; |
33 | 35 | import org.apache.fluss.metadata.SchemaGetter; |
|
48 | 50 | import org.apache.fluss.rpc.messages.PbFetchLogReqForTable; |
49 | 51 | import org.apache.fluss.rpc.messages.PbFetchLogRespForBucket; |
50 | 52 | import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; |
| 53 | +import org.apache.fluss.rpc.protocol.ApiError; |
51 | 54 | import org.apache.fluss.rpc.protocol.Errors; |
52 | 55 | import org.apache.fluss.utils.IOUtils; |
53 | 56 | import org.apache.fluss.utils.Projection; |
@@ -217,14 +220,27 @@ private void checkAndUpdateMetadata(List<TableBucket> tableBuckets) { |
217 | 220 | } |
218 | 221 | } |
219 | 222 |
|
220 | | - if (isPartitioned && !partitionIds.isEmpty()) { |
221 | | - metadataUpdater.updateMetadata(Collections.singleton(tablePath), null, partitionIds); |
222 | | - } else if (needUpdate) { |
223 | | - metadataUpdater.updateTableOrPartitionMetadata(tablePath, null); |
| 223 | + try { |
| 224 | + if (isPartitioned && !partitionIds.isEmpty()) { |
| 225 | + metadataUpdater.updateMetadata( |
| 226 | + Collections.singleton(tablePath), null, partitionIds); |
| 227 | + } else if (needUpdate) { |
| 228 | + metadataUpdater.updateTableOrPartitionMetadata(tablePath, null); |
| 229 | + } |
| 230 | + } catch (Exception e) { |
| 231 | + if (e instanceof PartitionNotExistException) { |
| 232 | + // ignore this exception, this is probably happen because the partition is deleted. |
| 233 | + // The fetcher can also work fine. The caller like flink can remove the partition |
| 234 | + // from fetch list when receive exception. |
| 235 | + LOG.warn("Receive PartitionNotExistException when update metadata, ignore it", e); |
| 236 | + } else { |
| 237 | + throw e; |
| 238 | + } |
224 | 239 | } |
225 | 240 | } |
226 | 241 |
|
227 | | - private void sendFetchRequest(int destination, FetchLogRequest fetchLogRequest) { |
| 242 | + @VisibleForTesting |
| 243 | + void sendFetchRequest(int destination, FetchLogRequest fetchLogRequest) { |
228 | 244 | TableOrPartitions tableOrPartitionsInFetchRequest = |
229 | 245 | getTableOrPartitionsInFetchRequest(fetchLogRequest); |
230 | 246 | // TODO cache the tablet server gateway. |
@@ -345,6 +361,14 @@ private synchronized void handleFetchLogResponse( |
345 | 361 | respForBucket.getBucketId()); |
346 | 362 | FetchLogResultForBucket fetchResultForBucket = |
347 | 363 | getFetchLogResultForBucket(tb, tablePath, respForBucket); |
| 364 | + |
| 365 | + // if error code is not NONE, it means the fetch log request failed, we need to |
| 366 | + // clear table bucket meta for InvalidMetadataException. |
| 367 | + if (fetchResultForBucket.getErrorCode() != Errors.NONE.code()) { |
| 368 | + ApiError error = ApiError.fromErrorMessage(respForBucket); |
| 369 | + handleFetchLogExceptionForBucket(tb, destination, error); |
| 370 | + } |
| 371 | + |
348 | 372 | Long fetchOffset = logScannerStatus.getBucketOffset(tb); |
349 | 373 | // if the offset is null, it means the bucket has been unsubscribed, |
350 | 374 | // we just set a Long.MAX_VALUE as the next fetch offset |
@@ -387,6 +411,29 @@ private synchronized void handleFetchLogResponse( |
387 | 411 | } |
388 | 412 | } |
389 | 413 |
|
| 414 | + private void handleFetchLogExceptionForBucket(TableBucket tb, int destination, ApiError error) { |
| 415 | + ApiException exception = error.error().exception(); |
| 416 | + LOG.error("Failed to fetch log from node {} for bucket {}", destination, tb, exception); |
| 417 | + if (exception instanceof InvalidMetadataException) { |
| 418 | + LOG.warn( |
| 419 | + "Invalid metadata error in fetch log request. " |
| 420 | + + "Going to request metadata update.", |
| 421 | + exception); |
| 422 | + long tableId = tb.getTableId(); |
| 423 | + TableOrPartitions tableOrPartitions; |
| 424 | + if (tb.getPartitionId() == null) { |
| 425 | + tableOrPartitions = new TableOrPartitions(Collections.singleton(tableId), null); |
| 426 | + } else { |
| 427 | + tableOrPartitions = |
| 428 | + new TableOrPartitions( |
| 429 | + null, |
| 430 | + Collections.singleton( |
| 431 | + new TablePartition(tableId, tb.getPartitionId()))); |
| 432 | + } |
| 433 | + invalidTableOrPartitions(tableOrPartitions); |
| 434 | + } |
| 435 | + } |
| 436 | + |
390 | 437 | private void pendRemoteFetches( |
391 | 438 | RemoteLogFetchInfo remoteLogFetchInfo, long firstFetchOffset, long highWatermark) { |
392 | 439 | checkNotNull(remoteLogFetchInfo); |
@@ -417,7 +464,8 @@ private void pendRemoteFetches( |
417 | 464 | } |
418 | 465 | } |
419 | 466 |
|
420 | | - private Map<Integer, FetchLogRequest> prepareFetchLogRequests() { |
| 467 | + @VisibleForTesting |
| 468 | + Map<Integer, FetchLogRequest> prepareFetchLogRequests() { |
421 | 469 | Map<Integer, List<PbFetchLogReqForBucket>> fetchLogReqForBuckets = new HashMap<>(); |
422 | 470 | int readyForFetchCount = 0; |
423 | 471 | Long tableId = null; |
|
0 commit comments