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 static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.KeyValue.KVComparator;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
34  import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
35  import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
36  import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
37  
38  /**
39   * Stripe implementation of StoreFlusher. Flushes files either into L0 file w/o metadata, or
40   * into separate striped files, avoiding L0.
41   */
42  @InterfaceAudience.Private
43  public class StripeStoreFlusher extends StoreFlusher {
44    private static final Log LOG = LogFactory.getLog(StripeStoreFlusher.class);
45    private final Object flushLock = new Object();
46    private final StripeCompactionPolicy policy;
47    private final StripeCompactionPolicy.StripeInformationProvider stripes;
48  
49    public StripeStoreFlusher(Configuration conf, Store store,
50        StripeCompactionPolicy policy, StripeStoreFileManager stripes) {
51      super(conf, store);
52      this.policy = policy;
53      this.stripes = stripes;
54    }
55  
56    @Override
57    public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum,
58        MonitoredTask status, ThroughputController throughputController) throws IOException {
59      List<Path> result = new ArrayList<Path>();
60      int cellsCount = snapshot.getCellsCount();
61      if (cellsCount == 0) return result; // don't flush if there are no entries
62  
63      long smallestReadPoint = store.getSmallestReadPoint();
64      InternalScanner scanner = createScanner(snapshot.getScanner(), smallestReadPoint);
65      if (scanner == null) {
66        return result; // NULL scanner returned from coprocessor hooks means skip normal processing
67      }
68  
69      // Let policy select flush method.
70      StripeFlushRequest req = this.policy.selectFlush(store.getComparator(), this.stripes,
71        cellsCount);
72  
73      boolean success = false;
74      StripeMultiFileWriter mw = null;
75      try {
76        mw = req.createWriter(); // Writer according to the policy.
77        StripeMultiFileWriter.WriterFactory factory = createWriterFactory(
78            snapshot.getTimeRangeTracker(), cellsCount);
79        StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
80        mw.init(storeScanner, factory);
81  
82        synchronized (flushLock) {
83          performFlush(scanner, mw, smallestReadPoint, throughputController);
84          result = mw.commitWriters(cacheFlushSeqNum, false);
85          success = true;
86        }
87      } finally {
88        if (!success && (mw != null)) {
89          for (Path leftoverFile : mw.abortWriters()) {
90            try {
91              store.getFileSystem().delete(leftoverFile, false);
92            } catch (Exception e) {
93              LOG.error("Failed to delete a file after failed flush: " + e);
94            }
95          }
96        }
97        try {
98          scanner.close();
99        } catch (IOException ex) {
100         LOG.warn("Failed to close flush scanner, ignoring", ex);
101       }
102     }
103     return result;
104   }
105 
106   private StripeMultiFileWriter.WriterFactory createWriterFactory(
107       final TimeRangeTracker tracker, final long kvCount) {
108     return new StripeMultiFileWriter.WriterFactory() {
109       @Override
110       public Writer createWriter() throws IOException {
111         StoreFile.Writer writer = store.createWriterInTmp(
112             kvCount, store.getFamily().getCompression(),
113             /* isCompaction = */ false,
114             /* includeMVCCReadpoint = */ true,
115             /* includesTags = */ true,
116             /* shouldDropBehind = */ false,
117             tracker, -1);
118         return writer;
119       }
120     };
121   }
122 
123   /** Stripe flush request wrapper that writes a non-striped file. */
124   public static class StripeFlushRequest {
125 
126     protected final KVComparator comparator;
127 
128     public StripeFlushRequest(KVComparator comparator) {
129       this.comparator = comparator;
130     }
131 
132     public StripeMultiFileWriter createWriter() throws IOException {
133       StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(comparator, 1,
134           Long.MAX_VALUE, OPEN_KEY, OPEN_KEY);
135       writer.setNoStripeMetadata();
136       return writer;
137     }
138   }
139 
140   /** Stripe flush request wrapper based on boundaries. */
141   public static class BoundaryStripeFlushRequest extends StripeFlushRequest {
142     private final List<byte[]> targetBoundaries;
143 
144     /** @param targetBoundaries New files should be written with these boundaries. */
145     public BoundaryStripeFlushRequest(KVComparator comparator, List<byte[]> targetBoundaries) {
146       super(comparator);
147       this.targetBoundaries = targetBoundaries;
148     }
149 
150     @Override
151     public StripeMultiFileWriter createWriter() throws IOException {
152       return new StripeMultiFileWriter.BoundaryMultiWriter(comparator, targetBoundaries, null,
153           null);
154     }
155   }
156 
157   /** Stripe flush request wrapper based on size. */
158   public static class SizeStripeFlushRequest extends StripeFlushRequest {
159     private final int targetCount;
160     private final long targetKvs;
161 
162     /**
163      * @param targetCount The maximum number of stripes to flush into.
164      * @param targetKvs The KV count of each segment. If targetKvs*targetCount is less than
165      *                  total number of kvs, all the overflow data goes into the last stripe.
166      */
167     public SizeStripeFlushRequest(KVComparator comparator, int targetCount, long targetKvs) {
168       super(comparator);
169       this.targetCount = targetCount;
170       this.targetKvs = targetKvs;
171     }
172 
173     @Override
174     public StripeMultiFileWriter createWriter() throws IOException {
175       return new StripeMultiFileWriter.SizeMultiWriter(comparator, this.targetCount, this.targetKvs,
176           OPEN_KEY, OPEN_KEY);
177     }
178   }
179 }