View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collections;
24  import java.util.List;
25  
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.Cell;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.client.Scan;
32  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
33  import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
34  
35  /**
36   * Store flusher interface. Turns a snapshot of memstore into a set of store files (usually one).
37   * Custom implementation can be provided.
38   */
39  @InterfaceAudience.Private
40  abstract class StoreFlusher {
41    protected Configuration conf;
42    protected Store store;
43  
44    public StoreFlusher(Configuration conf, Store store) {
45      this.conf = conf;
46      this.store = store;
47    }
48  
49    /**
50     * Turns a snapshot of memstore into a set of store files.
51     * @param snapshot Memstore snapshot.
52     * @param cacheFlushSeqNum Log cache flush sequence number.
53     * @param status Task that represents the flush operation and may be updated with status.
54     * @return List of files written. Can be empty; must not be null.
55     */
56    public abstract List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum,
57        MonitoredTask status) throws IOException;
58  
59    protected void finalizeWriter(StoreFile.Writer writer, long cacheFlushSeqNum,
60        MonitoredTask status) throws IOException {
61      // Write out the log sequence number that corresponds to this output
62      // hfile. Also write current time in metadata as minFlushTime.
63      // The hfile is current up to and including cacheFlushSeqNum.
64      status.setStatus("Flushing " + store + ": appending metadata");
65      writer.appendMetadata(cacheFlushSeqNum, false);
66      status.setStatus("Flushing " + store + ": closing flushed file");
67      writer.close();
68    }
69  
70  
71    /**
72     * Creates the scanner for flushing snapshot. Also calls coprocessors.
73     * @param snapshotScanner
74     * @param smallestReadPoint
75     * @return The scanner; null if coprocessor is canceling the flush.
76     */
77    protected InternalScanner createScanner(KeyValueScanner snapshotScanner,
78        long smallestReadPoint) throws IOException {
79      InternalScanner scanner = null;
80      if (store.getCoprocessorHost() != null) {
81        scanner = store.getCoprocessorHost().preFlushScannerOpen(store, snapshotScanner);
82      }
83      if (scanner == null) {
84        Scan scan = new Scan();
85        scan.setMaxVersions(store.getScanInfo().getMaxVersions());
86        scanner = new StoreScanner(store, store.getScanInfo(), scan,
87            Collections.singletonList(snapshotScanner), ScanType.COMPACT_RETAIN_DELETES,
88            smallestReadPoint, HConstants.OLDEST_TIMESTAMP);
89      }
90      assert scanner != null;
91      if (store.getCoprocessorHost() != null) {
92        try {
93          return store.getCoprocessorHost().preFlush(store, scanner);
94        } catch (IOException ioe) {
95          scanner.close();
96          throw ioe;
97        }
98      }
99      return scanner;
100   }
101 
102   /**
103    * Performs memstore flush, writing data from scanner into sink.
104    * @param scanner Scanner to get data from.
105    * @param sink Sink to write data to. Could be StoreFile.Writer.
106    * @param smallestReadPoint Smallest read point used for the flush.
107    */
108   protected void performFlush(InternalScanner scanner,
109       Compactor.CellSink sink, long smallestReadPoint) throws IOException {
110     int compactionKVMax =
111       conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
112 
113     ScannerContext scannerContext =
114         ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
115 
116     List<Cell> kvs = new ArrayList<Cell>();
117     boolean hasMore;
118     do {
119       hasMore = scanner.next(kvs, scannerContext);
120       if (!kvs.isEmpty()) {
121         for (Cell c : kvs) {
122           // If we know that this KV is going to be included always, then let us
123           // set its memstoreTS to 0. This will help us save space when writing to
124           // disk.
125           sink.append(c);
126         }
127         kvs.clear();
128       }
129     } while (hasMore);
130   }
131 }