001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.client; 019 020import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut; 021import static org.apache.hadoop.hbase.util.FutureUtils.addListener; 022 023import java.io.IOException; 024import java.util.ArrayList; 025import java.util.Iterator; 026import java.util.List; 027import java.util.concurrent.CompletableFuture; 028import java.util.concurrent.TimeUnit; 029import java.util.stream.Collectors; 030import java.util.stream.Stream; 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.hbase.TableName; 033import org.apache.yetus.audience.InterfaceAudience; 034 035import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 036import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; 037import org.apache.hbase.thirdparty.io.netty.util.Timeout; 038 039/** 040 * The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}. 041 */ 042@InterfaceAudience.Private 043class AsyncBufferedMutatorImpl implements AsyncBufferedMutator { 044 045 private final HashedWheelTimer periodicalFlushTimer; 046 047 private final AsyncTable<?> table; 048 049 private final long writeBufferSize; 050 051 private final long periodicFlushTimeoutNs; 052 053 private final int maxKeyValueSize; 054 055 private List<Mutation> mutations = new ArrayList<>(); 056 057 private List<CompletableFuture<Void>> futures = new ArrayList<>(); 058 059 private long bufferedSize; 060 061 private boolean closed; 062 063 @VisibleForTesting 064 Timeout periodicFlushTask; 065 066 AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table, 067 long writeBufferSize, long periodicFlushTimeoutNs, int maxKeyValueSize) { 068 this.periodicalFlushTimer = periodicalFlushTimer; 069 this.table = table; 070 this.writeBufferSize = writeBufferSize; 071 this.periodicFlushTimeoutNs = periodicFlushTimeoutNs; 072 this.maxKeyValueSize = maxKeyValueSize; 073 } 074 075 @Override 076 public TableName getName() { 077 return table.getName(); 078 } 079 080 @Override 081 public Configuration getConfiguration() { 082 return table.getConfiguration(); 083 } 084 085 // will be overridden in test 086 @VisibleForTesting 087 protected void internalFlush() { 088 if (periodicFlushTask != null) { 089 periodicFlushTask.cancel(); 090 periodicFlushTask = null; 091 } 092 List<Mutation> toSend = this.mutations; 093 if (toSend.isEmpty()) { 094 return; 095 } 096 List<CompletableFuture<Void>> toComplete = this.futures; 097 assert toSend.size() == toComplete.size(); 098 this.mutations = new ArrayList<>(); 099 this.futures = new ArrayList<>(); 100 bufferedSize = 0L; 101 Iterator<CompletableFuture<Void>> toCompleteIter = toComplete.iterator(); 102 for (CompletableFuture<?> future : table.batch(toSend)) { 103 CompletableFuture<Void> toCompleteFuture = toCompleteIter.next(); 104 addListener(future, (r, e) -> { 105 if (e != null) { 106 toCompleteFuture.completeExceptionally(e); 107 } else { 108 toCompleteFuture.complete(null); 109 } 110 }); 111 } 112 } 113 114 @Override 115 public List<CompletableFuture<Void>> mutate(List<? extends Mutation> mutations) { 116 List<CompletableFuture<Void>> futures = 117 Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutations.size()) 118 .collect(Collectors.toList()); 119 long heapSize = 0; 120 for (Mutation mutation : mutations) { 121 heapSize += mutation.heapSize(); 122 if (mutation instanceof Put) { 123 validatePut((Put)mutation, maxKeyValueSize); 124 } 125 } 126 synchronized (this) { 127 if (closed) { 128 IOException ioe = new IOException("Already closed"); 129 futures.forEach(f -> f.completeExceptionally(ioe)); 130 return futures; 131 } 132 if (this.mutations.isEmpty() && periodicFlushTimeoutNs > 0) { 133 periodicFlushTask = periodicalFlushTimer.newTimeout(timeout -> { 134 synchronized (AsyncBufferedMutatorImpl.this) { 135 // confirm that we are still valid, if there is already an internalFlush call before us, 136 // then we should not execute any more. And in internalFlush we will set periodicFlush 137 // to null, and since we may schedule a new one, so here we check whether the references 138 // are equal. 139 if (timeout == periodicFlushTask) { 140 periodicFlushTask = null; 141 internalFlush(); 142 } 143 } 144 }, periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 145 } 146 this.mutations.addAll(mutations); 147 this.futures.addAll(futures); 148 bufferedSize += heapSize; 149 if (bufferedSize >= writeBufferSize) { 150 internalFlush(); 151 } 152 } 153 return futures; 154 } 155 156 @Override 157 public synchronized void flush() { 158 internalFlush(); 159 } 160 161 @Override 162 public synchronized void close() { 163 internalFlush(); 164 closed = true; 165 } 166 167 @Override 168 public long getWriteBufferSize() { 169 return writeBufferSize; 170 } 171 172 @Override 173 public long getPeriodicalFlushTimeout(TimeUnit unit) { 174 return unit.convert(periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 175 } 176}