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