|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.fluss.client.lookup; |
| 19 | + |
| 20 | +import org.apache.fluss.client.metadata.MetadataUpdater; |
| 21 | +import org.apache.fluss.memory.MemorySegment; |
| 22 | +import org.apache.fluss.metadata.Schema; |
| 23 | +import org.apache.fluss.metadata.SchemaGetter; |
| 24 | +import org.apache.fluss.metadata.SchemaInfo; |
| 25 | +import org.apache.fluss.metadata.TableInfo; |
| 26 | +import org.apache.fluss.row.InternalRow; |
| 27 | +import org.apache.fluss.row.decode.FixedSchemaDecoder; |
| 28 | +import org.apache.fluss.utils.CopyOnWriteMap; |
| 29 | +import org.apache.fluss.utils.concurrent.FutureUtils; |
| 30 | + |
| 31 | +import java.util.ArrayList; |
| 32 | +import java.util.Collection; |
| 33 | +import java.util.HashMap; |
| 34 | +import java.util.HashSet; |
| 35 | +import java.util.List; |
| 36 | +import java.util.Map; |
| 37 | +import java.util.Set; |
| 38 | +import java.util.concurrent.CompletableFuture; |
| 39 | + |
| 40 | +import static org.apache.fluss.utils.Preconditions.checkArgument; |
| 41 | + |
| 42 | +/** Abstract lookuper implementation for common methods. */ |
| 43 | +abstract class AbstractLookuper implements Lookuper { |
| 44 | + protected final TableInfo tableInfo; |
| 45 | + |
| 46 | + protected final MetadataUpdater metadataUpdater; |
| 47 | + |
| 48 | + protected final LookupClient lookupClient; |
| 49 | + |
| 50 | + protected final short targetSchemaId; |
| 51 | + |
| 52 | + private final SchemaGetter schemaGetter; |
| 53 | + |
| 54 | + /** |
| 55 | + * Cache for row decoders for different schema ids. Use CopyOnWriteMap for fast access, as it is |
| 56 | + * not frequently updated. |
| 57 | + */ |
| 58 | + private final CopyOnWriteMap<Short, FixedSchemaDecoder> decoders; |
| 59 | + |
| 60 | + AbstractLookuper( |
| 61 | + TableInfo tableInfo, |
| 62 | + MetadataUpdater metadataUpdater, |
| 63 | + LookupClient lookupClient, |
| 64 | + SchemaGetter schemaGetter) { |
| 65 | + this.tableInfo = tableInfo; |
| 66 | + this.metadataUpdater = metadataUpdater; |
| 67 | + this.lookupClient = lookupClient; |
| 68 | + this.targetSchemaId = (short) tableInfo.getSchemaId(); |
| 69 | + this.schemaGetter = schemaGetter; |
| 70 | + this.decoders = new CopyOnWriteMap<>(); |
| 71 | + // initialize the decoder for the same schema |
| 72 | + this.decoders.put( |
| 73 | + targetSchemaId, |
| 74 | + new FixedSchemaDecoder( |
| 75 | + tableInfo.getTableConfig().getKvFormat(), tableInfo.getSchema())); |
| 76 | + } |
| 77 | + |
| 78 | + protected void handleLookupResponse( |
| 79 | + List<byte[]> result, CompletableFuture<LookupResult> lookupFuture) { |
| 80 | + List<MemorySegment> valueList = new ArrayList<>(result.size()); |
| 81 | + Set<Short> schemaIdsToRequest = new HashSet<>(); |
| 82 | + boolean allTargetSchema = true; |
| 83 | + for (byte[] valueBytes : result) { |
| 84 | + if (valueBytes == null) { |
| 85 | + continue; |
| 86 | + } |
| 87 | + MemorySegment memorySegment = MemorySegment.wrap(valueBytes); |
| 88 | + short schemaId = memorySegment.getShort(0); |
| 89 | + if (targetSchemaId != schemaId) { |
| 90 | + allTargetSchema = false; |
| 91 | + if (!decoders.containsKey(schemaId)) { |
| 92 | + schemaIdsToRequest.add(schemaId); |
| 93 | + } |
| 94 | + } |
| 95 | + valueList.add(memorySegment); |
| 96 | + } |
| 97 | + // all schema ids are the target schema id, fast path |
| 98 | + if (allTargetSchema) { |
| 99 | + lookupFuture.complete(processAllTargetSchemaRows(valueList)); |
| 100 | + return; |
| 101 | + } |
| 102 | + |
| 103 | + // all schema ids and decoders are cached in local |
| 104 | + if (schemaIdsToRequest.isEmpty()) { |
| 105 | + lookupFuture.complete(processSchemaMismatchedRows(valueList)); |
| 106 | + return; |
| 107 | + } |
| 108 | + |
| 109 | + // need to fetch schema infos for schema ids |
| 110 | + List<CompletableFuture<SchemaInfo>> schemaFutures = |
| 111 | + new ArrayList<>(schemaIdsToRequest.size()); |
| 112 | + for (short schemaId : schemaIdsToRequest) { |
| 113 | + CompletableFuture<SchemaInfo> schemaFuture = schemaGetter.getSchemaInfoAsync(schemaId); |
| 114 | + schemaFutures.add(schemaFuture); |
| 115 | + } |
| 116 | + FutureUtils.ConjunctFuture<Collection<SchemaInfo>> allFutures = |
| 117 | + FutureUtils.combineAll(schemaFutures); |
| 118 | + // normal async path |
| 119 | + allFutures.whenComplete( |
| 120 | + (schemas, error) -> { |
| 121 | + if (error != null) { |
| 122 | + lookupFuture.completeExceptionally( |
| 123 | + new RuntimeException( |
| 124 | + "Failed to get schema infos for prefix lookup", error)); |
| 125 | + } else { |
| 126 | + LookupResult lookupResult = processSchemaRequestedRows(schemas, valueList); |
| 127 | + lookupFuture.complete(lookupResult); |
| 128 | + } |
| 129 | + }); |
| 130 | + } |
| 131 | + |
| 132 | + protected LookupResult processAllTargetSchemaRows(List<MemorySegment> valueList) { |
| 133 | + FixedSchemaDecoder decoder = decoders.get(targetSchemaId); |
| 134 | + List<InternalRow> rowList = new ArrayList<>(valueList.size()); |
| 135 | + for (MemorySegment value : valueList) { |
| 136 | + rowList.add(decoder.decode(value)); |
| 137 | + } |
| 138 | + return new LookupResult(rowList); |
| 139 | + } |
| 140 | + |
| 141 | + protected LookupResult processSchemaMismatchedRows(List<MemorySegment> valueList) { |
| 142 | + List<InternalRow> rowList = new ArrayList<>(valueList.size()); |
| 143 | + for (MemorySegment value : valueList) { |
| 144 | + short schemaId = value.getShort(0); |
| 145 | + FixedSchemaDecoder decoder = decoders.get(schemaId); |
| 146 | + checkArgument(decoder != null, "Decoder for schema id %s not found", schemaId); |
| 147 | + InternalRow row = decoder.decode(value); |
| 148 | + rowList.add(row); |
| 149 | + } |
| 150 | + return new LookupResult(rowList); |
| 151 | + } |
| 152 | + |
| 153 | + protected LookupResult processSchemaRequestedRows( |
| 154 | + Collection<SchemaInfo> schemaInfos, List<MemorySegment> valueList) { |
| 155 | + // build a map from schema id to the target schema decoder |
| 156 | + Map<Short, Schema> schemaMap = new HashMap<>(); |
| 157 | + for (SchemaInfo schemaInfo : schemaInfos) { |
| 158 | + schemaMap.put((short) schemaInfo.getSchemaId(), schemaInfo.getSchema()); |
| 159 | + } |
| 160 | + |
| 161 | + // process the value list to convert to target schema |
| 162 | + List<InternalRow> rowList = new ArrayList<>(valueList.size()); |
| 163 | + for (MemorySegment value : valueList) { |
| 164 | + short schemaId = value.getShort(0); |
| 165 | + FixedSchemaDecoder decoder = |
| 166 | + decoders.computeIfAbsent( |
| 167 | + schemaId, |
| 168 | + (id) -> { |
| 169 | + Schema sourceSchema = schemaMap.get(id); |
| 170 | + return new FixedSchemaDecoder( |
| 171 | + tableInfo.getTableConfig().getKvFormat(), |
| 172 | + sourceSchema, |
| 173 | + tableInfo.getSchema()); |
| 174 | + }); |
| 175 | + InternalRow row = decoder.decode(value); |
| 176 | + rowList.add(row); |
| 177 | + } |
| 178 | + return new LookupResult(rowList); |
| 179 | + } |
| 180 | +} |
0 commit comments