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  import java.util.SortedSet;
25  import java.util.concurrent.atomic.AtomicLong;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
34  import org.apache.hadoop.util.StringUtils;
35  
36  /**
37   * Default implementation of StoreFlusher.
38   */
39  @InterfaceAudience.Private
40  public class DefaultStoreFlusher extends StoreFlusher {
41    private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
42    private final Object flushLock = new Object();
43  
44    public DefaultStoreFlusher(Configuration conf, Store store) {
45      super(conf, store);
46    }
47  
48    @Override
49    public List<Path> flushSnapshot(SortedSet<KeyValue> snapshot, long cacheFlushId,
50        TimeRangeTracker snapshotTimeRangeTracker, AtomicLong flushedSize,
51        MonitoredTask status) throws IOException {
52      ArrayList<Path> result = new ArrayList<Path>();
53      if (snapshot.size() == 0) return result; // don't flush if there are no entries
54  
55      // Use a store scanner to find which rows to flush.
56      long smallestReadPoint = store.getSmallestReadPoint();
57      InternalScanner scanner = createScanner(snapshot, smallestReadPoint);
58      if (scanner == null) {
59        return result; // NULL scanner returned from coprocessor hooks means skip normal processing
60      }
61  
62      StoreFile.Writer writer;
63      long flushed = 0;
64      try {
65        // TODO:  We can fail in the below block before we complete adding this flush to
66        //        list of store files.  Add cleanup of anything put on filesystem if we fail.
67        synchronized (flushLock) {
68          status.setStatus("Flushing " + store + ": creating writer");
69          // Write the map out to the disk
70          writer = store.createWriterInTmp(
71              snapshot.size(), store.getFamily().getCompression(), false, true, true);
72          writer.setTimeRangeTracker(snapshotTimeRangeTracker);
73          IOException e = null;
74          try {
75            flushed = performFlush(scanner, writer, smallestReadPoint);
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        flushedSize.set(flushed);
90        scanner.close();
91      }
92      LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
93          + StringUtils.humanReadableInt(flushed) +
94          ", hasBloomFilter=" + writer.hasGeneralBloom() +
95          ", into tmp file " + writer.getPath());
96      result.add(writer.getPath());
97      return result;
98    }
99  }