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.hbase.thirdparty.io.netty.util.HashedWheelTimer; 034import org.apache.hbase.thirdparty.io.netty.util.Timeout; 035import org.apache.yetus.audience.InterfaceAudience; 036 037/** 038 * The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}. 039 */ 040@InterfaceAudience.Private 041class AsyncBufferedMutatorImpl implements AsyncBufferedMutator { 042 043 private final HashedWheelTimer periodicalFlushTimer; 044 045 private final AsyncTable<?> table; 046 047 private final long writeBufferSize; 048 049 private final long periodicFlushTimeoutNs; 050 051 private final int maxKeyValueSize; 052 053 private List<Mutation> mutations = new ArrayList<>(); 054 055 private List<CompletableFuture<Void>> futures = new ArrayList<>(); 056 057 private long bufferedSize; 058 059 private boolean closed; 060 061 Timeout periodicFlushTask; 062 063 AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table, 064 long writeBufferSize, long periodicFlushTimeoutNs, int maxKeyValueSize) { 065 this.periodicalFlushTimer = periodicalFlushTimer; 066 this.table = table; 067 this.writeBufferSize = writeBufferSize; 068 this.periodicFlushTimeoutNs = periodicFlushTimeoutNs; 069 this.maxKeyValueSize = maxKeyValueSize; 070 } 071 072 @Override 073 public TableName getName() { 074 return table.getName(); 075 } 076 077 @Override 078 public Configuration getConfiguration() { 079 return table.getConfiguration(); 080 } 081 082 // will be overridden in test 083 protected void internalFlush() { 084 if (periodicFlushTask != null) { 085 periodicFlushTask.cancel(); 086 periodicFlushTask = null; 087 } 088 List<Mutation> toSend = this.mutations; 089 if (toSend.isEmpty()) { 090 return; 091 } 092 List<CompletableFuture<Void>> toComplete = this.futures; 093 assert toSend.size() == toComplete.size(); 094 this.mutations = new ArrayList<>(); 095 this.futures = new ArrayList<>(); 096 bufferedSize = 0L; 097 Iterator<CompletableFuture<Void>> toCompleteIter = toComplete.iterator(); 098 for (CompletableFuture<?> future : table.batch(toSend)) { 099 CompletableFuture<Void> toCompleteFuture = toCompleteIter.next(); 100 addListener(future, (r, e) -> { 101 if (e != null) { 102 toCompleteFuture.completeExceptionally(e); 103 } else { 104 toCompleteFuture.complete(null); 105 } 106 }); 107 } 108 } 109 110 @Override 111 public List<CompletableFuture<Void>> mutate(List<? extends Mutation> mutations) { 112 List<CompletableFuture<Void>> futures = 113 Stream.<CompletableFuture<Void>> generate(CompletableFuture::new).limit(mutations.size()) 114 .collect(Collectors.toList()); 115 long heapSize = 0; 116 for (Mutation mutation : mutations) { 117 heapSize += mutation.heapSize(); 118 if (mutation instanceof Put) { 119 validatePut((Put)mutation, maxKeyValueSize); 120 } 121 } 122 synchronized (this) { 123 if (closed) { 124 IOException ioe = new IOException("Already closed"); 125 futures.forEach(f -> f.completeExceptionally(ioe)); 126 return futures; 127 } 128 if (this.mutations.isEmpty() && periodicFlushTimeoutNs > 0) { 129 periodicFlushTask = periodicalFlushTimer.newTimeout(timeout -> { 130 synchronized (AsyncBufferedMutatorImpl.this) { 131 // confirm that we are still valid, if there is already an internalFlush call before us, 132 // then we should not execute any more. And in internalFlush we will set periodicFlush 133 // to null, and since we may schedule a new one, so here we check whether the references 134 // are equal. 135 if (timeout == periodicFlushTask) { 136 periodicFlushTask = null; 137 internalFlush(); 138 } 139 } 140 }, periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 141 } 142 this.mutations.addAll(mutations); 143 this.futures.addAll(futures); 144 bufferedSize += heapSize; 145 if (bufferedSize >= writeBufferSize) { 146 internalFlush(); 147 } 148 } 149 return futures; 150 } 151 152 @Override 153 public synchronized void flush() { 154 internalFlush(); 155 } 156 157 @Override 158 public synchronized void close() { 159 internalFlush(); 160 closed = true; 161 } 162 163 @Override 164 public long getWriteBufferSize() { 165 return writeBufferSize; 166 } 167 168 @Override 169 public long getPeriodicalFlushTimeout(TimeUnit unit) { 170 return unit.convert(periodicFlushTimeoutNs, TimeUnit.NANOSECONDS); 171 } 172}