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.util.StringUtils;
32  
33  /**
34   * Default implementation of StoreFlusher.
35   */
36  @InterfaceAudience.Private
37  public class DefaultStoreFlusher extends StoreFlusher {
38    private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
39    private final Object flushLock = new Object();
40  
41    public DefaultStoreFlusher(Configuration conf, Store store) {
42      super(conf, store);
43    }
44  
45    @Override
46    public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
47        MonitoredTask status) throws IOException {
48      ArrayList<Path> result = new ArrayList<Path>();
49      int cellsCount = snapshot.getCellsCount();
50      if (cellsCount == 0) return result; // don't flush if there are no entries
51  
52      // Use a store scanner to find which rows to flush.
53      long smallestReadPoint = store.getSmallestReadPoint();
54      InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);
55      if (scanner == null) {
56        return result; // NULL scanner returned from coprocessor hooks means skip normal processing
57      }
58  
59      StoreFile.Writer writer;
60      try {
61        // TODO:  We can fail in the below block before we complete adding this flush to
62        //        list of store files.  Add cleanup of anything put on filesystem if we fail.
63        synchronized (flushLock) {
64          status.setStatus("Flushing " + store + ": creating writer");
65          // Write the map out to the disk
66          writer = store.createWriterInTmp(
67              cellsCount, store.getFamily().getCompression(), false, true, true);
68          writer.setTimeRangeTracker(snapshot.getTimeRangeTracker());
69          IOException e = null;
70          try {
71            performFlush(scanner, writer, smallestReadPoint);
72          } catch (IOException ioe) {
73            e = ioe;
74            // throw the exception out
75            throw ioe;
76          } finally {
77            if (e != null) {
78              writer.close();
79            } else {
80              finalizeWriter(writer, cacheFlushId, status);
81            }
82          }
83        }
84      } finally {
85        scanner.close();
86      }
87      LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
88          + StringUtils.humanReadableInt(snapshot.getSize()) +
89          ", hasBloomFilter=" + writer.hasGeneralBloom() +
90          ", into tmp file " + writer.getPath());
91      result.add(writer.getPath());
92      return result;
93    }
94  }