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.io.InterruptedIOException;
23  import java.util.ArrayList;
24  import java.util.Collections;
25  import java.util.List;
26  
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.HConstants;
31  import org.apache.hadoop.hbase.KeyValueUtil;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.client.Scan;
34  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
35  import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
36  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
37  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
38  
39  /**
40   * Store flusher interface. Turns a snapshot of memstore into a set of store files (usually one).
41   * Custom implementation can be provided.
42   */
43  @InterfaceAudience.Private
44  abstract class StoreFlusher {
45    protected Configuration conf;
46    protected Store store;
47  
48    public StoreFlusher(Configuration conf, Store store) {
49      this.conf = conf;
50      this.store = store;
51    }
52  
53    /**
54     * Turns a snapshot of memstore into a set of store files.
55     * @param snapshot Memstore snapshot.
56     * @param cacheFlushSeqNum Log cache flush sequence number.
57     * @param status Task that represents the flush operation and may be updated with status.
58     * @param throughputController A controller to avoid flush too fast
59     * @return List of files written. Can be empty; must not be null.
60     */
61    public abstract List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum,
62        MonitoredTask status, ThroughputController throughputController) throws IOException;
63  
64    protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum,
65        MonitoredTask status) throws IOException {
66      // Write out the log sequence number that corresponds to this output
67      // hfile. Also write current time in metadata as minFlushTime.
68      // The hfile is current up to and including cacheFlushSeqNum.
69      status.setStatus("Flushing " + store + ": appending metadata");
70      writer.appendMetadata(cacheFlushSeqNum, false);
71      status.setStatus("Flushing " + store + ": closing flushed file");
72      writer.close();
73    }
74  
75  
76    /**
77     * Creates the scanner for flushing snapshot. Also calls coprocessors.
78     * @param snapshotScanner
79     * @param smallestReadPoint
80     * @return The scanner; null if coprocessor is canceling the flush.
81     */
82    protected InternalScanner createScanner(KeyValueScanner snapshotScanner,
83        long smallestReadPoint) throws IOException {
84      InternalScanner scanner = null;
85      if (store.getCoprocessorHost() != null) {
86        scanner = store.getCoprocessorHost().preFlushScannerOpen(store, snapshotScanner);
87      }
88      if (scanner == null) {
89        Scan scan = new Scan();
90        scan.setMaxVersions(store.getScanInfo().getMaxVersions());
91        scanner = new StoreScanner(store, store.getScanInfo(), scan,
92            Collections.singletonList(snapshotScanner), ScanType.COMPACT_RETAIN_DELETES,
93            smallestReadPoint, HConstants.OLDEST_TIMESTAMP);
94      }
95      assert scanner != null;
96      if (store.getCoprocessorHost() != null) {
97        try {
98          return store.getCoprocessorHost().preFlush(store, scanner);
99        } catch (IOException ioe) {
100         scanner.close();
101         throw ioe;
102       }
103     }
104     return scanner;
105   }
106 
107   /**
108    * Performs memstore flush, writing data from scanner into sink.
109    * @param scanner Scanner to get data from.
110    * @param sink Sink to write data to. Could be StoreFile.Writer.
111    * @param smallestReadPoint Smallest read point used for the flush.
112    * @param throughputController A controller to avoid flush too fast
113    */
114   protected void performFlush(InternalScanner scanner, Compactor.CellSink sink,
115       long smallestReadPoint, ThroughputController throughputController) throws IOException {
116     int compactionKVMax =
117       conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
118 
119     ScannerContext scannerContext =
120         ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
121 
122     List<Cell> kvs = new ArrayList<Cell>();
123     boolean hasMore;
124     String flushName = ThroughputControlUtil.getNameForThrottling(store, "flush");
125     // no control on system table (such as meta, namespace, etc) flush
126     boolean control = throughputController != null && !store.getRegionInfo().isSystemTable();
127     if (control) {
128       throughputController.start(flushName);
129     }
130     try {
131       do {
132         hasMore = scanner.next(kvs, scannerContext);
133         if (!kvs.isEmpty()) {
134           for (Cell c : kvs) {
135             // If we know that this KV is going to be included always, then let us
136             // set its memstoreTS to 0. This will help us save space when writing to
137             // disk.
138             sink.append(c);
139             int len = KeyValueUtil.length(c);
140             if (control) {
141               throughputController.control(flushName, len);
142             }
143           }
144           kvs.clear();
145         }
146       } while (hasMore);
147     } catch (InterruptedException e) {
148       throw new InterruptedIOException("Interrupted while control throughput of flushing "
149           + flushName);
150     } finally {
151       if (control) {
152         throughputController.finish(flushName);
153       }
154     }
155   }
156 }