HBASE-3173 HBase 2984 breaks ability to specify BLOOMFILTER & COMPRESSION via shell AND HBASE-3310 Failing creating/altering table with compression agrument

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1043173 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-12-07 19:40:25 +00:00
parent 5964303163
commit 7142228dd2
2 changed files with 22 additions and 2 deletions

View File

@ -11,6 +11,10 @@ Release 0.91.0 - Unreleased
previously expired RS instances to rejoin cluster previously expired RS instances to rejoin cluster
HBASE-3283 NPE in AssignmentManager if processing shutdown of RS who HBASE-3283 NPE in AssignmentManager if processing shutdown of RS who
doesn't have any regions assigned to it doesn't have any regions assigned to it
HBASE-3173 HBase 2984 breaks ability to specify BLOOMFILTER &
COMPRESSION via shell
HBASE-3310 Failing creating/altering table with compression agrument from
the HBase shell (Igor Ranitovic via Stack)
IMPROVEMENTS IMPROVEMENTS
HBASE-2001 Coprocessors: Colocate user code with regions (Mingjie Lai via HBASE-2001 Coprocessors: Colocate user code with regions (Mingjie Lai via

View File

@ -27,6 +27,8 @@ java_import org.apache.hadoop.hbase.HTableDescriptor
java_import org.apache.hadoop.hbase.HRegionInfo java_import org.apache.hadoop.hbase.HRegionInfo
java_import org.apache.hadoop.hbase.util.Bytes java_import org.apache.hadoop.hbase.util.Bytes
java_import org.apache.zookeeper.ZooKeeper java_import org.apache.zookeeper.ZooKeeper
java_import org.apache.hadoop.hbase.io.hfile.Compression
java_import org.apache.hadoop.hbase.regionserver.StoreFile
# Wrapper for org.apache.hadoop.hbase.client.HBaseAdmin # Wrapper for org.apache.hadoop.hbase.client.HBaseAdmin
@ -352,14 +354,28 @@ module Hbase
family ||= HColumnDescriptor.new(name.to_java_bytes) family ||= HColumnDescriptor.new(name.to_java_bytes)
family.setBlockCacheEnabled(JBoolean.valueOf(arg[HColumnDescriptor::BLOCKCACHE])) if arg.include?(HColumnDescriptor::BLOCKCACHE) family.setBlockCacheEnabled(JBoolean.valueOf(arg[HColumnDescriptor::BLOCKCACHE])) if arg.include?(HColumnDescriptor::BLOCKCACHE)
family.setBloomFilterType(arg[HColumnDescriptor::BLOOMFILTER]) if arg.include?(HColumnDescriptor::BLOOMFILTER)
family.setScope(JInteger.valueOf(arg[REPLICATION_SCOPE])) if arg.include?(HColumnDescriptor::REPLICATION_SCOPE) family.setScope(JInteger.valueOf(arg[REPLICATION_SCOPE])) if arg.include?(HColumnDescriptor::REPLICATION_SCOPE)
family.setInMemory(JBoolean.valueOf(arg[IN_MEMORY])) if arg.include?(HColumnDescriptor::IN_MEMORY) family.setInMemory(JBoolean.valueOf(arg[IN_MEMORY])) if arg.include?(HColumnDescriptor::IN_MEMORY)
family.setTimeToLive(JInteger.valueOf(arg[HColumnDescriptor::TTL])) if arg.include?(HColumnDescriptor::TTL) family.setTimeToLive(JInteger.valueOf(arg[HColumnDescriptor::TTL])) if arg.include?(HColumnDescriptor::TTL)
family.setCompressionType(arg[HColumnDescriptor::COMPRESSION]) if arg.include?(HColumnDescriptor::COMPRESSION) family.setCompressionType(arg[HColumnDescriptor::COMPRESSION]) if arg.include?(HColumnDescriptor::COMPRESSION)
family.setBlocksize(JInteger.valueOf(arg[HColumnDescriptor::BLOCKSIZE])) if arg.include?(HColumnDescriptor::BLOCKSIZE) family.setBlocksize(JInteger.valueOf(arg[HColumnDescriptor::BLOCKSIZE])) if arg.include?(HColumnDescriptor::BLOCKSIZE)
family.setMaxVersions(JInteger.valueOf(arg[VERSIONS])) if arg.include?(HColumnDescriptor::VERSIONS) family.setMaxVersions(JInteger.valueOf(arg[VERSIONS])) if arg.include?(HColumnDescriptor::VERSIONS)
if arg.include?(HColumnDescriptor::BLOOMFILTER)
bloomtype = arg[HColumnDescriptor::BLOOMFILTER].upcase
unless StoreFile::BloomType.constants.include?(bloomtype)
raise(ArgumentError, "BloomFilter type #{bloomtype} is not supported. Use one of " + StoreFile::BloomType.constants.join(" "))
else
family.setBloomFilterType(StoreFile::BloomType.valueOf(bloomtype))
end
end
if arg.include?(HColumnDescriptor::COMPRESSION)
compression = arg[HColumnDescriptor::COMPRESSION].upcase
unless Compression::Algorithm.constants.include?(compression)
raise(ArgumentError, "Compression #{compression} is not supported. Use one of " + Compression::Algorithm.constants.join(" "))
else
family.setCompressionType(Compression::Algorithm.valueOf(compression))
end
end
return family return family
end end