HBASE-21922 BloomContext#sanityCheck may failed when use ROWPREFIX_DELIMITED bloom filter
This commit is contained in:
parent
07d84eb809
commit
0fc5173eef
|
@ -38,9 +38,5 @@ public enum BloomType {
|
|||
/**
|
||||
* Bloom enabled with Table row prefix as Key, specify the length of the prefix
|
||||
*/
|
||||
ROWPREFIX_FIXED_LENGTH,
|
||||
/**
|
||||
* Bloom enabled with Table row prefix as Key, specify the delimiter of the prefix
|
||||
*/
|
||||
ROWPREFIX_DELIMITED
|
||||
ROWPREFIX_FIXED_LENGTH
|
||||
}
|
||||
|
|
|
@ -56,8 +56,6 @@ public class ChangeBloomFilterAction extends Action {
|
|||
columnBuilder.setBloomFilterType(bloomType);
|
||||
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
columnBuilder.setConfiguration(BloomFilterUtil.PREFIX_LENGTH_KEY, "10");
|
||||
} else if (bloomType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
columnBuilder.setConfiguration(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -406,8 +406,6 @@ public class HFileOutputFormat2
|
|||
String bloomParam = bloomParamMap.get(tableAndFamily);
|
||||
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, bloomParam);
|
||||
} else if (bloomType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, bloomParam);
|
||||
}
|
||||
Integer blockSize = blockSizeMap.get(tableAndFamily);
|
||||
blockSize = blockSize == null ? HConstants.DEFAULT_BLOCKSIZE : blockSize;
|
||||
|
@ -951,8 +949,6 @@ public class HFileOutputFormat2
|
|||
String bloomParam = "";
|
||||
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
bloomParam = familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_LENGTH_KEY);
|
||||
} else if (bloomType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
bloomParam = familyDescriptor.getConfigurationValue(BloomFilterUtil.DELIMITER_KEY);
|
||||
}
|
||||
return bloomParam;
|
||||
};
|
||||
|
|
|
@ -561,14 +561,6 @@ public class LoadTestTool extends AbstractHBaseTool {
|
|||
}
|
||||
}
|
||||
|
||||
if (bloomType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
if (!cmd.hasOption(OPT_BLOOM_PARAM)) {
|
||||
LOG.error("the parameter of bloom filter {} is not specified", bloomType.name());
|
||||
} else {
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, cmd.getOptionValue(OPT_BLOOM_PARAM));
|
||||
}
|
||||
}
|
||||
|
||||
inMemoryCF = cmd.hasOption(OPT_INMEMORY);
|
||||
if (cmd.hasOption(OPT_ENCRYPTION)) {
|
||||
cipher = Encryption.getCipher(conf, cmd.getOptionValue(OPT_ENCRYPTION));
|
||||
|
|
|
@ -76,7 +76,6 @@ public class StoreFileReader {
|
|||
private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
|
||||
private boolean skipResetSeqId = true;
|
||||
private int prefixLength = -1;
|
||||
private byte[] delimiter = null;
|
||||
|
||||
// Counter that is incremented every time a scanner is created on the
|
||||
// store file. It is decremented when the scan on the store file is
|
||||
|
@ -123,7 +122,6 @@ public class StoreFileReader {
|
|||
this.lastBloomKeyOnlyKV = reader.lastBloomKeyOnlyKV;
|
||||
this.skipResetSeqId = reader.skipResetSeqId;
|
||||
this.prefixLength = reader.prefixLength;
|
||||
this.delimiter = reader.delimiter;
|
||||
}
|
||||
|
||||
public boolean isPrimaryReplicaReader() {
|
||||
|
@ -295,8 +293,6 @@ public class StoreFileReader {
|
|||
return true;
|
||||
case ROWPREFIX_FIXED_LENGTH:
|
||||
return passesGeneralRowPrefixBloomFilter(scan);
|
||||
case ROWPREFIX_DELIMITED:
|
||||
return passesGeneralDelimitedRowPrefixBloomFilter(scan);
|
||||
default:
|
||||
return true;
|
||||
}
|
||||
|
@ -408,45 +404,6 @@ public class StoreFileReader {
|
|||
return checkGeneralBloomFilter(rowPrefix, null, bloomFilter);
|
||||
}
|
||||
|
||||
/**
|
||||
* A method for checking Bloom filters. Called directly from
|
||||
* StoreFileScanner in case of a multi-column query.
|
||||
*
|
||||
* @return True if passes
|
||||
*/
|
||||
private boolean passesGeneralDelimitedRowPrefixBloomFilter(Scan scan) {
|
||||
BloomFilter bloomFilter = this.generalBloomFilter;
|
||||
if (bloomFilter == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
byte[] row = scan.getStartRow();
|
||||
byte[] rowPrefix;
|
||||
if (scan.isGetScan()) {
|
||||
int rowPrefixLength = Bytes.indexOf(row, delimiter);
|
||||
if (rowPrefixLength <= 0) {
|
||||
rowPrefix = row;
|
||||
} else {
|
||||
rowPrefix = Bytes.copy(row, 0, rowPrefixLength);
|
||||
}
|
||||
} else {
|
||||
// For non-get scans
|
||||
// If startRow does not contain delimiter, return true directly.
|
||||
int startRowPrefixLength = Bytes.indexOf(row, delimiter);
|
||||
if (startRowPrefixLength <= 0) {
|
||||
return true;
|
||||
}
|
||||
// If stopRow does not have the same prefix as startRow, return true directly.
|
||||
int commonLength = Bytes.findCommonPrefix(scan.getStartRow(), scan.getStopRow(),
|
||||
startRowPrefixLength, scan.getStopRow().length, 0, 0);
|
||||
if (commonLength < startRowPrefixLength) {
|
||||
return true;
|
||||
}
|
||||
rowPrefix = Bytes.copy(row, 0, startRowPrefixLength);
|
||||
}
|
||||
return checkGeneralBloomFilter(rowPrefix, null, bloomFilter);
|
||||
}
|
||||
|
||||
private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
|
||||
// Empty file
|
||||
if (reader.getTrailer().getEntryCount() == 0) {
|
||||
|
@ -557,8 +514,6 @@ public class StoreFileReader {
|
|||
byte[] p = fi.get(BLOOM_FILTER_PARAM_KEY);
|
||||
if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
prefixLength = Bytes.toInt(p);
|
||||
} else if (bloomFilterType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
delimiter = p;
|
||||
}
|
||||
|
||||
lastBloomKey = fi.get(LAST_BLOOM_KEY);
|
||||
|
@ -762,12 +717,7 @@ public class StoreFileReader {
|
|||
void storeFileReaderClosed(StoreFileReader reader);
|
||||
}
|
||||
|
||||
|
||||
public int getPrefixLength() {
|
||||
return prefixLength;
|
||||
}
|
||||
|
||||
public byte[] getDelimiter() {
|
||||
return delimiter;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.RowBloomContext;
|
||||
import org.apache.hadoop.hbase.util.RowColBloomContext;
|
||||
import org.apache.hadoop.hbase.util.RowPrefixDelimiterBloomContext;
|
||||
import org.apache.hadoop.hbase.util.RowPrefixFixedLengthBloomContext;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -134,13 +133,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
|||
bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter, comparator,
|
||||
Bytes.toInt(bloomParam));
|
||||
break;
|
||||
case ROWPREFIX_DELIMITED:
|
||||
bloomContext = new RowPrefixDelimiterBloomContext(generalBloomFilterWriter, comparator,
|
||||
bloomParam);
|
||||
break;
|
||||
default:
|
||||
throw new IOException("Invalid Bloom filter type: "
|
||||
+ bloomType + " (ROW or ROWCOL or ROWPREFIX or ROWPREFIX_DELIMITED expected)");
|
||||
throw new IOException(
|
||||
"Invalid Bloom filter type: " + bloomType + " (ROW or ROWCOL or ROWPREFIX expected)");
|
||||
}
|
||||
} else {
|
||||
// Not using Bloom filters.
|
||||
|
@ -222,11 +217,10 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
|||
* http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
|
||||
* Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + Timestamp
|
||||
*
|
||||
* 4 Types of Filtering:
|
||||
* 3 Types of Filtering:
|
||||
* 1. Row = Row
|
||||
* 2. RowCol = Row + Qualifier
|
||||
* 3. RowPrefixFixedLength = Fixed Length Row Prefix
|
||||
* 4. RowPrefixDelimiter = Delimited Row Prefix
|
||||
*/
|
||||
bloomContext.writeBloom(cell);
|
||||
}
|
||||
|
|
|
@ -17,6 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.BloomType.ROWPREFIX_FIXED_LENGTH;
|
||||
|
||||
import java.text.NumberFormat;
|
||||
import java.util.Random;
|
||||
|
||||
|
@ -50,7 +52,6 @@ public final class BloomFilterUtil {
|
|||
private static Random randomGeneratorForTest;
|
||||
|
||||
public static final String PREFIX_LENGTH_KEY = "RowPrefixBloomFilter.prefix_length";
|
||||
public static final String DELIMITER_KEY = "RowPrefixDelimitedBloomFilter.delimiter";
|
||||
|
||||
/** Bit-value lookup array to prevent doing the same work over and over */
|
||||
public static final byte [] bitvals = {
|
||||
|
@ -291,42 +292,29 @@ public final class BloomFilterUtil {
|
|||
}
|
||||
|
||||
public static byte[] getBloomFilterParam(BloomType bloomFilterType, Configuration conf)
|
||||
throws IllegalArgumentException{
|
||||
throws IllegalArgumentException {
|
||||
byte[] bloomParam = null;
|
||||
String message = "Bloom filter type is " + bloomFilterType + ", ";
|
||||
switch (bloomFilterType) {
|
||||
case ROWPREFIX_FIXED_LENGTH:
|
||||
String prefixLengthString = conf.get(PREFIX_LENGTH_KEY);
|
||||
if (prefixLengthString == null) {
|
||||
message += PREFIX_LENGTH_KEY + " not specified.";
|
||||
if (bloomFilterType.equals(ROWPREFIX_FIXED_LENGTH)) {
|
||||
String prefixLengthString = conf.get(PREFIX_LENGTH_KEY);
|
||||
if (prefixLengthString == null) {
|
||||
message += PREFIX_LENGTH_KEY + " not specified.";
|
||||
throw new IllegalArgumentException(message);
|
||||
}
|
||||
int prefixLength;
|
||||
try {
|
||||
prefixLength = Integer.parseInt(prefixLengthString);
|
||||
if (prefixLength <= 0 || prefixLength > HConstants.MAX_ROW_LENGTH) {
|
||||
message +=
|
||||
"the value of " + PREFIX_LENGTH_KEY + " must >=0 and < " + HConstants.MAX_ROW_LENGTH;
|
||||
throw new IllegalArgumentException(message);
|
||||
}
|
||||
int prefixLength;
|
||||
try {
|
||||
prefixLength = Integer.parseInt(prefixLengthString);
|
||||
if (prefixLength <= 0 || prefixLength > HConstants.MAX_ROW_LENGTH) {
|
||||
message += "the value of " + PREFIX_LENGTH_KEY
|
||||
+ " must >=0 and < " + HConstants.MAX_ROW_LENGTH;
|
||||
throw new IllegalArgumentException(message);
|
||||
}
|
||||
} catch (NumberFormatException nfe) {
|
||||
message = "Number format exception when parsing " + PREFIX_LENGTH_KEY + " for BloomType "
|
||||
+ bloomFilterType.toString() + ":"
|
||||
+ prefixLengthString;
|
||||
throw new IllegalArgumentException(message, nfe);
|
||||
}
|
||||
bloomParam = Bytes.toBytes(prefixLength);
|
||||
break;
|
||||
case ROWPREFIX_DELIMITED:
|
||||
String delimiterString = conf.get(DELIMITER_KEY);
|
||||
if (delimiterString == null || delimiterString.length() == 0) {
|
||||
message += DELIMITER_KEY + " not specified.";
|
||||
throw new IllegalArgumentException(message);
|
||||
}
|
||||
bloomParam = Bytes.toBytes(delimiterString);
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
} catch (NumberFormatException nfe) {
|
||||
message = "Number format exception when parsing " + PREFIX_LENGTH_KEY + " for BloomType " +
|
||||
bloomFilterType.toString() + ":" + prefixLengthString;
|
||||
throw new IllegalArgumentException(message, nfe);
|
||||
}
|
||||
bloomParam = Bytes.toBytes(prefixLength);
|
||||
}
|
||||
return bloomParam;
|
||||
}
|
||||
|
|
|
@ -1,62 +0,0 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Handles ROWPREFIX_DELIMITED bloom related context.
|
||||
* It works with both ByteBufferedCell and byte[] backed cells
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RowPrefixDelimiterBloomContext extends RowBloomContext {
|
||||
private final byte[] delimiter;
|
||||
|
||||
public RowPrefixDelimiterBloomContext(BloomFilterWriter bloomFilterWriter,
|
||||
CellComparator comparator, byte[] delimiter) {
|
||||
super(bloomFilterWriter, comparator);
|
||||
this.delimiter = delimiter;
|
||||
}
|
||||
|
||||
public void writeBloom(Cell cell) throws IOException {
|
||||
super.writeBloom(getDelimitedRowPrefixCell(cell));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param cell the new cell
|
||||
* @return the new cell created by delimited row prefix
|
||||
*/
|
||||
private Cell getDelimitedRowPrefixCell(Cell cell) {
|
||||
byte[] row = CellUtil.copyRow(cell);
|
||||
int prefixLength = Bytes.indexOf(row, delimiter);
|
||||
if (prefixLength <= 0) {
|
||||
return cell;
|
||||
}
|
||||
return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
|
||||
.setRow(row, 0, Math.min(prefixLength, row.length))
|
||||
.setType(Cell.Type.Put)
|
||||
.build();
|
||||
}
|
||||
}
|
|
@ -82,7 +82,6 @@ public class TestSeekBeforeWithInlineBlocks {
|
|||
public void testMultiIndexLevelRandomHFileWithBlooms() throws IOException {
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
|
||||
TEST_UTIL.getConfiguration().set(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
|
||||
// Try out different HFile versions to ensure reverse scan works on each version
|
||||
for (int hfileVersion = HFile.MIN_FORMAT_VERSION_WITH_TAGS;
|
||||
|
@ -105,7 +104,6 @@ public class TestSeekBeforeWithInlineBlocks {
|
|||
conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
|
||||
conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
|
||||
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
|
||||
Cell[] cells = new Cell[NUM_KV];
|
||||
|
||||
|
|
|
@ -183,15 +183,6 @@ public class CreateRandomStoreFile {
|
|||
}
|
||||
}
|
||||
|
||||
if (bloomType == BloomType.ROWPREFIX_DELIMITED) {
|
||||
if (!cmdLine.hasOption(BLOOM_FILTER_PARAM_OPTION)) {
|
||||
LOG.error("the parameter of bloom filter is not specified");
|
||||
return false;
|
||||
} else {
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, cmdLine.getOptionValue(BLOOM_FILTER_PARAM_OPTION));
|
||||
}
|
||||
}
|
||||
|
||||
int blockSize = HConstants.DEFAULT_BLOCKSIZE;
|
||||
if (cmdLine.hasOption(BLOCK_SIZE_OPTION))
|
||||
blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION));
|
||||
|
|
|
@ -138,7 +138,6 @@ public abstract class TestMultiColumnScanner {
|
|||
@Test
|
||||
public void testMultiColumnScanner() throws IOException {
|
||||
TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
|
||||
TEST_UTIL.getConfiguration().set(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME,
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES).setCompressionType(comprAlgo)
|
||||
.setBloomFilterType(bloomType).setMaxVersions(MAX_VERSIONS)
|
||||
|
|
|
@ -78,10 +78,13 @@ public class TestRowPrefixBloomFilter {
|
|||
private static final int BLOCKSIZE_SMALL = 8192;
|
||||
private static final float err = (float) 0.01;
|
||||
private static final int prefixLength = 10;
|
||||
private static final String delimiter = "#";
|
||||
private static final String invalidFormatter = "%08d";
|
||||
private static final String prefixFormatter = "%010d";
|
||||
private static final String suffixFormatter = "%010d";
|
||||
private static final int prefixRowCount = 50;
|
||||
private static final int suffixRowCount = 10;
|
||||
private static final int fixedLengthExpKeys = prefixRowCount;
|
||||
private static final BloomType bt = BloomType.ROWPREFIX_FIXED_LENGTH;
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
@ -92,7 +95,6 @@ public class TestRowPrefixBloomFilter {
|
|||
conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
|
||||
conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
|
||||
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, prefixLength);
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, delimiter);
|
||||
|
||||
localfs =
|
||||
(conf.get("fs.defaultFS", "file:///").compareTo("file:///") == 0);
|
||||
|
@ -132,8 +134,7 @@ public class TestRowPrefixBloomFilter {
|
|||
return reader.getStoreFileScanner(false, false, false, 0, 0, false);
|
||||
}
|
||||
|
||||
private void writeStoreFile(final Path f, BloomType bt, int expKeys, int prefixRowCount,
|
||||
int suffixRowCount) throws IOException {
|
||||
private void writeStoreFile(final Path f, BloomType bt, int expKeys) throws IOException {
|
||||
HFileContext meta = new HFileContextBuilder()
|
||||
.withBlockSize(BLOCKSIZE_SMALL)
|
||||
.withChecksumType(CKTYPE)
|
||||
|
@ -152,18 +153,20 @@ public class TestRowPrefixBloomFilter {
|
|||
for (int i = 0; i < prefixRowCount; i += 2) { // prefix rows
|
||||
String prefixRow = String.format(prefixFormatter, i);
|
||||
for (int j = 0; j < suffixRowCount; j++) { // suffix rows
|
||||
String row = prefixRow + "#" + String.format(suffixFormatter, j);
|
||||
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
|
||||
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
|
||||
String row = generateRowWithSuffix(prefixRow, j);
|
||||
KeyValue kv =
|
||||
new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("col"), now,
|
||||
Bytes.toBytes("value"));
|
||||
writer.append(kv);
|
||||
}
|
||||
}
|
||||
|
||||
//Put with invalid row style
|
||||
for (int i = prefixRowCount; i < prefixRowCount*2; i += 2) { // prefix rows
|
||||
for (int i = prefixRowCount; i < prefixRowCount * 2; i += 2) { // prefix rows
|
||||
String row = String.format(invalidFormatter, i);
|
||||
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
|
||||
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
|
||||
KeyValue kv =
|
||||
new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("col"), now,
|
||||
Bytes.toBytes("value"));
|
||||
writer.append(kv);
|
||||
}
|
||||
} finally {
|
||||
|
@ -171,66 +174,45 @@ public class TestRowPrefixBloomFilter {
|
|||
}
|
||||
}
|
||||
|
||||
private String generateRowWithSuffix(String prefixRow, int suffix) {
|
||||
StringBuilder row = new StringBuilder(prefixRow);
|
||||
row.append("#");
|
||||
row.append(String.format(suffixFormatter, suffix));
|
||||
return row.toString();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRowPrefixBloomFilter() throws Exception {
|
||||
FileSystem fs = FileSystem.getLocal(conf);
|
||||
BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED};
|
||||
int prefixRowCount = 50;
|
||||
int suffixRowCount = 10;
|
||||
int expKeys = 50;
|
||||
float expErr = 2*prefixRowCount*suffixRowCount*err;
|
||||
for (int x : new int[]{0,1}) {
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount);
|
||||
float expErr = 2 * prefixRowCount * suffixRowCount * err;
|
||||
int expKeys = fixedLengthExpKeys;
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt, expKeys);
|
||||
|
||||
// read the file
|
||||
StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true,
|
||||
new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
// read the file
|
||||
StoreFileReader reader =
|
||||
new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
|
||||
//check basic param
|
||||
assertEquals(bt[x], reader.getBloomFilterType());
|
||||
if (bt[x] == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
assertEquals(prefixLength, reader.getPrefixLength());
|
||||
assertEquals("null", Bytes.toStringBinary(reader.getDelimiter()));
|
||||
} else if (bt[x] == BloomType.ROWPREFIX_DELIMITED){
|
||||
assertEquals(-1, reader.getPrefixLength());
|
||||
assertEquals(delimiter, Bytes.toStringBinary(reader.getDelimiter()));
|
||||
}
|
||||
assertEquals(expKeys, reader.getGeneralBloomFilter().getKeyCount());
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor())
|
||||
.thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
// check false positives rate
|
||||
int falsePos = 0;
|
||||
int falseNeg = 0;
|
||||
for (int i = 0; i < prefixRowCount; i++) { // prefix rows
|
||||
String prefixRow = String.format(prefixFormatter, i);
|
||||
for (int j = 0; j < suffixRowCount; j++) { // suffix rows
|
||||
String startRow = prefixRow + "#" + String.format(suffixFormatter, j);
|
||||
String stopRow = prefixRow + "#" + String.format(suffixFormatter, j+1);
|
||||
Scan scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
boolean shouldPrefixRowExist = i % 2 == 0;
|
||||
if (shouldPrefixRowExist) {
|
||||
if (!exists) {
|
||||
falseNeg++;
|
||||
}
|
||||
} else {
|
||||
if (exists) {
|
||||
falsePos++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = prefixRowCount; i < prefixRowCount * 2; i++) { // prefix rows
|
||||
String row = String.format(invalidFormatter, i);
|
||||
Scan scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
//check basic param
|
||||
assertEquals(bt, reader.getBloomFilterType());
|
||||
assertEquals(prefixLength, reader.getPrefixLength());
|
||||
assertEquals(expKeys, reader.getGeneralBloomFilter().getKeyCount());
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
// check false positives rate
|
||||
int falsePos = 0;
|
||||
int falseNeg = 0;
|
||||
for (int i = 0; i < prefixRowCount; i++) { // prefix rows
|
||||
String prefixRow = String.format(prefixFormatter, i);
|
||||
for (int j = 0; j < suffixRowCount; j++) { // suffix rows
|
||||
String startRow = generateRowWithSuffix(prefixRow, j);
|
||||
String stopRow = generateRowWithSuffix(prefixRow, j + 1);
|
||||
Scan scan =
|
||||
new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
boolean shouldPrefixRowExist = i % 2 == 0;
|
||||
if (shouldPrefixRowExist) {
|
||||
|
@ -243,157 +225,136 @@ public class TestRowPrefixBloomFilter {
|
|||
}
|
||||
}
|
||||
}
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
assertEquals("False negatives: " + falseNeg, 0, falseNeg);
|
||||
int maxFalsePos = (int) (2 * expErr);
|
||||
assertTrue("Too many false positives: " + falsePos
|
||||
+ " (err=" + err + ", expected no more than " + maxFalsePos + ")",
|
||||
falsePos <= maxFalsePos);
|
||||
}
|
||||
|
||||
for (int i = prefixRowCount; i < prefixRowCount * 2; i++) { // prefix rows
|
||||
String row = String.format(invalidFormatter, i);
|
||||
Scan scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
boolean shouldPrefixRowExist = i % 2 == 0;
|
||||
if (shouldPrefixRowExist) {
|
||||
if (!exists) {
|
||||
falseNeg++;
|
||||
}
|
||||
} else {
|
||||
if (exists) {
|
||||
falsePos++;
|
||||
}
|
||||
}
|
||||
}
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
assertEquals("False negatives: " + falseNeg, 0, falseNeg);
|
||||
int maxFalsePos = (int) (2 * expErr);
|
||||
assertTrue(
|
||||
"Too many false positives: " + falsePos + " (err=" + err + ", expected no more than " +
|
||||
maxFalsePos + ")", falsePos <= maxFalsePos);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRowPrefixBloomFilterWithGet() throws Exception {
|
||||
FileSystem fs = FileSystem.getLocal(conf);
|
||||
BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED};
|
||||
int prefixRowCount = 50;
|
||||
int suffixRowCount = 10;
|
||||
int expKeys = 50;
|
||||
for (int x : new int[]{0,1}) {
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount);
|
||||
int expKeys = fixedLengthExpKeys;
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt, expKeys);
|
||||
|
||||
StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true,
|
||||
new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
StoreFileReader reader =
|
||||
new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor())
|
||||
.thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
|
||||
//Get with valid row style
|
||||
//prefix row in bloom
|
||||
String prefixRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
String row = prefixRow + "#" + String.format(suffixFormatter, 0);
|
||||
Scan scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
//Get with valid row style
|
||||
//prefix row in bloom
|
||||
String prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
|
||||
String row = generateRowWithSuffix(prefixRow, 0);
|
||||
Scan scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
|
||||
// prefix row not in bloom
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount-1);
|
||||
row = prefixRow + "#" + String.format(suffixFormatter, 0);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
// prefix row not in bloom
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount - 1);
|
||||
row = generateRowWithSuffix(prefixRow, 0);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
|
||||
// Get with invalid row style
|
||||
// ROWPREFIX: the length of row is less than prefixLength
|
||||
// ROWPREFIX_DELIMITED: Row does not contain delimiter
|
||||
// row in bloom
|
||||
row = String.format(invalidFormatter, prefixRowCount+2);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
// Get with invalid row style
|
||||
// ROWPREFIX: the length of row is less than prefixLength
|
||||
// row in bloom
|
||||
row = String.format(invalidFormatter, prefixRowCount + 2);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
|
||||
// row not in bloom
|
||||
row = String.format(invalidFormatter, prefixRowCount+1);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
// row not in bloom
|
||||
row = String.format(invalidFormatter, prefixRowCount + 1);
|
||||
scan = new Scan(new Get(Bytes.toBytes(row)));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
}
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRowPrefixBloomFilterWithScan() throws Exception {
|
||||
FileSystem fs = FileSystem.getLocal(conf);
|
||||
BloomType[] bt = {BloomType.ROWPREFIX_FIXED_LENGTH, BloomType.ROWPREFIX_DELIMITED};
|
||||
int prefixRowCount = 50;
|
||||
int suffixRowCount = 10;
|
||||
int expKeys = 50;
|
||||
for (int x : new int[]{0,1}) {
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt[x], expKeys, prefixRowCount, suffixRowCount);
|
||||
int expKeys = fixedLengthExpKeys;
|
||||
// write the file
|
||||
Path f = new Path(testDir, name.getMethodName());
|
||||
writeStoreFile(f, bt, expKeys);
|
||||
|
||||
StoreFileReader reader = new StoreFileReader(fs, f, cacheConf, true,
|
||||
new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
StoreFileReader reader =
|
||||
new StoreFileReader(fs, f, cacheConf, true, new AtomicInteger(0), true, conf);
|
||||
reader.loadFileInfo();
|
||||
reader.loadBloomfilter();
|
||||
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor())
|
||||
.thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
|
||||
//Scan with valid row style. startRow and stopRow have a common prefix.
|
||||
//And the length of the common prefix is no less than prefixLength.
|
||||
//prefix row in bloom
|
||||
String prefixRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
String startRow = prefixRow + "#" + String.format(suffixFormatter, 0);
|
||||
String stopRow = prefixRow + "#" + String.format(suffixFormatter, 1);
|
||||
Scan scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
//Scan with valid row style. startRow and stopRow have a common prefix.
|
||||
//And the length of the common prefix is no less than prefixLength.
|
||||
//prefix row in bloom
|
||||
String prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
|
||||
String startRow = generateRowWithSuffix(prefixRow, 0);
|
||||
String stopRow = generateRowWithSuffix(prefixRow, 1);
|
||||
Scan scan =
|
||||
new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
|
||||
// prefix row not in bloom
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount-1);
|
||||
startRow = prefixRow + "#" + String.format(suffixFormatter, 0);
|
||||
stopRow = prefixRow + "#" + String.format(suffixFormatter, 1);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
// prefix row not in bloom
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount - 1);
|
||||
startRow = generateRowWithSuffix(prefixRow, 0);
|
||||
stopRow = generateRowWithSuffix(prefixRow, 1);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertFalse(exists);
|
||||
|
||||
// There is no common prefix between startRow and stopRow.
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
startRow = prefixRow + "#" + String.format(suffixFormatter, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
// There is no common prefix between startRow and stopRow.
|
||||
prefixRow = String.format(prefixFormatter, prefixRowCount - 2);
|
||||
startRow = generateRowWithSuffix(prefixRow, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
|
||||
if (bt[x] == BloomType.ROWPREFIX_FIXED_LENGTH) {
|
||||
// startRow and stopRow have a common prefix.
|
||||
// But the length of the common prefix is less than prefixLength.
|
||||
String prefixStartRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
String prefixStopRow = String.format(prefixFormatter, prefixRowCount-1);
|
||||
startRow = prefixStartRow + "#" + String.format(suffixFormatter, 0);
|
||||
stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
}else if (bt[x] == BloomType.ROWPREFIX_DELIMITED) {
|
||||
// startRow does not contain delimiter
|
||||
String prefixStartRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
String prefixStopRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
startRow = prefixStartRow + String.format(suffixFormatter, 0);
|
||||
stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
// startRow and stopRow have a common prefix.
|
||||
// But the length of the common prefix is less than prefixLength.
|
||||
String prefixStartRow = String.format(prefixFormatter, prefixRowCount - 2);
|
||||
String prefixStopRow = String.format(prefixFormatter, prefixRowCount - 1);
|
||||
startRow = generateRowWithSuffix(prefixStartRow, 0);
|
||||
stopRow = generateRowWithSuffix(prefixStopRow, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow)).withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
|
||||
// startRow contains delimiter, but stopRow does not have the same prefix as startRow.
|
||||
prefixStartRow = String.format(prefixFormatter, prefixRowCount-2);
|
||||
prefixStopRow = String.format(prefixFormatter, prefixRowCount-1);
|
||||
startRow = prefixStartRow + "#" + String.format(suffixFormatter, 0);
|
||||
stopRow = prefixStopRow + "#" + String.format(suffixFormatter, 0);
|
||||
scan = new Scan().withStartRow(Bytes.toBytes(startRow))
|
||||
.withStopRow(Bytes.toBytes(stopRow));
|
||||
exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
assertTrue(exists);
|
||||
}
|
||||
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
}
|
||||
reader.close(true); // evict because we are about to delete the file
|
||||
fs.delete(f, true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -105,7 +105,6 @@ public class TestScanWithBloomError {
|
|||
conf = TEST_UTIL.getConfiguration();
|
||||
fs = FileSystem.get(conf);
|
||||
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
|
||||
conf.set(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -144,7 +144,6 @@ public class TestSeekOptimizations {
|
|||
rand = new Random(91238123L);
|
||||
expectedKVs.clear();
|
||||
TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
|
||||
TEST_UTIL.getConfiguration().set(BloomFilterUtil.DELIMITER_KEY, "#");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -485,6 +484,5 @@ public class TestSeekOptimizations {
|
|||
HBaseTestingUtility.safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* An AlreadyExists exceptions signals that a table with the specified
|
||||
* name already exists
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class AlreadyExists extends org.apache.thrift.TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* A BatchMutation object is used to apply a number of Mutations to a single row.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* such as the number of versions, compression settings, etc. It is
|
||||
* used as input when creating a table or adding a column.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class Hbase {
|
||||
|
||||
public interface Iface {
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* to the Hbase master or an Hbase region server. Also used to return
|
||||
* more general Hbase error conditions.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class IOError extends org.apache.thrift.TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* An IllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class IllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* A Mutation object is used to either update or delete a column-value.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* An Append object is used to specify the parameters for performing the append operation.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
|
||||
|
||||
|
|
|
@ -13,7 +13,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* the timestamp of a cell to a first-class value, making it easy to take
|
||||
* note of temporal data. Cell is used all the way from HStore up to HTable.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* Holds column name and the cell.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
* For increments that are not incrementColumnValue
|
||||
* equivalents.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* A TRegionInfo contains information about an HTable region.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* Holds row name and then a map of columns to cells.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift.generated;
|
|||
/**
|
||||
* A Scan object is used to specify scanner parameters when opening a scanner.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
|
||||
|
|
|
@ -885,7 +885,6 @@ public final class ThriftUtilities {
|
|||
case 1: return BloomType.ROW;
|
||||
case 2: return BloomType.ROWCOL;
|
||||
case 3: return BloomType.ROWPREFIX_FIXED_LENGTH;
|
||||
case 4: return BloomType.ROWPREFIX_DELIMITED;
|
||||
default: return BloomType.ROW;
|
||||
}
|
||||
}
|
||||
|
@ -1110,7 +1109,6 @@ public final class ThriftUtilities {
|
|||
case ROW: return TBloomFilterType.ROW;
|
||||
case ROWCOL: return TBloomFilterType.ROWCOL;
|
||||
case ROWPREFIX_FIXED_LENGTH: return TBloomFilterType.ROWPREFIX_FIXED_LENGTH;
|
||||
case ROWPREFIX_DELIMITED: return TBloomFilterType.ROWPREFIX_DELIMITED;
|
||||
default: return TBloomFilterType.ROW;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.regionserver.BloomType
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TBloomFilterType implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Bloomfilters disabled
|
||||
|
@ -28,11 +28,7 @@ public enum TBloomFilterType implements org.apache.thrift.TEnum {
|
|||
/**
|
||||
* Bloom enabled with Table row prefix as Key, specify the length of the prefix
|
||||
*/
|
||||
ROWPREFIX_FIXED_LENGTH(3),
|
||||
/**
|
||||
* Bloom enabled with Table row prefix as Key, specify the delimiter of the prefix
|
||||
*/
|
||||
ROWPREFIX_DELIMITED(4);
|
||||
ROWPREFIX_FIXED_LENGTH(3);
|
||||
|
||||
private final int value;
|
||||
|
||||
|
@ -62,8 +58,6 @@ public enum TBloomFilterType implements org.apache.thrift.TEnum {
|
|||
return ROWCOL;
|
||||
case 3:
|
||||
return ROWPREFIX_FIXED_LENGTH;
|
||||
case 4:
|
||||
return ROWPREFIX_DELIMITED;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* in a HBase table by column family and optionally
|
||||
* a column qualifier and timestamp
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Represents a single cell and the amount to increment it by
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Represents a single cell and its value.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.filter.CompareFilter$CompareOp.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TCompareOp implements org.apache.thrift.TEnum {
|
||||
LESS(0),
|
||||
LESS_OR_EQUAL(1),
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.io.compress.Algorithm
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TCompressionAlgorithm implements org.apache.thrift.TEnum {
|
||||
LZO(0),
|
||||
GZ(1),
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - STRONG means reads only from primary region
|
||||
* - TIMELINE means reads might return values from secondary region replicas
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TConsistency implements org.apache.thrift.TEnum {
|
||||
STRONG(1),
|
||||
TIMELINE(2);
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TDataBlockEncoding implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Disable data block encoding.
|
||||
|
|
|
@ -33,7 +33,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - DELETE_COLUMN means exactly one version will be removed,
|
||||
* - DELETE_COLUMNS means previous versions will also be removed.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TDeleteType implements org.apache.thrift.TEnum {
|
||||
DELETE_COLUMN(0),
|
||||
DELETE_COLUMNS(1),
|
||||
|
|
|
@ -14,7 +14,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* - SYNC_WAL means write the Mutation to the WAL synchronously,
|
||||
* - FSYNC_WAL means Write the Mutation to the WAL synchronously and force the entries to disk.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TDurability implements org.apache.thrift.TEnum {
|
||||
USE_DEFAULT(0),
|
||||
SKIP_WAL(1),
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* If you specify a time range and a timestamp the range is ignored.
|
||||
* Timestamps on TColumns are ignored.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class THBaseService {
|
||||
|
||||
public interface Iface {
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
|
||||
|
||||
|
|
|
@ -12,7 +12,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* to the HBase master or a HBase region server. Also used to return
|
||||
* more general HBase error conditions.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TIOError extends org.apache.thrift.TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* A TIllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TIllegalArgument extends org.apache.thrift.TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.KeepDeletedCells
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TKeepDeletedCells implements org.apache.thrift.TEnum {
|
||||
/**
|
||||
* Deleted Cells are not retained.
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* Atomic mutation for the specified row. It can be either Put or Delete.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TMutation extends org.apache.thrift.TUnion<TMutation, TMutation._Fields> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMutation");
|
||||
private static final org.apache.thrift.protocol.TField PUT_FIELD_DESC = new org.apache.thrift.protocol.TField("put", org.apache.thrift.protocol.TType.STRUCT, (short)1);
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.NamespaceDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TNamespaceDescriptor implements org.apache.thrift.TBase<TNamespaceDescriptor, TNamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TNamespaceDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor");
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public enum TReadType implements org.apache.thrift.TEnum {
|
||||
DEFAULT(1),
|
||||
STREAM(2),
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* if no Result is found, row and columnValues will not be set.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
|
||||
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
/**
|
||||
* A TRowMutations object is used to apply a number of Mutations to a single row.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp.
|
||||
* Max versions defaults to 1.
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.TableDescriptor
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TTableDescriptor implements org.apache.thrift.TBase<TTableDescriptor, TTableDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TTableDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor");
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ package org.apache.hadoop.hbase.thrift2.generated;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.TableName
|
||||
*/
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TTableName implements org.apache.thrift.TBase<TTableName, TTableName._Fields>, java.io.Serializable, Cloneable, Comparable<TTableName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName");
|
||||
|
||||
|
|
|
@ -7,7 +7,7 @@
|
|||
package org.apache.hadoop.hbase.thrift2.generated;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-01-27")
|
||||
@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.12.0)", date = "2019-02-22")
|
||||
public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
|
||||
|
||||
|
|
|
@ -339,10 +339,6 @@ enum TBloomFilterType {
|
|||
* Bloom enabled with Table row prefix as Key, specify the length of the prefix
|
||||
*/
|
||||
ROWPREFIX_FIXED_LENGTH = 3,
|
||||
/**
|
||||
* Bloom enabled with Table row prefix as Key, specify the delimiter of the prefix
|
||||
*/
|
||||
ROWPREFIX_DELIMITED = 4
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue