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 org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
021
022import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize;
023
024import java.io.IOException;
025import java.io.InterruptedIOException;
026import java.util.Queue;
027import java.util.concurrent.ConcurrentLinkedQueue;
028import java.util.concurrent.ExecutorService;
029import java.util.concurrent.LinkedBlockingQueue;
030import java.util.concurrent.atomic.AtomicBoolean;
031import java.util.concurrent.atomic.AtomicLong;
032import java.util.concurrent.locks.Condition;
033import java.util.concurrent.locks.Lock;
034import java.util.concurrent.locks.ReentrantLock;
035import java.util.function.Consumer;
036
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.yetus.audience.InterfaceAudience;
040import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
041import org.apache.hadoop.hbase.util.Threads;
042
043/**
044 * ClientAsyncPrefetchScanner implements async scanner behaviour.
045 * Specifically, the cache used by this scanner is a concurrent queue which allows both
046 * the producer (hbase client) and consumer (application) to access the queue in parallel.
047 * The number of rows returned in a prefetch is defined by the caching factor and the result size
048 * factor.
049 * This class allocates a buffer cache, whose size is a function of both factors.
050 * The prefetch is invoked when the cache is half­filled, instead of waiting for it to be empty.
051 * This is defined in the method {@link ClientAsyncPrefetchScanner#prefetchCondition()}.
052 */
053@InterfaceAudience.Private
054public class ClientAsyncPrefetchScanner extends ClientSimpleScanner {
055
056  private long maxCacheSize;
057  private AtomicLong cacheSizeInBytes;
058  // exception queue (from prefetch to main scan execution)
059  private Queue<Exception> exceptionsQueue;
060  // prefetch thread to be executed asynchronously
061  private Thread prefetcher;
062  // used for testing
063  private Consumer<Boolean> prefetchListener;
064
065  private final Lock lock = new ReentrantLock();
066  private final Condition notEmpty = lock.newCondition();
067  private final Condition notFull = lock.newCondition();
068
069  public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name,
070      ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
071      RpcControllerFactory rpcControllerFactory, ExecutorService pool,
072      int replicaCallTimeoutMicroSecondScan) throws IOException {
073    super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool,
074        replicaCallTimeoutMicroSecondScan);
075  }
076
077  @VisibleForTesting
078  void setPrefetchListener(Consumer<Boolean> prefetchListener) {
079    this.prefetchListener = prefetchListener;
080  }
081
082  @Override
083  protected void initCache() {
084    // concurrent cache
085    maxCacheSize = resultSize2CacheSize(maxScannerResultSize);
086    cache = new LinkedBlockingQueue<>();
087    cacheSizeInBytes = new AtomicLong(0);
088    exceptionsQueue = new ConcurrentLinkedQueue<>();
089    prefetcher = new Thread(new PrefetchRunnable());
090    Threads.setDaemonThreadRunning(prefetcher, tableName + ".asyncPrefetcher");
091  }
092
093  private long resultSize2CacheSize(long maxResultSize) {
094    // * 2 if possible
095    return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
096  }
097
098  @Override
099  public Result next() throws IOException {
100    try {
101      lock.lock();
102      while (cache.isEmpty()) {
103        handleException();
104        if (this.closed) {
105          return null;
106        }
107        try {
108          notEmpty.await();
109        } catch (InterruptedException e) {
110          throw new InterruptedIOException("Interrupted when wait to load cache");
111        }
112      }
113
114      Result result = pollCache();
115      if (prefetchCondition()) {
116        notFull.signalAll();
117      }
118      return result;
119    } finally {
120      lock.unlock();
121      handleException();
122    }
123  }
124
125  @Override
126  public void close() {
127    try {
128      lock.lock();
129      super.close();
130      closed = true;
131      notFull.signalAll();
132      notEmpty.signalAll();
133    } finally {
134      lock.unlock();
135    }
136  }
137
138  @Override
139  protected void addEstimatedSize(long estimatedSize) {
140    cacheSizeInBytes.addAndGet(estimatedSize);
141  }
142
143  private void handleException() throws IOException {
144    //The prefetch task running in the background puts any exception it
145    //catches into this exception queue.
146    // Rethrow the exception so the application can handle it.
147    while (!exceptionsQueue.isEmpty()) {
148      Exception first = exceptionsQueue.peek();
149      first.printStackTrace();
150      if (first instanceof IOException) {
151        throw (IOException) first;
152      }
153      throw (RuntimeException) first;
154    }
155  }
156
157  private boolean prefetchCondition() {
158    return cacheSizeInBytes.get() < maxCacheSize / 2;
159  }
160
161  private Result pollCache() {
162    Result res = cache.poll();
163    long estimatedSize = calcEstimatedSize(res);
164    addEstimatedSize(-estimatedSize);
165    return res;
166  }
167
168  private class PrefetchRunnable implements Runnable {
169
170    @Override
171    public void run() {
172      while (!closed) {
173        boolean succeed = false;
174        try {
175          lock.lock();
176          while (!prefetchCondition()) {
177            notFull.await();
178          }
179          loadCache();
180          succeed = true;
181        } catch (Exception e) {
182          exceptionsQueue.add(e);
183        } finally {
184          notEmpty.signalAll();
185          lock.unlock();
186          if (prefetchListener != null) {
187            prefetchListener.accept(succeed);
188          }
189        }
190      }
191    }
192
193  }
194
195}