View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.List;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
31  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
32  import org.apache.hadoop.util.StringUtils;
33  
34  /**
35   * Default implementation of StoreFlusher.
36   */
37  @InterfaceAudience.Private
38  public class DefaultStoreFlusher extends StoreFlusher {
39    private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
40    private final Object flushLock = new Object();
41  
42    public DefaultStoreFlusher(Configuration conf, Store store) {
43      super(conf, store);
44    }
45  
46    @Override
47    public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
48        MonitoredTask status, ThroughputController throughputController) throws IOException {
49      ArrayList<Path> result = new ArrayList<Path>();
50      int cellsCount = snapshot.getCellsCount();
51      if (cellsCount == 0) return result; // don't flush if there are no entries
52  
53      // Use a store scanner to find which rows to flush.
54      long smallestReadPoint = store.getSmallestReadPoint();
55      InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);
56      if (scanner == null) {
57        return result; // NULL scanner returned from coprocessor hooks means skip normal processing
58      }
59  
60      StoreFile.Writer writer;
61      try {
62        // TODO:  We can fail in the below block before we complete adding this flush to
63        //        list of store files.  Add cleanup of anything put on filesystem if we fail.
64        synchronized (flushLock) {
65          status.setStatus("Flushing " + store + ": creating writer");
66          // Write the map out to the disk
67          writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompression(),
68              /* isCompaction = */ false,
69              /* includeMVCCReadpoint = */ true,
70              /* includesTags = */ snapshot.isTagsPresent(),
71              /* shouldDropBehind = */ false,
72              snapshot.getTimeRangeTracker(), -1);
73          IOException e = null;
74          try {
75            performFlush(scanner, writer, smallestReadPoint, throughputController);
76          } catch (IOException ioe) {
77            e = ioe;
78            // throw the exception out
79            throw ioe;
80          } finally {
81            if (e != null) {
82              writer.close();
83            } else {
84              finalizeWriter(writer, cacheFlushId, status);
85            }
86          }
87        }
88      } finally {
89        scanner.close();
90      }
91      LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
92          + StringUtils.humanReadableInt(snapshot.getSize()) +
93          ", hasBloomFilter=" + writer.hasGeneralBloom() +
94          ", into tmp file " + writer.getPath());
95      result.add(writer.getPath());
96      return result;
97    }
98  }