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.util.FutureUtils.addListener;
021
022import java.io.IOException;
023import java.util.ArrayList;
024import java.util.Iterator;
025import java.util.List;
026import java.util.concurrent.CompletableFuture;
027import java.util.concurrent.TimeUnit;
028import java.util.stream.Collectors;
029import java.util.stream.Stream;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.yetus.audience.InterfaceAudience;
033
034import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
035import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
036import org.apache.hbase.thirdparty.io.netty.util.Timeout;
037
038/**
039 * The implementation of {@link AsyncBufferedMutator}. Simply wrap an {@link AsyncTable}.
040 */
041@InterfaceAudience.Private
042class AsyncBufferedMutatorImpl implements AsyncBufferedMutator {
043
044  private final HashedWheelTimer periodicalFlushTimer;
045
046  private final AsyncTable<?> table;
047
048  private final long writeBufferSize;
049
050  private final long periodicFlushTimeoutNs;
051
052  private List<Mutation> mutations = new ArrayList<>();
053
054  private List<CompletableFuture<Void>> futures = new ArrayList<>();
055
056  private long bufferedSize;
057
058  private boolean closed;
059
060  @VisibleForTesting
061  Timeout periodicFlushTask;
062
063  AsyncBufferedMutatorImpl(HashedWheelTimer periodicalFlushTimer, AsyncTable<?> table,
064      long writeBufferSize, long periodicFlushTimeoutNs) {
065    this.periodicalFlushTimer = periodicalFlushTimer;
066    this.table = table;
067    this.writeBufferSize = writeBufferSize;
068    this.periodicFlushTimeoutNs = periodicFlushTimeoutNs;
069  }
070
071  @Override
072  public TableName getName() {
073    return table.getName();
074  }
075
076  @Override
077  public Configuration getConfiguration() {
078    return table.getConfiguration();
079  }
080
081  // will be overridden in test
082  @VisibleForTesting
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 = mutations.stream().mapToLong(m -> m.heapSize()).sum();
116    synchronized (this) {
117      if (closed) {
118        IOException ioe = new IOException("Already closed");
119        futures.forEach(f -> f.completeExceptionally(ioe));
120        return futures;
121      }
122      if (this.mutations.isEmpty() && periodicFlushTimeoutNs > 0) {
123        periodicFlushTask = periodicalFlushTimer.newTimeout(timeout -> {
124          synchronized (AsyncBufferedMutatorImpl.this) {
125            // confirm that we are still valid, if there is already an internalFlush call before us,
126            // then we should not execute any more. And in internalFlush we will set periodicFlush
127            // to null, and since we may schedule a new one, so here we check whether the references
128            // are equal.
129            if (timeout == periodicFlushTask) {
130              periodicFlushTask = null;
131              internalFlush();
132            }
133          }
134        }, periodicFlushTimeoutNs, TimeUnit.NANOSECONDS);
135      }
136      this.mutations.addAll(mutations);
137      this.futures.addAll(futures);
138      bufferedSize += heapSize;
139      if (bufferedSize >= writeBufferSize) {
140        internalFlush();
141      }
142    }
143    return futures;
144  }
145
146  @Override
147  public synchronized void flush() {
148    internalFlush();
149  }
150
151  @Override
152  public synchronized void close() {
153    internalFlush();
154    closed = true;
155  }
156
157  @Override
158  public long getWriteBufferSize() {
159    return writeBufferSize;
160  }
161
162  @Override
163  public long getPeriodicalFlushTimeout(TimeUnit unit) {
164    return unit.convert(periodicFlushTimeoutNs, TimeUnit.NANOSECONDS);
165  }
166}