Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5FB9D19811 for ; Thu, 14 Apr 2016 05:43:13 +0000 (UTC) Received: (qmail 23691 invoked by uid 500); 14 Apr 2016 05:43:13 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 23535 invoked by uid 500); 14 Apr 2016 05:43:13 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 23442 invoked by uid 99); 14 Apr 2016 05:43:12 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 14 Apr 2016 05:43:12 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id A1893DFB73; Thu, 14 Apr 2016 05:43:12 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: stack@apache.org To: commits@hbase.apache.org Date: Thu, 14 Apr 2016 05:43:13 -0000 Message-Id: <015b636b8519478992a83cceeec97a9e@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/4] hbase git commit: HBASE-15296 Break out writer and reader from StoreFile. Done using Intellij15 Refactor > Move. (Apekshit) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java new file mode 100644 index 0000000..3a7ae5e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -0,0 +1,514 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import com.google.common.base.Preconditions; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.regionserver.compactions.Compactor; +import org.apache.hadoop.hbase.util.BloomFilterFactory; +import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.WritableUtils; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Arrays; + +/** + * A StoreFile writer. Use this to read/write HBase Store Files. It is package + * local because it is an implementation detail of the HBase regionserver. + */ +@InterfaceAudience.Private +public class StoreFileWriter implements Compactor.CellSink { + private static final Log LOG = LogFactory.getLog(StoreFileWriter.class.getName()); + + private final BloomFilterWriter generalBloomFilterWriter; + private final BloomFilterWriter deleteFamilyBloomFilterWriter; + private final BloomType bloomType; + private byte[] lastBloomKey; + private int lastBloomKeyOffset, lastBloomKeyLen; + private Cell lastCell = null; + private long earliestPutTs = HConstants.LATEST_TIMESTAMP; + private Cell lastDeleteFamilyCell = null; + private long deleteFamilyCnt = 0; + + TimeRangeTracker timeRangeTracker = new TimeRangeTracker(); + /* isTimeRangeTrackerSet keeps track if the timeRange has already been set + * When flushing a memstore, we set TimeRange and use this variable to + * indicate that it doesn't need to be calculated again while + * appending KeyValues. + * It is not set in cases of compactions when it is recalculated using only + * the appended KeyValues*/ + boolean isTimeRangeTrackerSet = false; + + protected HFile.Writer writer; + private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null; + + /** + * Creates an HFile.Writer that also write helpful meta data. + * @param fs file system to write to + * @param path file name to create + * @param conf user configuration + * @param comparator key comparator + * @param bloomType bloom filter setting + * @param maxKeys the expected maximum number of keys to be added. Was used + * for Bloom filter size in {@link HFile} format version 1. + * @param fileContext - The HFile context + * @throws IOException problem writing to FS + */ + StoreFileWriter(FileSystem fs, Path path, final Configuration conf, CacheConfig cacheConf, + final CellComparator comparator, BloomType bloomType, long maxKeys, + InetSocketAddress[] favoredNodes, HFileContext fileContext) + throws IOException { + writer = HFile.getWriterFactory(conf, cacheConf) + .withPath(fs, path) + .withComparator(comparator) + .withFavoredNodes(favoredNodes) + .withFileContext(fileContext) + .create(); + + generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite( + conf, cacheConf, bloomType, + (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + + if (generalBloomFilterWriter != null) { + this.bloomType = bloomType; + if(this.bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(); + } + if (LOG.isTraceEnabled()) { + LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", " + + generalBloomFilterWriter.getClass().getSimpleName()); + } + } else { + // Not using Bloom filters. + this.bloomType = BloomType.NONE; + } + + // initialize delete family Bloom filter when there is NO RowCol Bloom + // filter + if (this.bloomType != BloomType.ROWCOL) { + this.deleteFamilyBloomFilterWriter = BloomFilterFactory + .createDeleteBloomAtWrite(conf, cacheConf, + (int) Math.min(maxKeys, Integer.MAX_VALUE), writer); + } else { + deleteFamilyBloomFilterWriter = null; + } + if (deleteFamilyBloomFilterWriter != null && LOG.isTraceEnabled()) { + LOG.trace("Delete Family Bloom filter type for " + path + ": " + + deleteFamilyBloomFilterWriter.getClass().getSimpleName()); + } + } + + /** + * Writes meta data. + * Call before {@link #close()} since its written as meta data to this file. + * @param maxSequenceId Maximum sequence id. + * @param majorCompaction True if this file is product of a major compaction + * @throws IOException problem writing to FS + */ + public void appendMetadata(final long maxSequenceId, final boolean majorCompaction) + throws IOException { + writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); + writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, + Bytes.toBytes(majorCompaction)); + appendTrackedTimestampsToMetadata(); + } + + /** + * Writes meta data. + * Call before {@link #close()} since its written as meta data to this file. + * @param maxSequenceId Maximum sequence id. + * @param majorCompaction True if this file is product of a major compaction + * @param mobCellsCount The number of mob cells. + * @throws IOException problem writing to FS + */ + public void appendMetadata(final long maxSequenceId, final boolean majorCompaction, + final long mobCellsCount) throws IOException { + writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); + writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); + writer.appendFileInfo(StoreFile.MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount)); + appendTrackedTimestampsToMetadata(); + } + + /** + * Add TimestampRange and earliest put timestamp to Metadata + */ + public void appendTrackedTimestampsToMetadata() throws IOException { + appendFileInfo(StoreFile.TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker)); + appendFileInfo(StoreFile.EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs)); + } + + /** + * Set TimeRangeTracker + */ + public void setTimeRangeTracker(final TimeRangeTracker trt) { + this.timeRangeTracker = trt; + isTimeRangeTrackerSet = true; + } + + /** + * Record the earlest Put timestamp. + * + * If the timeRangeTracker is not set, + * update TimeRangeTracker to include the timestamp of this key + */ + public void trackTimestamps(final Cell cell) { + if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) { + earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp()); + } + if (!isTimeRangeTrackerSet) { + timeRangeTracker.includeTimestamp(cell); + } + } + + private void appendGeneralBloomfilter(final Cell cell) throws IOException { + if (this.generalBloomFilterWriter != null) { + // only add to the bloom filter on a new, unique key + boolean newKey = true; + if (this.lastCell != null) { + switch(bloomType) { + case ROW: + newKey = ! CellUtil.matchingRows(cell, lastCell); + break; + case ROWCOL: + newKey = ! CellUtil.matchingRowColumn(cell, lastCell); + break; + case NONE: + newKey = false; + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL expected)"); + } + } + if (newKey) { + /* + * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png + * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp + * + * 2 Types of Filtering: + * 1. Row = Row + * 2. RowCol = Row + Qualifier + */ + byte[] bloomKey = null; + // Used with ROW_COL bloom + KeyValue bloomKeyKV = null; + int bloomKeyOffset, bloomKeyLen; + + switch (bloomType) { + case ROW: + bloomKey = cell.getRowArray(); + bloomKeyOffset = cell.getRowOffset(); + bloomKeyLen = cell.getRowLength(); + break; + case ROWCOL: + // merge(row, qualifier) + // TODO: could save one buffer copy in case of compound Bloom + // filters when this involves creating a KeyValue + // TODO : Handle while writes also + bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(), + cell.getRowLength(), + HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(), + cell.getQualifierOffset(), + cell.getQualifierLength()); + bloomKey = bloomKeyKV.getBuffer(); + bloomKeyOffset = bloomKeyKV.getKeyOffset(); + bloomKeyLen = bloomKeyKV.getKeyLength(); + break; + default: + throw new IOException("Invalid Bloom filter type: " + bloomType + + " (ROW or ROWCOL expected)"); + } + generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); + if (lastBloomKey != null) { + int res = 0; + // hbase:meta does not have blooms. So we need not have special interpretation + // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom + if (bloomType == BloomType.ROW) { + res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen, + lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen); + } else { + // TODO : Caching of kv components becomes important in these cases + res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); + } + if (res <= 0) { + throw new IOException("Non-increasing Bloom keys: " + + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); + } + } + lastBloomKey = bloomKey; + lastBloomKeyOffset = bloomKeyOffset; + lastBloomKeyLen = bloomKeyLen; + if (bloomType == BloomType.ROWCOL) { + lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen); + } + this.lastCell = cell; + } + } + } + + private void appendDeleteFamilyBloomFilter(final Cell cell) + throws IOException { + if (!CellUtil.isDeleteFamily(cell) && !CellUtil.isDeleteFamilyVersion(cell)) { + return; + } + + // increase the number of delete family in the store file + deleteFamilyCnt++; + if (null != this.deleteFamilyBloomFilterWriter) { + boolean newKey = true; + if (lastDeleteFamilyCell != null) { + // hbase:meta does not have blooms. So we need not have special interpretation + // of the hbase:meta cells + newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell); + } + if (newKey) { + this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(), + cell.getRowOffset(), cell.getRowLength()); + this.lastDeleteFamilyCell = cell; + } + } + } + + public void append(final Cell cell) throws IOException { + appendGeneralBloomfilter(cell); + appendDeleteFamilyBloomFilter(cell); + writer.append(cell); + trackTimestamps(cell); + } + + public Path getPath() { + return this.writer.getPath(); + } + + public boolean hasGeneralBloom() { + return this.generalBloomFilterWriter != null; + } + + /** + * For unit testing only. + * + * @return the Bloom filter used by this writer. + */ + BloomFilterWriter getGeneralBloomWriter() { + return generalBloomFilterWriter; + } + + private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException { + boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0); + if (haveBloom) { + bfw.compactBloom(); + } + return haveBloom; + } + + private boolean closeGeneralBloomFilter() throws IOException { + boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter); + + // add the general Bloom filter writer and append file info + if (hasGeneralBloom) { + writer.addGeneralBloomFilter(generalBloomFilterWriter); + writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY, + Bytes.toBytes(bloomType.toString())); + if (lastBloomKey != null) { + writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, Arrays.copyOfRange( + lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset + + lastBloomKeyLen)); + } + } + return hasGeneralBloom; + } + + private boolean closeDeleteFamilyBloomFilter() throws IOException { + boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter); + + // add the delete family Bloom filter writer + if (hasDeleteFamilyBloom) { + writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter); + } + + // append file info about the number of delete family kvs + // even if there is no delete family Bloom. + writer.appendFileInfo(StoreFile.DELETE_FAMILY_COUNT, + Bytes.toBytes(this.deleteFamilyCnt)); + + return hasDeleteFamilyBloom; + } + + public void close() throws IOException { + boolean hasGeneralBloom = this.closeGeneralBloomFilter(); + boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter(); + + writer.close(); + + // Log final Bloom filter statistics. This needs to be done after close() + // because compound Bloom filters might be finalized as part of closing. + if (LOG.isTraceEnabled()) { + LOG.trace((hasGeneralBloom ? "" : "NO ") + "General Bloom and " + + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily" + " was added to HFile " + + getPath()); + } + + } + + public void appendFileInfo(byte[] key, byte[] value) throws IOException { + writer.appendFileInfo(key, value); + } + + /** For use in testing. + */ + HFile.Writer getHFileWriter() { + return writer; + } + + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG", + justification="Will not overflow") + public static class Builder { + private final Configuration conf; + private final CacheConfig cacheConf; + private final FileSystem fs; + + private CellComparator comparator = CellComparator.COMPARATOR; + private BloomType bloomType = BloomType.NONE; + private long maxKeyCount = 0; + private Path dir; + private Path filePath; + private InetSocketAddress[] favoredNodes; + private HFileContext fileContext; + + public Builder(Configuration conf, CacheConfig cacheConf, + FileSystem fs) { + this.conf = conf; + this.cacheConf = cacheConf; + this.fs = fs; + } + + /** + * Use either this method or {@link #withFilePath}, but not both. + * @param dir Path to column family directory. The directory is created if + * does not exist. The file is given a unique name within this + * directory. + * @return this (for chained invocation) + */ + public Builder withOutputDir(Path dir) { + Preconditions.checkNotNull(dir); + this.dir = dir; + return this; + } + + /** + * Use either this method or {@link #withOutputDir}, but not both. + * @param filePath the StoreFile path to write + * @return this (for chained invocation) + */ + public Builder withFilePath(Path filePath) { + Preconditions.checkNotNull(filePath); + this.filePath = filePath; + return this; + } + + /** + * @param favoredNodes an array of favored nodes or possibly null + * @return this (for chained invocation) + */ + public Builder withFavoredNodes(InetSocketAddress[] favoredNodes) { + this.favoredNodes = favoredNodes; + return this; + } + + public Builder withComparator(CellComparator comparator) { + Preconditions.checkNotNull(comparator); + this.comparator = comparator; + return this; + } + + public Builder withBloomType(BloomType bloomType) { + Preconditions.checkNotNull(bloomType); + this.bloomType = bloomType; + return this; + } + + /** + * @param maxKeyCount estimated maximum number of keys we expect to add + * @return this (for chained invocation) + */ + public Builder withMaxKeyCount(long maxKeyCount) { + this.maxKeyCount = maxKeyCount; + return this; + } + + public Builder withFileContext(HFileContext fileContext) { + this.fileContext = fileContext; + return this; + } + + public Builder withShouldDropCacheBehind(boolean shouldDropCacheBehind/*NOT USED!!*/) { + // TODO: HAS NO EFFECT!!! FIX!! + return this; + } + /** + * Create a store file writer. Client is responsible for closing file when + * done. If metadata, add BEFORE closing using + * {@link StoreFileWriter#appendMetadata}. + */ + public StoreFileWriter build() throws IOException { + if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) { + throw new IllegalArgumentException("Either specify parent directory " + + "or file path"); + } + + if (dir == null) { + dir = filePath.getParent(); + } + + if (!fs.exists(dir)) { + fs.mkdirs(dir); + } + + if (filePath == null) { + filePath = StoreFile.getUniqueFile(fs, dir); + if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) { + bloomType = BloomType.NONE; + } + } + + if (comparator == null) { + comparator = CellComparator.COMPARATOR; + } + return new StoreFileWriter(fs, filePath, + conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext); + } + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java index 9b182a2..5ba7d33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java @@ -61,7 +61,7 @@ abstract class StoreFlusher { public abstract List flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushSeqNum, MonitoredTask status, ThroughputController throughputController) throws IOException; - protected void finalizeWriter(StoreFile.Writer writer, long cacheFlushSeqNum, + protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum, MonitoredTask status) throws IOException { // Write out the log sequence number that corresponds to this output // hfile. Also write current time in metadata as minFlushTime. http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java index 3d4e990..196c889 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java @@ -74,7 +74,7 @@ public class StoreUtils { long maxSize = -1L; StoreFile largestSf = null; for (StoreFile sf : candidates) { - StoreFile.Reader r = sf.getReader(); + StoreFileReader r = sf.getReader(); if (r == null) continue; long size = r.length(); if (size > maxSize) { http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java index 1c3f14c..2662dd1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; import org.apache.hadoop.hbase.util.Bytes; /** @@ -43,7 +42,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class); protected final CellComparator comparator; - protected List existingWriters; + protected List existingWriters; protected List boundaries; /** Whether to write stripe metadata */ @@ -58,7 +57,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { } @Override - protected Collection writers() { + protected Collection writers() { return existingWriters; } @@ -73,7 +72,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { } @Override - protected void preCloseWriter(Writer writer) throws IOException { + protected void preCloseWriter(StoreFileWriter writer) throws IOException { if (doWriteStripeMetadata) { if (LOG.isDebugEnabled()) { LOG.debug("Write stripe metadata for " + writer.getPath().toString()); @@ -126,7 +125,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { * separate from all other such pairs. */ public static class BoundaryMultiWriter extends StripeMultiFileWriter { - private StoreFile.Writer currentWriter; + private StoreFileWriter currentWriter; private byte[] currentWriterEndKey; private Cell lastCell; @@ -144,7 +143,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { byte[] majorRangeFrom, byte[] majorRangeTo) throws IOException { super(comparator); this.boundaries = targetBoundaries; - this.existingWriters = new ArrayList(this.boundaries.size() - 1); + this.existingWriters = new ArrayList(this.boundaries.size() - 1); // "major" range (range for which all files are included) boundaries, if any, // must match some target boundaries, let's find them. assert (majorRangeFrom == null) == (majorRangeTo == null); @@ -264,7 +263,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { private byte[] right; private Cell lastCell; - private StoreFile.Writer currentWriter; + private StoreFileWriter currentWriter; protected byte[] lastRowInCurrentWriter = null; private long cellsInCurrentWriter = 0; private long cellsSeen = 0; @@ -284,7 +283,7 @@ public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { this.left = left; this.right = right; int preallocate = Math.min(this.targetCount, 64); - this.existingWriters = new ArrayList(preallocate); + this.existingWriters = new ArrayList(preallocate); this.boundaries = new ArrayList(preallocate + 1); } http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java index 34e8497..cf0de1f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java @@ -31,8 +31,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; -import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; @@ -110,8 +108,8 @@ public class StripeStoreFlusher extends StoreFlusher { final TimeRangeTracker tracker, final long kvCount) { return new StripeMultiFileWriter.WriterFactory() { @Override - public Writer createWriter() throws IOException { - StoreFile.Writer writer = store.createWriterInTmp( + public StoreFileWriter createWriter() throws IOException { + StoreFileWriter writer = store.createWriterInTmp( kvCount, store.getFamily().getCompressionType(), /* isCompaction = */ false, /* includeMVCCReadpoint = */ true, http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java index ef39a6c..d6f975a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter; import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.StoreScanner; /** @@ -50,7 +50,7 @@ public abstract class AbstractMultiOutputCompactor { private void recalculateSize() { long sz = 0; for (StoreFile sf : this.filesToCompact) { - Reader r = sf.getReader(); + StoreFileReader r = sf.getReader(); sz += r == null ? 0 : r.length(); } this.totalSize = sz; http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index f32e60a..0693a2b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -46,8 +46,9 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.regionserver.compactions.Compactor.CellSink; @@ -144,7 +145,7 @@ public abstract class Compactor { } long seqNum = file.getMaxSequenceId(); fd.maxSeqId = Math.max(fd.maxSeqId, seqNum); - StoreFile.Reader r = file.getReader(); + StoreFileReader r = file.getReader(); if (r == null) { LOG.warn("Null reader for " + file.getPath()); continue; @@ -258,7 +259,8 @@ public abstract class Compactor { * @return Writer for a new StoreFile in the tmp dir. * @throws IOException if creation failed */ - protected Writer createTmpWriter(FileDetails fd, boolean shouldDropBehind) throws IOException { + protected StoreFileWriter createTmpWriter(FileDetails fd, boolean shouldDropBehind) + throws IOException { // When all MVCC readpoints are 0, don't write them. // See HBASE-8166, HBASE-12600, and HBASE-13389. return store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index 9759d2b..020b526 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; @@ -41,22 +41,22 @@ import com.google.common.collect.Lists; * {@link #compact(CompactionRequest, ThroughputController, User)} */ @InterfaceAudience.Private -public class DefaultCompactor extends Compactor { +public class DefaultCompactor extends Compactor { private static final Log LOG = LogFactory.getLog(DefaultCompactor.class); public DefaultCompactor(final Configuration conf, final Store store) { super(conf, store); } - private final CellSinkFactory writerFactory = new CellSinkFactory() { - - @Override - public Writer createWriter(InternalScanner scanner, - org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, - boolean shouldDropBehind) throws IOException { - return createTmpWriter(fd, shouldDropBehind); - } - }; + private final CellSinkFactory writerFactory = + new CellSinkFactory() { + @Override + public StoreFileWriter createWriter(InternalScanner scanner, + org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd, + boolean shouldDropBehind) throws IOException { + return createTmpWriter(fd, shouldDropBehind); + } + }; /** * Do a minor/major compaction on an explicit set of storefiles from a Store. @@ -84,7 +84,7 @@ public class DefaultCompactor extends Compactor { } @Override - protected List commitWriter(Writer writer, FileDetails fd, + protected List commitWriter(StoreFileWriter writer, FileDetails fd, CompactionRequest request) throws IOException { List newFiles = Lists.newArrayList(writer.getPath()); writer.appendMetadata(fd.maxSeqId, request.isAllFiles()); @@ -93,7 +93,7 @@ public class DefaultCompactor extends Compactor { } @Override - protected void abortWriter(Writer writer) throws IOException { + protected void abortWriter(StoreFileWriter writer) throws IOException { Path leftoverFile = writer.getPath(); try { writer.close(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index 58a2820..9e50e4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; +import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -683,17 +683,17 @@ public class SimpleRegionObserver extends BaseRegionObserver { } @Override - public Reader preStoreFileReaderOpen(ObserverContext ctx, + public StoreFileReader preStoreFileReaderOpen(ObserverContext ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, - Reference r, Reader reader) throws IOException { + Reference r, StoreFileReader reader) throws IOException { ctPreStoreFileReaderOpen.incrementAndGet(); return null; } @Override - public Reader postStoreFileReaderOpen(ObserverContext ctx, + public StoreFileReader postStoreFileReaderOpen(ObserverContext ctx, FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf, - Reference r, Reader reader) throws IOException { + Reference r, StoreFileReader reader) throws IOException { ctPostStoreFileReaderOpen.incrementAndGet(); return reader; } http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java index a9d8258..b9cbb16 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.BloomFilterFactory; @@ -369,7 +369,7 @@ public class TestCacheOnWrite { .withBlockSize(DATA_BLOCK_SIZE) .withDataBlockEncoding(NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding()) .withIncludesTags(useTags).build(); - StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter sfw = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(storeFileParentDir).withComparator(CellComparator.COMPARATOR) .withFileContext(meta) .withBloomType(BLOOM_TYPE).withMaxKeyCount(NUM_KV).build(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java index d20ba2b..e31ac52 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java @@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Before; @@ -96,7 +96,7 @@ public class TestPrefetch { HFileContext meta = new HFileContextBuilder() .withBlockSize(DATA_BLOCK_SIZE) .build(); - StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter sfw = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(storeFileParentDir) .withComparator(CellComparator.COMPARATOR) .withFileContext(meta) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java index 0dbb4e0..d46af4a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekBeforeWithInlineBlocks.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.BloomFilterFactory; @@ -115,8 +115,8 @@ public class TestSeekBeforeWithInlineBlocks { .withBlockSize(DATA_BLOCK_SIZE) .build(); - StoreFile.Writer storeFileWriter = - new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter storeFileWriter = + new StoreFileWriter.Builder(conf, cacheConf, fs) .withFilePath(hfilePath) .withFileContext(meta) .withBloomType(bloomType) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java index 3e0e2d9..3ecc578 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobTestUtil.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; @@ -47,7 +47,7 @@ public class MobTestUtil { } return sb.toString(); } - protected static void writeStoreFile(final StoreFile.Writer writer, String caseName) + protected static void writeStoreFile(final StoreFileWriter writer, String caseName) throws IOException { writeStoreFile(writer, Bytes.toBytes(caseName), Bytes.toBytes(caseName)); } @@ -60,7 +60,7 @@ public class MobTestUtil { * * @throws IOException */ - private static void writeStoreFile(final StoreFile.Writer writer, byte[] fam, + private static void writeStoreFile(final StoreFileWriter writer, byte[] fam, byte[] qualifier) throws IOException { long now = System.currentTimeMillis(); try { http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java index 7eb63d1..276fedb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestCachedMobFile.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Assert; @@ -57,7 +57,7 @@ public class TestCachedMobFile extends TestCase{ Path testDir = TEST_UTIL.getDataTestDir(); FileSystem fs = testDir.getFileSystem(conf); HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(testDir).withFileContext(meta).build(); MobTestUtil.writeStoreFile(writer, caseName); CachedMobFile cachedMobFile = CachedMobFile.create(fs, writer.getPath(), conf, cacheConf); @@ -79,12 +79,12 @@ public class TestCachedMobFile extends TestCase{ FileSystem fs = testDir.getFileSystem(conf); Path outputDir1 = new Path(testDir, FAMILY1); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - StoreFile.Writer writer1 = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer1 = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(outputDir1).withFileContext(meta).build(); MobTestUtil.writeStoreFile(writer1, caseName); CachedMobFile cachedMobFile1 = CachedMobFile.create(fs, writer1.getPath(), conf, cacheConf); Path outputDir2 = new Path(testDir, FAMILY2); - StoreFile.Writer writer2 = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer2 = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(outputDir2) .withFileContext(meta) .build(); @@ -102,7 +102,7 @@ public class TestCachedMobFile extends TestCase{ Path testDir = TEST_UTIL.getDataTestDir(); FileSystem fs = testDir.getFileSystem(conf); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(testDir).withFileContext(meta).build(); String caseName = getName(); MobTestUtil.writeStoreFile(writer, caseName); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java index 7523ca5..84a2ba7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; @@ -52,7 +53,7 @@ public class TestMobFile extends TestCase { Path testDir = TEST_UTIL.getDataTestDir(); FileSystem fs = testDir.getFileSystem(conf); HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(testDir) .withFileContext(meta) .build(); @@ -105,7 +106,7 @@ public class TestMobFile extends TestCase { Path testDir = TEST_UTIL.getDataTestDir(); FileSystem fs = testDir.getFileSystem(conf); HFileContext meta = new HFileContextBuilder().withBlockSize(8*1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(testDir) .withFileContext(meta) .build(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java index 95fa1b9..9abe040 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; import org.junit.Test; @@ -133,7 +133,7 @@ public class TestMobFileCache extends TestCase { KeyValue[] keys = new KeyValue[] { key1, key2, key3 }; int maxKeyCount = keys.length; HRegionInfo regionInfo = new HRegionInfo(tn); - StoreFile.Writer mobWriter = mobStore.createWriterInTmp(currentDate, + StoreFileWriter mobWriter = mobStore.createWriterInTmp(currentDate, maxKeyCount, hcd.getCompactionCompression(), regionInfo.getStartKey()); Path mobFilePath = mobWriter.getPath(); String fileName = mobFilePath.getName(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java index 6774ac3..0e0bdbe 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java @@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobFileName; import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest; -import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; import org.apache.hadoop.hbase.util.Bytes; @@ -306,7 +304,7 @@ public class TestPartitionedMobCompactor { mobFileName = MobFileName.create(Bytes.toBytes(startKey + i), MobUtils.formatDate( new Date()), mobSuffix); } - StoreFile.Writer mobFileWriter = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) .withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build(); writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier), type, (i+1)*1000); @@ -322,7 +320,7 @@ public class TestPartitionedMobCompactor { * @param type the key type * @param size the size of value */ - private static void writeStoreFile(final StoreFile.Writer writer, byte[]row, byte[] family, + private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family, byte[] qualifier, Type type, int size) throws IOException { long now = System.currentTimeMillis(); try { http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java index c3f7b82..a311501 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java @@ -185,7 +185,7 @@ public class CreateRandomStoreFile { HFileContext meta = new HFileContextBuilder().withCompression(compr) .withBlockSize(blockSize).build(); - StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf, + StoreFileWriter sfw = new StoreFileWriter.Builder(conf, new CacheConfig(conf), fs) .withOutputDir(outputDir) .withBloomType(bloomType) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java index 6544c72..9499366 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java @@ -595,7 +595,7 @@ public class DataBlockEncodingTool { StoreFile hsf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE); - StoreFile.Reader reader = hsf.createReader(); + StoreFileReader reader = hsf.createReader(); reader.loadFileInfo(); KeyValueScanner scanner = reader.getStoreFileScanner(true, true); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java index 6c801bf..45444cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java @@ -62,7 +62,7 @@ public class EncodedSeekPerformanceTest { StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), path, configuration, cacheConf, BloomType.NONE); - StoreFile.Reader reader = storeFile.createReader(); + StoreFileReader reader = storeFile.createReader(); StoreFileScanner scanner = reader.getStoreFileScanner(true, false); Cell current; @@ -94,7 +94,7 @@ public class EncodedSeekPerformanceTest { long totalSize = 0; - StoreFile.Reader reader = storeFile.createReader(); + StoreFileReader reader = storeFile.createReader(); StoreFileScanner scanner = reader.getStoreFileScanner(true, false); long startReadingTime = System.nanoTime(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java index 32dc227..506f554 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -131,11 +130,11 @@ public class MockStoreFile extends StoreFile { } @Override - public StoreFile.Reader getReader() { + public StoreFileReader getReader() { final long len = this.length; final TimeRangeTracker timeRange = this.timeRangeTracker; final long entries = this.entryCount; - return new StoreFile.Reader() { + return new StoreFileReader() { @Override public long length() { return len; http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 2357bef..ce47fa3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -207,7 +207,7 @@ public class TestCacheOnWriteInSchema { @Test public void testCacheOnWriteInSchema() throws IOException { // Write some random data into the store - StoreFile.Writer writer = store.createWriterInTmp(Integer.MAX_VALUE, + StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE, HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false); writeStoreFile(writer); writer.close(); @@ -267,7 +267,7 @@ public class TestCacheOnWriteInSchema { } } - private void writeStoreFile(StoreFile.Writer writer) throws IOException { + private void writeStoreFile(StoreFileWriter writer) throws IOException { final int rowLen = 32; for (int i = 0; i < NUM_KV; ++i) { byte[] k = RandomKeyValueUtil.randomOrderedKey(rand, i); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 72dcbe7..d7cb4a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -562,7 +562,7 @@ public class TestCompaction { private static StoreFile createFile() throws Exception { StoreFile sf = mock(StoreFile.class); when(sf.getPath()).thenReturn(new Path("file")); - StoreFile.Reader r = mock(StoreFile.Reader.class); + StoreFileReader r = mock(StoreFileReader.class); when(r.length()).thenReturn(10L); when(sf.getReader()).thenReturn(r); return sf; http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 44e249d..d0c0089 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -201,7 +201,7 @@ public class TestCompoundBloomFilter { private void readStoreFile(int t, BloomType bt, List kvs, Path sfPath) throws IOException { StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt); - StoreFile.Reader r = sf.createReader(); + StoreFileReader r = sf.createReader(); final boolean pread = true; // does not really matter StoreFileScanner scanner = r.getStoreFileScanner(true, pread); @@ -301,7 +301,7 @@ public class TestCompoundBloomFilter { conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true); cacheConf = new CacheConfig(conf); HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build(); - StoreFile.Writer w = new StoreFile.WriterBuilder(conf, cacheConf, fs) + StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs) .withOutputDir(TEST_UTIL.getDataTestDir()) .withBloomType(bt) .withFileContext(meta) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index 0a573b2..94daf83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -81,7 +81,7 @@ public class TestFSErrorsExposed { FileSystem fs = new HFileSystem(faultyfs); CacheConfig cacheConf = new CacheConfig(util.getConfiguration()); HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder( + StoreFileWriter writer = new StoreFileWriter.Builder( util.getConfiguration(), cacheConf, hfs) .withOutputDir(hfilePath) .withFileContext(meta) @@ -92,7 +92,7 @@ public class TestFSErrorsExposed { StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, BloomType.NONE); - StoreFile.Reader reader = sf.createReader(); + StoreFileReader reader = sf.createReader(); HFileScanner scanner = reader.getScanner(false, true); FaultyInputStream inStream = faultyfs.inStreams.get(0).get(); @@ -131,7 +131,7 @@ public class TestFSErrorsExposed { HFileSystem fs = new HFileSystem(faultyfs); CacheConfig cacheConf = new CacheConfig(util.getConfiguration()); HFileContext meta = new HFileContextBuilder().withBlockSize(2 * 1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder( + StoreFileWriter writer = new StoreFileWriter.Builder( util.getConfiguration(), cacheConf, hfs) .withOutputDir(hfilePath) .withFileContext(meta) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java index b237490..dfc97e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java @@ -178,7 +178,7 @@ public class TestHMobStore { KeyValue key3 = new KeyValue(row2, family, qf3, 1, value2); KeyValue[] keys = new KeyValue[] { key1, key2, key3 }; int maxKeyCount = keys.length; - StoreFile.Writer mobWriter = store.createWriterInTmp(currentDate, maxKeyCount, + StoreFileWriter mobWriter = store.createWriterInTmp(currentDate, maxKeyCount, hcd.getCompactionCompression(), region.getRegionInfo().getStartKey()); mobFilePath = mobWriter.getPath(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index ed7623c..713c8e5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -4214,7 +4214,7 @@ public class TestHRegion { HStore store = (HStore) region.getStore(fam1); Collection storeFiles = store.getStorefiles(); for (StoreFile storefile : storeFiles) { - StoreFile.Reader reader = storefile.getReader(); + StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); assertEquals(num_unique_rows * duplicate_multiplier, reader.getEntries()); @@ -4226,7 +4226,7 @@ public class TestHRegion { // after compaction storeFiles = store.getStorefiles(); for (StoreFile storefile : storeFiles) { - StoreFile.Reader reader = storefile.getReader(); + StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); assertEquals(num_unique_rows * duplicate_multiplier * num_storefiles, reader.getEntries()); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index c6cc902..0ec859c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -97,7 +97,7 @@ public class TestReversibleScanners { hcBuilder.withBlockSize(2 * 1024); hcBuilder.withDataBlockEncoding(encoding); HFileContext hFileContext = hcBuilder.build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder( + StoreFileWriter writer = new StoreFileWriter.Builder( TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir(hfilePath) .withFileContext(hFileContext).build(); writeStoreFile(writer); @@ -145,15 +145,15 @@ public class TestReversibleScanners { HFileContextBuilder hcBuilder = new HFileContextBuilder(); hcBuilder.withBlockSize(2 * 1024); HFileContext hFileContext = hcBuilder.build(); - StoreFile.Writer writer1 = new StoreFile.WriterBuilder( + StoreFileWriter writer1 = new StoreFileWriter.Builder( TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir( hfilePath).withFileContext(hFileContext).build(); - StoreFile.Writer writer2 = new StoreFile.WriterBuilder( + StoreFileWriter writer2 = new StoreFileWriter.Builder( TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir( hfilePath).withFileContext(hFileContext).build(); MemStore memstore = new DefaultMemStore(); - writeMemstoreAndStoreFiles(memstore, new StoreFile.Writer[] { writer1, + writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, writer2 }); StoreFile sf1 = new StoreFile(fs, writer1.getPath(), @@ -235,15 +235,15 @@ public class TestReversibleScanners { HFileContextBuilder hcBuilder = new HFileContextBuilder(); hcBuilder.withBlockSize(2 * 1024); HFileContext hFileContext = hcBuilder.build(); - StoreFile.Writer writer1 = new StoreFile.WriterBuilder( + StoreFileWriter writer1 = new StoreFileWriter.Builder( TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir( hfilePath).withFileContext(hFileContext).build(); - StoreFile.Writer writer2 = new StoreFile.WriterBuilder( + StoreFileWriter writer2 = new StoreFileWriter.Builder( TEST_UTIL.getConfiguration(), cacheConf, fs).withOutputDir( hfilePath).withFileContext(hFileContext).build(); MemStore memstore = new DefaultMemStore(); - writeMemstoreAndStoreFiles(memstore, new StoreFile.Writer[] { writer1, + writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, writer2 }); StoreFile sf1 = new StoreFile(fs, writer1.getPath(), @@ -633,7 +633,7 @@ public class TestReversibleScanners { } private static void writeMemstoreAndStoreFiles(MemStore memstore, - final StoreFile.Writer[] writers) throws IOException { + final StoreFileWriter[] writers) throws IOException { try { for (int i = 0; i < ROWSIZE; i++) { for (int j = 0; j < QUALSIZE; j++) { @@ -651,7 +651,7 @@ public class TestReversibleScanners { } } - private static void writeStoreFile(final StoreFile.Writer writer) + private static void writeStoreFile(final StoreFileWriter writer) throws IOException { try { for (int i = 0; i < ROWSIZE; i++) { http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java index eada3fb..027193f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScanWithBloomError.java @@ -147,7 +147,7 @@ public class TestScanWithBloomError { } }); - StoreFile.Reader lastStoreFileReader = null; + StoreFileReader lastStoreFileReader = null; for (StoreFileScanner sfScanner : scanners) lastStoreFileReader = sfScanner.getReader(); http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index 0a67ff8..59370bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -267,7 +267,7 @@ public class TestStore { init(name.getMethodName(), conf, hcd); // Test createWriterInTmp() - StoreFile.Writer writer = store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false); + StoreFileWriter writer = store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false); Path path = writer.getPath(); writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1))); writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2))); @@ -434,7 +434,7 @@ public class TestStore { Configuration c = HBaseConfiguration.create(); FileSystem fs = FileSystem.get(c); HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build(); - StoreFile.Writer w = new StoreFile.WriterBuilder(c, new CacheConfig(c), + StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs) .withOutputDir(storedir) .withFileContext(meta) @@ -1009,7 +1009,7 @@ public class TestStore { Configuration c = TEST_UTIL.getConfiguration(); FileSystem fs = FileSystem.get(c); HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build(); - StoreFile.Writer w = new StoreFile.WriterBuilder(c, new CacheConfig(c), + StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs) .withOutputDir(storedir) .withFileContext(fileContext) http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 829aa73..d8acd44 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -109,7 +109,7 @@ public class TestStoreFile extends HBaseTestCase { conf, fs, new Path(testDir, hri.getTable().getNameAsString()), hri); HFileContext meta = new HFileContextBuilder().withBlockSize(2*1024).build(); - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(regionFs.createTempName()) .withFileContext(meta) .build(); @@ -121,7 +121,7 @@ public class TestStoreFile extends HBaseTestCase { checkHalfHFile(regionFs, sf); } - private void writeStoreFile(final StoreFile.Writer writer) throws IOException { + private void writeStoreFile(final StoreFileWriter writer) throws IOException { writeStoreFile(writer, Bytes.toBytes(getName()), Bytes.toBytes(getName())); } @@ -134,7 +134,7 @@ public class TestStoreFile extends HBaseTestCase { * @param writer * @throws IOException */ - public static void writeStoreFile(final StoreFile.Writer writer, byte[] fam, byte[] qualifier) + public static void writeStoreFile(final StoreFileWriter writer, byte[] fam, byte[] qualifier) throws IOException { long now = System.currentTimeMillis(); try { @@ -162,7 +162,7 @@ public class TestStoreFile extends HBaseTestCase { HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(regionFs.createTempName()) .withFileContext(meta) .build(); @@ -171,7 +171,7 @@ public class TestStoreFile extends HBaseTestCase { Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath()); StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE); - StoreFile.Reader reader = hsf.createReader(); + StoreFileReader reader = hsf.createReader(); // Split on a row, not in middle of row. Midkey returned by reader // may be in middle of row. Create new one with empty column and // timestamp. @@ -204,7 +204,7 @@ public class TestStoreFile extends HBaseTestCase { @Test public void testEmptyStoreFileRestrictKeyRanges() throws Exception { - StoreFile.Reader reader = mock(StoreFile.Reader.class); + StoreFileReader reader = mock(StoreFileReader.class); Store store = mock(Store.class); HColumnDescriptor hcd = mock(HColumnDescriptor.class); byte[] cf = Bytes.toBytes("ty"); @@ -228,7 +228,7 @@ public class TestStoreFile extends HBaseTestCase { HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(regionFs.createTempName()) .withFileContext(meta) .build(); @@ -273,7 +273,7 @@ public class TestStoreFile extends HBaseTestCase { HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. //// - StoreFile.Writer writer = new StoreFile.WriterBuilder(testConf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(testConf, cacheConf, this.fs) .withFilePath(regionFs.createTempName()) .withFileContext(meta) .build(); @@ -351,9 +351,9 @@ public class TestStoreFile extends HBaseTestCase { midRow, null); Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false); // Make readers on top and bottom. - StoreFile.Reader top = new StoreFile( + StoreFileReader top = new StoreFile( this.fs, topPath, conf, cacheConf, BloomType.NONE).createReader(); - StoreFile.Reader bottom = new StoreFile( + StoreFileReader bottom = new StoreFile( this.fs, bottomPath, conf, cacheConf, BloomType.NONE).createReader(); ByteBuffer previous = null; LOG.info("Midkey: " + midKV.toString()); @@ -485,7 +485,7 @@ public class TestStoreFile extends HBaseTestCase { private static final String localFormatter = "%010d"; - private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs) throws Exception { + private void bloomWriteRead(StoreFileWriter writer, FileSystem fs) throws Exception { float err = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0); Path f = writer.getPath(); long now = System.currentTimeMillis(); @@ -497,7 +497,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, conf); + StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf); reader.loadFileInfo(); reader.loadBloomfilter(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); @@ -545,7 +545,7 @@ public class TestStoreFile extends HBaseTestCase { .withChecksumType(CKTYPE) .withBytesPerCheckSum(CKBYTES).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(f) .withBloomType(BloomType.ROW) .withMaxKeyCount(2000) @@ -569,7 +569,7 @@ public class TestStoreFile extends HBaseTestCase { .withChecksumType(CKTYPE) .withBytesPerCheckSum(CKBYTES).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(f) .withMaxKeyCount(2000) .withFileContext(meta) @@ -585,7 +585,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, conf); + StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf); reader.loadFileInfo(); reader.loadBloomfilter(); @@ -623,7 +623,7 @@ public class TestStoreFile extends HBaseTestCase { Path f = new Path(ROOT_DIR, getName()); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(f) .withFileContext(meta) .build(); @@ -631,7 +631,7 @@ public class TestStoreFile extends HBaseTestCase { writeStoreFile(writer); writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, conf); + StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf); // Now do reseek with empty KV to position to the beginning of the file @@ -669,7 +669,7 @@ public class TestStoreFile extends HBaseTestCase { .withChecksumType(CKTYPE) .withBytesPerCheckSum(CKBYTES).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(f) .withBloomType(bt[x]) .withMaxKeyCount(expKeys[x]) @@ -691,7 +691,7 @@ public class TestStoreFile extends HBaseTestCase { } writer.close(); - StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf, conf); + StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, conf); reader.loadFileInfo(); reader.loadBloomfilter(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); @@ -768,7 +768,7 @@ public class TestStoreFile extends HBaseTestCase { long seqId, String path) { StoreFile mock = Mockito.mock(StoreFile.class); - StoreFile.Reader reader = Mockito.mock(StoreFile.Reader.class); + StoreFileReader reader = Mockito.mock(StoreFileReader.class); Mockito.doReturn(size).when(reader).length(); @@ -825,7 +825,7 @@ public class TestStoreFile extends HBaseTestCase { Path dir = new Path(storedir, "1234567890"); HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withOutputDir(dir) .withFileContext(meta) .build(); @@ -845,7 +845,7 @@ public class TestStoreFile extends HBaseTestCase { HColumnDescriptor hcd = mock(HColumnDescriptor.class); when(hcd.getName()).thenReturn(family); when(store.getFamily()).thenReturn(hcd); - StoreFile.Reader reader = hsf.createReader(); + StoreFileReader reader = hsf.createReader(); StoreFileScanner scanner = reader.getStoreFileScanner(false, false); TreeSet columns = new TreeSet(Bytes.BYTES_COMPARATOR); columns.add(qualifier); @@ -895,13 +895,13 @@ public class TestStoreFile extends HBaseTestCase { conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false); CacheConfig cacheConf = new CacheConfig(conf); Path pathCowOff = new Path(baseDir, "123456789"); - StoreFile.Writer writer = writeStoreFile(conf, cacheConf, pathCowOff, 3); + StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3); StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, BloomType.NONE); LOG.debug(hsf.getPath().toString()); // Read this file, we should see 3 misses - StoreFile.Reader reader = hsf.createReader(); + StoreFileReader reader = hsf.createReader(); reader.loadFileInfo(); StoreFileScanner scanner = reader.getStoreFileScanner(true, true); scanner.seek(KeyValue.LOWESTKEY); @@ -936,13 +936,13 @@ public class TestStoreFile extends HBaseTestCase { // Let's read back the two files to ensure the blocks exactly match hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE); - StoreFile.Reader readerOne = hsf.createReader(); + StoreFileReader readerOne = hsf.createReader(); readerOne.loadFileInfo(); StoreFileScanner scannerOne = readerOne.getStoreFileScanner(true, true); scannerOne.seek(KeyValue.LOWESTKEY); hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE); - StoreFile.Reader readerTwo = hsf.createReader(); + StoreFileReader readerTwo = hsf.createReader(); readerTwo.loadFileInfo(); StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true); scannerTwo.seek(KeyValue.LOWESTKEY); @@ -1010,7 +1010,7 @@ public class TestStoreFile extends HBaseTestCase { return new Path(new Path(regionDir, family), path.getName()); } - private StoreFile.Writer writeStoreFile(Configuration conf, + private StoreFileWriter writeStoreFile(Configuration conf, CacheConfig cacheConf, Path path, int numBlocks) throws IOException { // Let's put ~5 small KVs in each block, so let's make 5*numBlocks KVs @@ -1030,7 +1030,7 @@ public class TestStoreFile extends HBaseTestCase { .withBytesPerCheckSum(CKBYTES) .build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(path) .withMaxKeyCount(2000) .withFileContext(meta) @@ -1067,7 +1067,7 @@ public class TestStoreFile extends HBaseTestCase { .withDataBlockEncoding(dataBlockEncoderAlgo) .build(); // Make a store file and write data to it. - StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, cacheConf, this.fs) + StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs) .withFilePath(path) .withMaxKeyCount(2000) .withFileContext(meta) @@ -1076,7 +1076,7 @@ public class TestStoreFile extends HBaseTestCase { StoreFile storeFile = new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE); - StoreFile.Reader reader = storeFile.createReader(); + StoreFileReader reader = storeFile.createReader(); Map fileInfo = reader.loadFileInfo(); byte[] value = fileInfo.get(HFileDataBlockEncoder.DATA_BLOCK_ENCODING);