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.Map; 028import java.util.concurrent.CompletableFuture; 029import java.util.concurrent.TimeUnit; 030import java.util.stream.Collectors; 031import java.util.stream.Stream; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.hbase.TableName; 034import org.apache.yetus.audience.InterfaceAudience; 035 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 Timeout periodicFlushTask; 064 065 AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table, 066 long writeBufferSize, long periodicFlushTimeoutNs, int maxKeyValueSize) { 067 this.periodicalFlushTimer = periodicalFlushTimer; 068 this.table = table; 069 this.writeBufferSize = writeBufferSize; 070 this.periodicFlushTimeoutNs = periodicFlushTimeoutNs; 071 this.maxKeyValueSize = maxKeyValueSize; 072 } 073 074 @Override 075 public TableName getName() { 076 return table.getName(); 077 } 078 079 @Override 080 public Configuration getConfiguration() { 081 return table.getConfiguration(); 082 } 083 084 // will be overridden in test 085 protected void internalFlush() { 086 if (periodicFlushTask != null) { 087 periodicFlushTask.cancel(); 088 periodicFlushTask = null; 089 } 090 List<Mutation> toSend = this.mutations; 091 if (toSend.isEmpty()) { 092 return; 093 } 094 List<CompletableFuture<Void>> toComplete = this.futures; 095 assert toSend.size() == toComplete.size(); 096 this.mutations = new ArrayList<>(); 097 this.futures = new ArrayList<>(); 098 bufferedSize = 0L; 099 Iterator<CompletableFuture<Void>> toCompleteIter = toComplete.iterator(); 100 for (CompletableFuture<?> future : table.batch(toSend)) { 101 CompletableFuture<Void> toCompleteFuture = toCompleteIter.next(); 102 addListener(future, (r, e) -> { 103 if (e != null) { 104 toCompleteFuture.completeExceptionally(e); 105 } else { 106 toCompleteFuture.complete(null); 107 } 108 }); 109 } 110 } 111 112 @Override 113 public List<CompletableFuture<Void>> mutate(List<? extends Mutation> mutations) { 114 List<CompletableFuture<Void>> futures = 115 Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutations.size()) 116 .collect(Collectors.toList()); 117 long heapSize = 0; 118 for (Mutation mutation : mutations) { 119 heapSize += mutation.heapSize(); 120 if (mutation instanceof Put) { 121 validatePut((Put) mutation, maxKeyValueSize); 122 } 123 } 124 synchronized (this) { 125 if (closed) { 126 IOException ioe = new IOException("Already closed"); 127 futures.forEach(f -> f.completeExceptionally(ioe)); 128 return futures; 129 } 130 if (this.mutations.isEmpty() && periodicFlushTimeoutNs > 0) { 131 periodicFlushTask = periodicalFlushTimer.newTimeout(timeout -> { 132 synchronized (AsyncBufferedMutatorImpl.this) { 133 // confirm that we are still valid, if there is already an internalFlush call before us, 134 // then we should not execute anymore. And in internalFlush we will set periodicFlush 135 // to null, and since we may schedule a new one, so here we check whether the references 136 // are equal. 137 if (timeout == periodicFlushTask) { 138 periodicFlushTask = null; 139 internalFlush(); 140 } 141 } 142 }, periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 143 } 144 this.mutations.addAll(mutations); 145 this.futures.addAll(futures); 146 bufferedSize += heapSize; 147 if (bufferedSize >= writeBufferSize) { 148 internalFlush(); 149 } 150 } 151 return futures; 152 } 153 154 @Override 155 public synchronized void flush() { 156 internalFlush(); 157 } 158 159 @Override 160 public synchronized void close() { 161 internalFlush(); 162 closed = true; 163 } 164 165 @Override 166 public long getWriteBufferSize() { 167 return writeBufferSize; 168 } 169 170 @Override 171 public long getPeriodicalFlushTimeout(TimeUnit unit) { 172 return unit.convert(periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 173 } 174 175 @Override 176 public Map<String, byte[]> getRequestAttributes() { 177 return table.getRequestAttributes(); 178 } 179}