|
| 1 | +/** |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.hbase.client; |
| 19 | + |
| 20 | +import static org.apache.hadoop.hbase.util.FutureUtils.addListener; |
| 21 | + |
| 22 | +import java.io.IOException; |
| 23 | +import java.util.ArrayList; |
| 24 | +import java.util.Collections; |
| 25 | +import java.util.List; |
| 26 | +import java.util.concurrent.CompletableFuture; |
| 27 | +import java.util.concurrent.CompletionException; |
| 28 | +import java.util.concurrent.ConcurrentLinkedQueue; |
| 29 | +import java.util.regex.Matcher; |
| 30 | +import java.util.regex.Pattern; |
| 31 | +import java.util.stream.Collectors; |
| 32 | +import org.apache.hadoop.conf.Configuration; |
| 33 | +import org.apache.hadoop.hbase.TableName; |
| 34 | +import org.apache.hadoop.hbase.util.Pair; |
| 35 | +import org.apache.yetus.audience.InterfaceAudience; |
| 36 | + |
| 37 | +/** |
| 38 | + * {@link BufferedMutator} implementation based on {@link AsyncBufferedMutator}. |
| 39 | + */ |
| 40 | +@InterfaceAudience.Private |
| 41 | +class BufferedMutatorOverAsyncBufferedMutator implements BufferedMutator { |
| 42 | + |
| 43 | + private final AsyncBufferedMutator mutator; |
| 44 | + |
| 45 | + private final ExceptionListener listener; |
| 46 | + |
| 47 | + private List<CompletableFuture<Void>> futures = new ArrayList<>(); |
| 48 | + |
| 49 | + private final ConcurrentLinkedQueue<Pair<Mutation, Throwable>> errors = |
| 50 | + new ConcurrentLinkedQueue<>(); |
| 51 | + |
| 52 | + private final static int BUFFERED_FUTURES_THRESHOLD = 1024; |
| 53 | + |
| 54 | + BufferedMutatorOverAsyncBufferedMutator(AsyncBufferedMutator mutator, |
| 55 | + ExceptionListener listener) { |
| 56 | + this.mutator = mutator; |
| 57 | + this.listener = listener; |
| 58 | + } |
| 59 | + |
| 60 | + @Override |
| 61 | + public TableName getName() { |
| 62 | + return mutator.getName(); |
| 63 | + } |
| 64 | + |
| 65 | + @Override |
| 66 | + public Configuration getConfiguration() { |
| 67 | + return mutator.getConfiguration(); |
| 68 | + } |
| 69 | + |
| 70 | + @Override |
| 71 | + public void mutate(Mutation mutation) throws IOException { |
| 72 | + mutate(Collections.singletonList(mutation)); |
| 73 | + } |
| 74 | + |
| 75 | + private static final Pattern ADDR_MSG_MATCHER = Pattern.compile("Call to (\\S+) failed"); |
| 76 | + |
| 77 | + // not always work, so may return an empty string |
| 78 | + private String getHostnameAndPort(Throwable error) { |
| 79 | + Matcher matcher = ADDR_MSG_MATCHER.matcher(error.getMessage()); |
| 80 | + if (matcher.matches()) { |
| 81 | + return matcher.group(1); |
| 82 | + } else { |
| 83 | + return ""; |
| 84 | + } |
| 85 | + } |
| 86 | + |
| 87 | + private RetriesExhaustedWithDetailsException makeError() { |
| 88 | + List<Row> rows = new ArrayList<>(); |
| 89 | + List<Throwable> throwables = new ArrayList<>(); |
| 90 | + List<String> hostnameAndPorts = new ArrayList<>(); |
| 91 | + for (;;) { |
| 92 | + Pair<Mutation, Throwable> pair = errors.poll(); |
| 93 | + if (pair == null) { |
| 94 | + break; |
| 95 | + } |
| 96 | + rows.add(pair.getFirst()); |
| 97 | + throwables.add(pair.getSecond()); |
| 98 | + hostnameAndPorts.add(getHostnameAndPort(pair.getSecond())); |
| 99 | + } |
| 100 | + return new RetriesExhaustedWithDetailsException(throwables, rows, hostnameAndPorts); |
| 101 | + } |
| 102 | + |
| 103 | + @Override |
| 104 | + public void mutate(List<? extends Mutation> mutations) throws IOException { |
| 105 | + List<CompletableFuture<Void>> toBuffered = new ArrayList<>(); |
| 106 | + List<CompletableFuture<Void>> fs = mutator.mutate(mutations); |
| 107 | + for (int i = 0, n = fs.size(); i < n; i++) { |
| 108 | + CompletableFuture<Void> toComplete = new CompletableFuture<>(); |
| 109 | + final int index = i; |
| 110 | + addListener(fs.get(index), (r, e) -> { |
| 111 | + if (e != null) { |
| 112 | + errors.add(Pair.newPair(mutations.get(index), e)); |
| 113 | + toComplete.completeExceptionally(e); |
| 114 | + } else { |
| 115 | + toComplete.complete(r); |
| 116 | + } |
| 117 | + }); |
| 118 | + toBuffered.add(toComplete); |
| 119 | + } |
| 120 | + synchronized (this) { |
| 121 | + futures.addAll(toBuffered); |
| 122 | + if (futures.size() > BUFFERED_FUTURES_THRESHOLD) { |
| 123 | + tryCompleteFuture(); |
| 124 | + } |
| 125 | + if (!errors.isEmpty()) { |
| 126 | + RetriesExhaustedWithDetailsException error = makeError(); |
| 127 | + listener.onException(error, this); |
| 128 | + } |
| 129 | + } |
| 130 | + } |
| 131 | + |
| 132 | + private void tryCompleteFuture() { |
| 133 | + futures = futures.stream().filter(f -> !f.isDone()).collect(Collectors.toList()); |
| 134 | + } |
| 135 | + |
| 136 | + @Override |
| 137 | + public void close() throws IOException { |
| 138 | + flush(); |
| 139 | + mutator.close(); |
| 140 | + } |
| 141 | + |
| 142 | + @Override |
| 143 | + public void flush() throws IOException { |
| 144 | + mutator.flush(); |
| 145 | + synchronized (this) { |
| 146 | + List<CompletableFuture<Void>> toComplete = this.futures; |
| 147 | + this.futures = new ArrayList<>(); |
| 148 | + try { |
| 149 | + CompletableFuture.allOf(toComplete.toArray(new CompletableFuture<?>[toComplete.size()])) |
| 150 | + .join(); |
| 151 | + } catch (CompletionException e) { |
| 152 | + // just ignore, we will record the actual error in the errors field |
| 153 | + } |
| 154 | + if (!errors.isEmpty()) { |
| 155 | + RetriesExhaustedWithDetailsException error = makeError(); |
| 156 | + listener.onException(error, this); |
| 157 | + } |
| 158 | + } |
| 159 | + } |
| 160 | + |
| 161 | + @Override |
| 162 | + public long getWriteBufferSize() { |
| 163 | + return mutator.getWriteBufferSize(); |
| 164 | + } |
| 165 | + |
| 166 | + @Override |
| 167 | + public void setRpcTimeout(int timeout) { |
| 168 | + // no effect |
| 169 | + } |
| 170 | + |
| 171 | + @Override |
| 172 | + public void setOperationTimeout(int timeout) { |
| 173 | + // no effect |
| 174 | + } |
| 175 | +} |
0 commit comments