You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/03/12 22:17:20 UTC

svn commit: r1576909 [14/18] - in /hbase/branches/0.89-fb/src: ./ examples/thrift/ main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/avro/ main/java/org/apache/hadoop/hbase/avro/generated/ main/java/org/apache/hadoop/hbase/client/ ma...

Added: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift (added)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift Wed Mar 12 21:17:13 2014
@@ -0,0 +1,247 @@
+namespace java.swift org.apache.hadoop.hbase
+namespace cpp facebook.hbase.hbcpp
+
+
+enum Type {
+  INTEGER, LIST_OF_RESULTS, EXCEPTION
+}
+
+enum HFileStat {
+  KEYVALUECOUNT
+}
+
+struct KeyValue {
+  1: binary bytes;
+}
+
+struct Put {
+  1: binary row;
+  2: i64 ts;
+  3: map<binary, list<KeyValue>> familyMap;
+  4: i64 lockId;
+  5: bool writeToWAL;
+}
+
+struct TimeRange {
+  1: bool allTime;
+  2: i64 minStamp;
+  3: i64 maxStamp;
+}
+
+struct TFilter {
+  1: binary filterBytes;
+}
+
+struct Get {
+  1: binary row;
+  2: i64 lockId;
+  3: i32 maxVersions;
+  4: i32 storeLimit;
+  5: i32 storeOffset;
+  6: TimeRange tr;
+  7: map<binary, set<binary>> familyMap;
+  8: i64 effectiveTS;
+  9: TFilter tFilter;
+}
+
+struct MultiPut {
+  1: map<binary, list<Put>> putsSerial;
+}
+
+struct Delete {
+  1: binary row;
+  2: i64 timeStamp;
+  3: map<binary, list<KeyValue>> familyMapSerial;
+  4: i64 lockId;
+  5: bool writeToWAL;
+}
+
+struct Scan {
+  1: binary startRow;
+  2: binary stopRow;
+  3: i32 maxVersions;
+  4: i32 batch;
+  5: i32 caching;
+  6: i32 storeLimit;
+  7: i32 storeOffset;
+  8: bool serverPrefetching;
+  9: i32 maxResponseSize;
+  10: bool partialRow;
+  11: bool cacheBlocks;
+  12: TimeRange tr;
+  13: map<binary, set<binary>> familyMap;
+  14: i64 effectiveTS;
+  15: i32 currentPartialResponseSize;
+  16: TFilter tFilter;
+  17: bool preloadBlocks;
+}
+
+struct HColumnDescriptor {
+  1: binary name;
+  2: map<binary, binary> valuesMap;
+}
+
+struct HTableDescriptor {
+  1: binary name;
+  2: list<HColumnDescriptor> families;
+  3: map<binary, binary> values;
+}
+
+struct HRegionInfo {
+  1: HTableDescriptor tableDesc;
+  2: binary startKey;
+  3: binary endKey;
+  4: bool split;
+  5: i64 regionid;
+  6: binary splitPoint;
+  7: bool offline;
+}
+
+struct MultiPutResponse {
+  1: map<binary, i32> answers;
+}
+
+struct Result {
+  1: list<KeyValue> kvs;
+}
+
+struct HServerAddress {
+  1: string bindAddress;
+  2: i32 port;
+}
+
+struct RegionLoad {
+  1: binary name;
+  2: i32 stores;
+  3: i32 storefiles;
+  4: i32 storefileSizeMB;
+  5: i32 memstoreSizeMB;
+  6: i32 storefileIndexSizeMB;
+  7: i32 rootIndexSizeKB;
+  8: i32 totalStaticIndexSizeKB;
+  9: i32 totalStaticBloomSizeKB;
+}
+
+struct HServerLoad {
+  1: i32 numberOfRegions;
+  2: i32 numberOfRequests;
+  3: i32 usedHeapMB;
+  4: i32 maxHeapMB;
+  5: list<RegionLoad> regionLoad;
+  6: i64 lastLoadRefreshTime;
+  7: i64 expireAfter;
+}
+
+struct HServerInfo {
+  1: HServerAddress serverAddress;
+  2: i64 startCode;
+  3: string hostname;
+  4: map<binary, i64> flushedSequenceIdByRegion;
+  5: bool sendSequenceIds;
+  6: string cachedHostnamePort;
+  7: HServerLoad load;
+}
+
+exception ThriftHBaseException {
+  1: string message;
+  2: string exceptionClass;
+  3: binary serializedServerJavaEx;
+}
+
+struct MultiAction {
+  1: map<binary, list<Get>> gets;
+  2: map<binary, list<Put>> puts;
+  3: map<binary, list<Delete>> deletes;
+}
+
+struct IntegerOrResultOrException {
+  1: i32 integer;
+  2: ThriftHBaseException ex;
+  3: list<Result> results;
+  4: Type type;
+}
+
+struct TMultiResponse {
+  1: map<binary, IntegerOrResultOrException> resultsForGet;
+  2: map<binary, IntegerOrResultOrException> resultsForPut;
+  3: map<binary, IntegerOrResultOrException> resultsForDelete;
+}
+
+struct TRowMutations {
+  1: binary row;
+  2: list<Put> puts;
+  3: list<Delete> deletes;
+}
+
+struct AssignmentPlan {
+  1: map<HRegionInfo, list<HServerAddress>> assignmentMap;
+}
+
+struct RowLock {
+  1: binary row;
+  2: i64 lockId;
+}
+
+struct Bucket {
+  1: binary startRow;
+  2: binary endRow;
+  3: double numRows;
+  4: map<HFileStat, double> hfileStats;
+}
+
+service ThriftHRegionInterface {
+  void bulkLoadHFile(1: string hfilePath, 2: binary regionName, 3: binary familyName) throws (1: ThriftHBaseException ex1);
+  void bulkLoadHFileSeqNum(1: string hfilePath, 2: binary regionName, 3: binary familyName, 4: bool assignSeqNum) throws (1: ThriftHBaseException ex1);
+  bool checkAndDelete(1: binary regionName, 2: binary row, 3: binary family, 4: binary qualifier, 5: binary value, 6: Delete deleteArg) throws (1: ThriftHBaseException ex1);
+  bool checkAndPut(1: binary regionName, 2: binary row, 3: binary family, 4: binary qualifier, 5: binary value, 6: Put put) throws (1: ThriftHBaseException ex1);
+  void close(1: i64 scannerId) throws (1: ThriftHBaseException ex1);
+  void closeRegion(1: HRegionInfo hri, 2: bool reportWhenCompleted) throws (1: ThriftHBaseException ex1);
+  void deleteAsync(1: binary regionName, 2: Delete deleteArg) throws (1: ThriftHBaseException ex1);
+  bool exists(1: binary regionName, 2: Get get) throws (1: ThriftHBaseException ex1);
+  void flushRegion(1: binary regionName) throws (1: ThriftHBaseException ex1);
+  void flushRegionIfOlderThanTS(1: binary regionName, 2: i64 ifOlderThanTS) throws (1: ThriftHBaseException ex1);
+  Result getAsync(1: binary regionName, 2: Get get) throws (1: ThriftHBaseException ex1);
+  Result getClosestRowBefore(1: binary regionName, 2: binary row, 3: binary family) throws (1: ThriftHBaseException ex1);
+  Result getClosestRowBeforeAsync(1: binary regionName, 2: binary row, 3: binary family) throws (1: ThriftHBaseException ex1);
+  string getConfProperty(1: string arg0) throws (1: ThriftHBaseException ex1);
+  i64 getCurrentTimeMillis();
+  list<string> getHLogsList(1: bool rollCurrentHLog) throws (1: ThriftHBaseException ex1);
+  HServerInfo getHServerInfo() throws (1: ThriftHBaseException ex1);
+  list<Bucket> getHistogram(1: binary arg0) throws (1: ThriftHBaseException ex1);
+  list<Bucket> getHistogramForStore(1: binary arg0, 2: binary arg1) throws (1: ThriftHBaseException ex1);
+  i64 getLastFlushTime(1: binary regionName);
+  map<binary, i64> getLastFlushTimes();
+  HRegionInfo getRegionInfo(1: binary regionName) throws (1: ThriftHBaseException ex1);
+  list<HRegionInfo> getRegionsAssignment() throws (1: ThriftHBaseException ex1);
+  list<Result> getRows(1: binary regionName, 2: list<Get> gets) throws (1: ThriftHBaseException ex1);
+  i64 getStartCode();
+  string getStopReason();
+  list<string> getStoreFileList(1: binary regionName, 2: binary columnFamily) throws (1: ThriftHBaseException ex1);
+  list<string> getStoreFileListForAllColumnFamilies(1: binary regionName) throws (1: ThriftHBaseException ex1);
+  list<string> getStoreFileListForColumnFamilies(1: binary regionName, 2: list<binary> columnFamilies) throws (1: ThriftHBaseException ex1);
+  i64 incrementColumnValue(1: binary regionName, 2: binary row, 3: binary family, 4: binary qualifier, 5: i64 amount, 6: bool writeToWAL) throws (1: ThriftHBaseException ex1);
+  bool isStopped();
+  i64 lockRow(1: binary regionName, 2: binary row) throws (1: ThriftHBaseException ex1);
+  RowLock lockRowAsync(1: binary regionName, 2: binary row) throws (1: ThriftHBaseException ex1);
+  TMultiResponse multiAction(1: MultiAction multi) throws (1: ThriftHBaseException ex1);
+  MultiPutResponse multiPut(1: MultiPut puts) throws (1: ThriftHBaseException ex1);
+  void mutateRow(1: binary regionName, 2: TRowMutations arm) throws (1: ThriftHBaseException ex1);
+  void mutateRowAsync(1: binary regionName, 2: TRowMutations arm) throws (1: ThriftHBaseException ex1);
+  void mutateRows(1: binary regionName, 2: list<TRowMutations> armList) throws (1: ThriftHBaseException ex1);
+  Result next(1: i64 scannerId) throws (1: ThriftHBaseException ex1);
+  list<Result> nextRows(1: i64 scannerId, 2: i32 numberOfRows) throws (1: ThriftHBaseException ex1);
+  i64 openScanner(1: binary regionName, 2: Scan scan) throws (1: ThriftHBaseException ex1);
+  void processDelete(1: binary regionName, 2: Delete deleteArg) throws (1: ThriftHBaseException ex1);
+  Result processGet(1: binary regionName, 2: Get get) throws (1: ThriftHBaseException ex1);
+  i32 processListOfDeletes(1: binary regionName, 2: list<Delete> deletes) throws (1: ThriftHBaseException ex1);
+  void processPut(1: binary regionName, 2: Put put) throws (1: ThriftHBaseException ex1);
+  i32 putRows(1: binary regionName, 2: list<Put> puts) throws (1: ThriftHBaseException ex1);
+  void setHDFSQuorumReadTimeoutMillis(1: i64 timeoutMillis);
+  void setNumHDFSQuorumReadThreads(1: i32 maxThreads);
+  void stop(1: string why);
+  void stopForRestart();
+  void unlockRow(1: binary regionName, 2: i64 lockId) throws (1: ThriftHBaseException ex1);
+  void unlockRowAsync(1: binary regionName, 2: i64 lockId) throws (1: ThriftHBaseException ex1);
+  void updateConfiguration() throws (1: ThriftHBaseException ex1);
+  i32 updateFavoredNodes(1: AssignmentPlan plan) throws (1: ThriftHBaseException ex1);
+}

Added: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/LegacyHBase.thrift
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/LegacyHBase.thrift?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/LegacyHBase.thrift (added)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/LegacyHBase.thrift Wed Mar 12 21:17:13 2014
@@ -0,0 +1,1139 @@
+/*
+ * 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.
+ */
+
+// ----------------------------------------------------------------
+// LegacyHBase.thrift
+//
+// This is a Thrift interface definition file for the Hbase service.
+// Target language libraries for C++, Java, Ruby, PHP, (and more) are
+// generated by running this file through the Thrift compiler with the
+// appropriate flags. The Thrift compiler binary and runtime
+// libraries for various languages are available
+// from the Apache Incubator (http://incubator.apache.org/thrift/)
+//
+// See the package.html file for information on the version of Thrift
+// used to generate the *.java files checked into the Hbase project.
+// ----------------------------------------------------------------
+
+namespace java org.apache.hadoop.hbase.thrift.generated
+namespace cpp  apache.hadoop.hbase.thrift
+namespace rb Apache.Hadoop.Hbase.Thrift
+namespace py hbase
+namespace perl Hbase
+
+//
+// Types
+//
+
+// NOTE: all variables with the Text type are assumed to be correctly
+// formatted UTF-8 strings.  This is a programming language and locale
+// dependent property that the client application is repsonsible for
+// maintaining.  If strings with an invalid encoding are sent, an
+// IOError will be thrown.
+
+typedef binary Text
+typedef binary Bytes
+typedef i32    ScannerID
+
+const i64 LATEST_TIMESTAMP = 0x7fffffffffffffff;
+
+/**
+ * TCell - Used to transport a cell value (byte[]) and the timestamp it was
+ * stored with together as a result for get and getRow methods. This promotes
+ * 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.
+ */
+struct TCell{
+  1:Bytes value,
+  2:i64 timestamp
+}
+
+/**
+ * An HColumnDescriptor contains information about a column family
+ * such as the number of versions, compression settings, etc. It is
+ * used as input when creating a table or adding a column.
+ */
+struct ColumnDescriptor {
+  1:Text name,
+  2:i32 maxVersions = 3,
+  3:string compression = "NONE",
+  4:bool inMemory = 0,
+  5:string bloomFilterType = "NONE",
+  6:i32 bloomFilterVectorSize = 0,
+  7:i32 bloomFilterNbHashes = 0,
+  8:bool blockCacheEnabled = 0,
+  9:i32 timeToLive = -1
+}
+
+/**
+ * A TRegionInfo contains information about an HTable region.
+ */
+struct TRegionInfo {
+  1:Text startKey,
+  2:Text endKey,
+  3:i64 id,
+  4:Text name,
+  5:byte version,
+  6:Text serverName,
+  7:i32 port
+}
+
+/**
+ * A Mutation object is used to either update or delete a column-value.
+ * If LATEST_TIMESTAMP is used, the server will select timestamp automatically.
+ */
+struct Mutation {
+  1:bool isDelete = 0,
+  2:Text column,
+  3:Text value,
+  4:bool writeToWAL = 1
+  5:i64 timestamp = LATEST_TIMESTAMP
+}
+
+/**
+ * A BatchMutation object is used to apply a number of Mutations to a single row.
+ */
+struct BatchMutation {
+  1:Text row,
+  2:list<Mutation> mutations
+}
+
+
+/**
+ * Holds row name and then a map of columns to cells.
+ */
+struct TRowResult {
+  1:Text row,
+  2:map<Text, TCell> columns
+}
+
+/**
+ * A Scan object is used to specify scanner parameters when opening a scanner.
+ */
+struct TScan {
+  1:optional Text startRow,
+  2:optional Text stopRow,
+  3:optional i64 timestamp,
+  4:optional list<Text> columns,
+  5:optional i32 caching,
+  6:optional Text filterString,
+  7:optional bool cachingBlocksEnabled,
+  8:optional i32 batchLimit
+}
+
+//
+// Exceptions
+//
+/**
+ * An IOError exception signals that an error occurred communicating
+ * to the Hbase master or an Hbase region server.  Also used to return
+ * more general Hbase error conditions.
+ */
+exception IOError {
+  1: string message,
+  2: i64 backoffTimeMillis,
+  3: string exceptionClass
+}
+
+/**
+ * An IllegalArgument exception indicates an illegal or invalid
+ * argument was passed into a procedure.
+ */
+exception IllegalArgument {
+  1:string message
+}
+
+/**
+ * An AlreadyExists exceptions signals that a table with the specified
+ * name already exists
+ */
+exception AlreadyExists {
+  1:string message
+}
+
+//
+// Service
+//
+// The method definitions should be kept in the alphabetical order for the ease
+// of merging changes between various client and server branches.
+
+service Hbase {
+  /**
+   * Atomically increment the column value specified. Returns the next value
+   * post increment.
+   */
+  i64 atomicIncrement(
+    /** name of table */
+    1:Text tableName,
+
+    /** row to increment */
+    2:Text row,
+
+    /** name of column */
+    3:Text column,
+
+    /** amount to increment by */
+    4:i64 value
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Applies a list of mutations to a single row only if the value for
+   * row, cf[:qualifier] equals valueCheck
+   *
+   * Accepts null or '' for valueCheck, in which case entry for
+   * row, cf[:qualifier] must not exist.
+   *
+   * @return bool whether the check passed and mutations were applied
+   */
+  bool checkAndMutateRow(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    3:Text columnCheck,
+    4:Text valueCheck,
+    /** list of mutation commands */
+    5:list<Mutation> mutations,
+
+    /** Put attributes */
+    6:map<Text, Text> attributes
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Same as above, but the puts and deletes are added at specified timestamp.
+   *
+   * NOTE: No way to specify what timerange to query for the checked value;
+   * it will look for most recent entry (the default Get behavior).
+   */
+  bool checkAndMutateRowTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    3:Text columnCheck,
+    4:Text valueCheck,
+    /** list of mutation commands */
+    5:list<Mutation> mutations,
+
+    /** timestamp */
+    6:i64 timestamp,
+
+    /** Put attributes */
+    7:map<Text, Text> attributes
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  void compact(1:Bytes tableNameOrRegionName)
+    throws (1:IOError io)
+
+  /**
+   * Create a table with the specified column families.  The name
+   * field for each ColumnDescriptor must be set and must end in a
+   * colon (:). All other fields are optional and will get default
+   * values if not explicitly specified.
+   *
+   * @throws IllegalArgument if an input parameter is invalid
+   *
+   * @throws AlreadyExists if the table name already exists
+   */
+  void createTable(
+    /** name of table to create */
+    1:Text tableName,
+
+    /** list of column family descriptors */
+    2:list<ColumnDescriptor> columnFamilies
+  ) throws (1:IOError io, 2:IllegalArgument ia, 3:AlreadyExists exist)
+
+  /**
+   * Delete all cells that match the passed row and column.
+   */
+  void deleteAll(
+    /** name of table */
+    1:Text tableName,
+
+    /** Row to update */
+    2:Text row,
+
+    /** name of column whose value is to be deleted */
+    3:Text column,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Completely delete the row's cells.
+   */
+  void deleteAllRow(
+    /** name of table */
+    1:Text tableName,
+
+    /** key of the row to be completely deleted. */
+    2:Text row
+
+    /** Delete attributes */
+    3:map<Text, Text> attributes,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Completely delete the row's cells marked with a timestamp
+   * equal-to or older than the passed timestamp.
+   */
+  void deleteAllRowTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** key of the row to be completely deleted. */
+    2:Text row,
+
+    /** timestamp */
+    3:i64 timestamp,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Delete all cells that match the passed row and column and whose
+   * timestamp is equal-to or older than the passed timestamp.
+   */
+  void deleteAllTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** Row to update */
+    2:Text row,
+
+    /** name of column whose value is to be deleted */
+    3:Text column,
+
+    /** timestamp */
+    4:i64 timestamp,
+
+    5:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Deletes a table
+   *
+   * @throws IOError if table doesn't exist on server or there was some other
+   * problem
+   */
+  void deleteTable(
+    /** name of table to delete */
+    1:Text tableName
+  ) throws (1:IOError io)
+
+  /**
+   * Disables a table (takes it off-line) If it is being served, the master
+   * will tell the servers to stop serving it.
+   */
+  void disableTable(
+    /** name of the table */
+    1:Bytes tableName
+  ) throws (1:IOError io)
+
+  /**
+   * Brings a table on-line (enables it)
+   */
+  void enableTable(
+    /** name of the table */
+    1:Bytes tableName
+  ) throws (1:IOError io)
+
+  /**
+   * Flush the given region if lastFlushTime < ifOlderThanTS. Only supported
+   * in the Thrift server embedded in the regionserver.
+   */
+  void flushRegion(1:Text regionName, 2:i64 ifOlderThanTS) throws (1:IOError io)
+
+  /**
+   * Get a single TCell for the specified table, row, and column at the
+   * latest timestamp. Returns an empty list if no such value exists.
+   *
+   * @return value for specified row/column
+   */
+  list<TCell> get(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** column name */
+    3:Text column,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * List all the column families associated with a table.
+   *
+   * @return list of column family descriptors
+   */
+  map<Text,ColumnDescriptor> getColumnDescriptors (
+    /** table name */
+    1:Text tableName
+  ) throws (1:IOError io)
+
+  /**
+   * Gets last flush time (in milliseconds) for all regions on the server.
+   * Only supported in the Thrift server embedded in the regionserver.
+   *
+   * @return a map of regionName to the last flush time for the region
+   */
+  map<Text,i64> getLastFlushTimes()
+
+  /**
+   * Gets the current time (in milliseconds) at the region server. Only
+   * supported in the Thrift server embedded in the regionserver.
+   *
+   * @return time in milliseconds at the regionserver.
+   */
+  i64 getCurrentTimeMillis()
+
+  /**
+   * Get the regininfo for the specified row. It scans
+   * the metatable to find region's start and end keys.
+   *
+   * @return value for specified row/column
+   */
+  TRegionInfo getRegionInfo(
+    /** row key */
+    1:Text row,
+
+  ) throws (1:IOError io)
+
+  /**
+   * Get all the data for the specified table and row at the latest
+   * timestamp. Returns an empty list if the row does not exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRow(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    3:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get all the data for the specified table and row at the specified
+   * timestamp. Returns an empty list if the row does not exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowTs(
+    /** name of the table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** timestamp */
+    3:i64 timestamp,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get the columns with the specified prefix for the specified table and
+   * row at the latest timestamp. Returns an empty list if the row does not
+   * exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowWithColumnPrefix(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /**
+     * column prefix, null for all columns
+     * family name can be specified as <family>:<qualifier prefix>
+     * If only <qualifier prefix> provided then all families are
+     * searched
+     */
+    3:Text prefix,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get the columns with the specified prefix for the specified table and
+   * row at the specified timestamp. Returns an empty list if the row does not
+   * exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowWithColumnPrefixTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /**
+     * column prefix, null for all columns
+     * family name can be specified as <family>:<qualifier prefix>
+     * If only <qualifier prefix> provided then all families are
+     * searched
+     */
+    3:Text prefix
+
+    4:i64 timestamp,
+
+    5:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get the specified columns for the specified table and row at the latest
+   * timestamp. Returns an empty list if the row does not exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowWithColumns(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** List of columns to return, null for all columns */
+    3:list<Text> columns,
+
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get the specified columns for the specified table and row at the specified
+   * timestamp. Returns an empty list if the row does not exist.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowWithColumnsTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** List of columns to return, null for all columns */
+    3:list<Text> columns,
+    4:i64 timestamp,
+    5:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get multiple rows with the same columns or timestamps for all.
+   * This returns latest entries, all columns and cfs.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRows(
+    1:Text tableName,
+    2:list<Text> rows,
+    3:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get multiple rows only up to a certain timestamp
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowsTs(
+    1:Text tableName,
+    2:list<Text> rows,
+    3:i64 timestamp,
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get multiple rows only with particular cf:qualifier pairs on all rows
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowsWithColumns(
+    1:Text tableName,
+    2:list<Text> rows,
+    3:list<Text> families,
+    4:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get multiple rows only with particular cf:qualifier pairs on all rows
+   * and only up to a particular timestamp.
+   *
+   * @return TRowResult containing the row and map of columns to TCells
+   */
+  list<TRowResult> getRowsWithColumnsTs(
+    1:Text tableName,
+    2:list<Text> rows,
+    3:list<Text> families
+    4:i64 timestamp,
+    5:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * List all the userspace tables.
+   *
+   * @return returns a list of names
+   */
+  list<Text> getTableNames()
+    throws (1:IOError io)
+
+  /**
+   * List the regions associated with a table.
+   *
+   * @return list of region descriptors
+   */
+  list<TRegionInfo> getTableRegions(
+    /** table name */
+    1:Text tableName)
+    throws (1:IOError io)
+
+  /**
+   * Get the specified number of versions for the specified table,
+   * row, and column.
+   *
+   * @return list of cells for specified row/column
+   */
+  list<TCell> getVer(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** column name */
+    3:Text column,
+
+    /** number of versions to retrieve */
+    4:i32 numVersions,
+
+    5:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * Get the specified number of versions for the specified table,
+   * row, and column.  Only versions less than or equal to the specified
+   * timestamp will be returned.
+   *
+   * @return list of cells for specified row/column
+   */
+  list<TCell> getVerTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** column name */
+    3:Text column,
+
+    /** timestamp */
+    4:i64 timestamp,
+
+    /** number of versions to retrieve */
+    5:i32 numVersions,
+
+    6:Text regionName
+  ) throws (1:IOError io)
+
+  /**
+   * @return true if table is on-line
+   */
+  bool isTableEnabled(
+    /** name of the table to check */
+    1:Bytes tableName
+  ) throws (1:IOError io)
+
+  void majorCompact(1:Bytes tableNameOrRegionName)
+    throws (1:IOError io)
+
+  /**
+   * Apply a batch of puts for the target region.
+   * It assumes all the BatchMuations are Put operations
+   * and ignores the isDelete field in Mutation.
+   */
+  void multiPut(
+    /** name of tableName */
+    1:Text tableName,
+
+    /** list of Put */
+    2:list<BatchMutation> rowBatches,
+
+    /** name of the region */
+    3:Text regionName
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Apply a series of mutations (updates/deletes) to a row in a
+   * single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  Default current timestamp is used, and
+   * all entries will have an identical timestamp.
+   */
+  void mutateRow(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** list of mutation commands */
+    3:list<Mutation> mutations,
+
+    /** Put attributes */
+    4:map<Text, Text> attributes,
+
+    7:Text regionName
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Apply a series of mutations (updates/deletes) to a row in a
+   * single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  The specified timestamp is used, and
+   * all entries will have an identical timestamp.
+   */
+  void mutateRowTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** row key */
+    2:Text row,
+
+    /** list of mutation commands */
+    3:list<Mutation> mutations,
+
+    /** timestamp */
+    4:i64 timestamp,
+
+    /** Put attributes */
+    5:map<Text, Text> attributes,
+
+    6:Text regionName
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Apply a series of batches (each a series of mutations on a single row)
+   * in a single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  Default current timestamp is used, and
+   * all entries will have an identical timestamp.
+   */
+  void mutateRows(
+    /** name of table */
+    1:Text tableName,
+
+    /** list of row batches */
+    2:list<BatchMutation> rowBatches,
+
+    /** Put attributes */
+    3:map<Text, Text> attributes,
+
+    4:Text regionName
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Async (oneway) versions of some of above.
+   * @TODO only the two multi mutates have been done yet
+   * @TODO currently no way for client to know success
+   *   or failure of the call!!
+   */
+
+  /**
+   * Asynchronous multi-row mutation call
+   * See mutateRows()
+   */
+  oneway void mutateRowsAsync(
+    /** name of table */
+    1:Text tableName,
+
+    /** list of row batches */
+    2:list<BatchMutation> rowBatches
+  )
+
+  /**
+   * Apply a series of batches (each a series of mutations on a single row)
+   * in a single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  The specified timestamp is used, and
+   * all entries will have an identical timestamp.
+   */
+  void mutateRowsTs(
+    /** name of table */
+    1:Text tableName,
+
+    /** list of row batches */
+    2:list<BatchMutation> rowBatches,
+
+    /** timestamp */
+    3:i64 timestamp,
+
+    /** Put attributes */
+    4:map<Text, Text> attributes,
+
+    5:Text regionName
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Asynchronous multi-row mutation call with ts
+   * See mutateRowsTs()
+   *
+   * The specified timestamp is used, and
+   * all entries will have an identical timestamp.
+   */
+  oneway void mutateRowsTsAsync(
+    /** name of table */
+    1:Text tableName,
+
+    /** list of row batches */
+    2:list<BatchMutation> rowBatches,
+
+    /** timestamp */
+    3:i64 timestamp
+  )
+
+  /**
+   * Closes the server-state associated with an open scanner.
+   *
+   * @throws IllegalArgument if ScannerID is invalid
+   */
+  void scannerClose(
+    /** id of a scanner returned by scannerOpen */
+    1:ScannerID id
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Returns the scanner's current row value and advances to the next
+   * row in the table.  When there are no more rows in the table, or a key
+   * greater-than-or-equal-to the scanner's specified stopRow is reached,
+   * an empty list is returned.
+   *
+   * @return a TRowResult containing the current row and
+   * a map of the columns to TCells.
+   *
+   * @throws IllegalArgument if ScannerID is invalid
+   *
+   * @throws NotFound when the scanner reaches the end
+   */
+  list<TRowResult> scannerGet(
+    /** id of a scanner returned by scannerOpen */
+    1:ScannerID id
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Returns, starting at the scanner's current row value nbRows worth of
+   * rows and advances to the next row in the table.  When there are no more
+   * rows in the table, or a key greater-than-or-equal-to the scanner's
+   * specified stopRow is reached,  an empty list is returned.
+   *
+   * @return a TRowResult containing the current row and
+   * a map of the columns to TCells.
+   *
+   * @throws IllegalArgument if ScannerID is invalid
+   *
+   * @throws NotFound when the scanner reaches the end
+   */
+  list<TRowResult> scannerGetList(
+    /** id of a scanner returned by scannerOpen */
+    1:ScannerID id,
+
+    /** number of results to return */
+    2:i32 nbRows
+  ) throws (1:IOError io, 2:IllegalArgument ia)
+
+  /**
+   * Get a scanner on the current table starting at the specified row and
+   * ending at the last row in the table.  Return the specified columns.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpen(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned. It's also possible
+     * to pass a regex in the column qualifier.
+     */
+    3:list<Text> columns
+
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting at the specified row and
+   * ending at the last row in the table.  Return the specified columns.
+   * Only values with the specified timestamp are returned.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenTs(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned. It's also possible
+     * to pass a regex in the column qualifier.
+     */
+    3:list<Text> columns,
+
+    /** timestamp */
+    4:i64 timestamp
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting at the first row and
+   * ending at the last row in the table.  Return the specified columns.
+   *
+   * Return the specified columns that pass the filter constructed
+   * by the filterString
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithFilterString(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Filter string
+     */
+    2:Text filterString
+
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting at the first row and
+   * ending at the last row in the table.
+   *
+   * Return the specified columns that pass the filter constructed
+   * by the filterString
+   *
+   * The timestamp of the keyvalue must also be within the specified timestamp
+   * No other columns will be returned.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithFilterStringTs(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Filter string
+     */
+    2:Text filterString
+
+   /** timestamp */
+    3:i64 timestamp
+
+  ) throws (1:IOError io)
+
+  /**
+   * Open a scanner for a given prefix.  That is all rows will have the specified
+   * prefix. No other rows will be returned.
+   *
+   * @return scanner id to use with other scanner calls
+   */
+  ScannerID scannerOpenWithPrefix(
+    /** name of table */
+    1:Text tableName,
+
+   /**
+    * The prefix (and thus start row) of the keys you want
+    * Only those rows with the specified prefix will be returned
+    */
+    2:Text startAndPrefix,
+
+   /**
+    * columns to scan. If column name is a column family, all
+    * columns of the specified column family are returned. It's also possible
+    * to pass a regex in the column qualifier.
+    */
+    3:list<Text> columns
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table, using the Scan instance
+   * for the scan parameters.
+   */
+  ScannerID scannerOpenWithScan(
+    /** name of table */
+    1:Text tableName,
+
+    /** Scan instance */
+    2:TScan scan
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.  ending at the last row in the table.  Return the
+   * specified columns.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStop(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * Row to stop scanning on. This row is *not* included in the
+     * scanner's results
+     * Send "" (empty string) to stop at the last row.
+     */
+    3:Text stopRow,
+
+    /**
+     * columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned. It's also possible
+     * to pass a regex in the column qualifier.
+     */
+    4:list<Text> columns
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.
+   *
+   * Return the specified columns that pass the filter constructed
+   * by the filterString
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStopAndFilterString(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * row to stop scanning on. This row is *not* included in the
+     * scanner's results.
+     * Send "" (empty string) to end at the last row.
+     */
+    3:Text stopRow,
+
+    /**
+     * Filter string
+     */
+    4:Text filterString
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.
+   *
+   * Return the specified columns that pass the filter constructed
+   * by the filterString
+   *
+   * The timestamp of the keyvalue must also be within the specified timestamp
+   * No other columns will be returned.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStopAndFilterStringTs(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * row to stop scanning on. This row is *not* included in the
+     * scanner's results.
+     * Send "" (empty string) to end at the last row.
+     */
+    3:Text stopRow,
+
+    /**
+     * Filter string
+     */
+    4:Text filterString
+
+   /** timestamp */
+    6:i64 timestamp
+  ) throws (1:IOError io)
+
+  /**
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.  ending at the last row in the table.  Return the
+   * specified columns.  Only values with the specified timestamp are
+   * returned.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStopTs(
+    /** name of table */
+    1:Text tableName,
+
+    /**
+     * Starting row in table to scan.
+     * Send "" (empty string) to start at the first row.
+     */
+    2:Text startRow,
+
+    /**
+     * Row to stop scanning on. This row is *not* included in the
+     * scanner's results
+     * Send "" (empty string) to stop at the last row.
+     */
+    3:Text stopRow,
+
+    /**
+     * columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned. It's also possible
+     * to pass a regex in the column qualifier.
+     */
+    4:list<Text> columns,
+
+    /** timestamp */
+    5:i64 timestamp
+  ) throws (1:IOError io)
+}

Added: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh (added)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh Wed Mar 12 21:17:13 2014
@@ -0,0 +1,11 @@
+# Generate Thrift IDL files for Swift annotated classes
+if [ $# -eq 0 ]
+then
+  echo "Usage"
+  echo "-----"
+  echo "./gen_thrift_from_swift <space separated list of HBase, Hadoop and swift2thrift jar paths>"
+  exit
+fi
+export CLASSPATH=$CLASSPATH:$1:$2:$3
+
+java com.facebook.swift.generator.swift2thrift.Main -allow_multiple_packages org.apache.hadoop.hbase org.apache.hadoop.hbase.KeyValue org.apache.hadoop.hbase.client.Put org.apache.hadoop.hbase.io.TimeRange org.apache.hadoop.hbase.filter.TFilter org.apache.hadoop.hbase.client.Get org.apache.hadoop.hbase.client.MultiPut org.apache.hadoop.hbase.client.Delete org.apache.hadoop.hbase.client.Scan org.apache.hadoop.hbase.HColumnDescriptor org.apache.hadoop.hbase.HTableDescriptor org.apache.hadoop.hbase.HRegionInfo org.apache.hadoop.hbase.client.MultiPutResponse org.apache.hadoop.hbase.client.Result org.apache.hadoop.hbase.HServerAddress 'org.apache.hadoop.hbase.HServerLoad$RegionLoad' org.apache.hadoop.hbase.HServerLoad org.apache.hadoop.hbase.HServerInfo org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException org.apache.hadoop.hbase.client.MultiAction 'org.apache.hadoop.hbase.client.IntegerOrResultOrException$Type' org.apache.hadoop.hbase.client.IntegerOrResultOrException org.a
 pache.hadoop.hbase.client.TMultiResponse org.apache.hadoop.hbase.client.TRowMutations org.apache.hadoop.hbase.master.AssignmentPlan org.apache.hadoop.hbase.client.RowLock 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$HFileStat' 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$Bucket' org.apache.hadoop.hbase.ipc.ThriftHRegionInterface -out HBase.thrift

Propchange: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh
------------------------------------------------------------------------------
    svn:executable = *

Modified: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/thrift.sh
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/thrift.sh?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/thrift.sh (original)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/thrift.sh Wed Mar 12 21:17:13 2014
@@ -1,7 +1,7 @@
 #!/bin/bash
 
 LANG="java"
-SOURCE="Hbase.thrift"
+SOURCE="LegacyHBase.thrift"
 THRIFT="thrift-apache"
 OUTPUT="../../../../../../java"
 PKG_PATH="org/apache/hadoop/hbase/thrift/generated"

Modified: hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/hbase/admin.rb Wed Mar 12 21:17:13 2014
@@ -305,8 +305,8 @@ module Hbase
            sleep 1
          end while status != nil && status.getFirst() != 0
          puts "Done."
-       end
-
+       end    
+       
     #----------------------------------------------------------------------------------------------
     # Change table structure or table options
     def alter(table_name, wait = true, *args)
@@ -376,7 +376,7 @@ module Hbase
         # Change table attributes
         if method == "table_att"
           # Table should be disabled
-          raise(ArgumentError, "Table #{table_name} is enabled. Disable it first before altering.") if enabled?(table_name)
+          raise(ArgumentError, "Table #{table_name} is enabled. Disable it first before altering.") if enabled?(table_name)          
           htd.setMaxFileSize(JLong.valueOf(arg[MAX_FILESIZE])) if arg[MAX_FILESIZE]
           htd.setReadOnly(JBoolean.valueOf(arg[READONLY])) if arg[READONLY]
           htd.setMemStoreFlushSize(JLong.valueOf(arg[MEMSTORE_FLUSHSIZE])) if arg[MEMSTORE_FLUSHSIZE]
@@ -401,7 +401,7 @@ module Hbase
       if wait == true
         puts "Updating all regions with the new schema..."
         alter_status(table_name)
-      end
+      end            
     end
 
     def status(format)

Modified: hbase/branches/0.89-fb/src/main/ruby/shell.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell.rb Wed Mar 12 21:17:13 2014
@@ -234,7 +234,7 @@ Shell.load_command_group(
     list
     locate_regionservers
     show_filters
-    alter_status
+    alter_status 
     alter_async
   ]
 )

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_async.rb Wed Mar 12 21:17:13 2014
@@ -23,9 +23,9 @@ module Shell
     class AlterAsync < Command
       def help
         return <<-EOF
-          Alter column family schema, does not wait for all regions to receive the
-          schema changes. Pass table name and a dictionary specifying new column
-          family schema. Dictionaries are described on the main help command output.
+          Alter column family schema, does not wait for all regions to receive the 
+          schema changes. Pass table name and a dictionary specifying new column 
+          family schema. Dictionaries are described on the main help command output. 
           Dictionary must include name of column family to alter. For example,
 
           To change or add the 'f1' column family in table 't1' from defaults

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_status.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_status.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_status.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/alter_status.rb Wed Mar 12 21:17:13 2014
@@ -25,7 +25,7 @@ module Shell
         return <<-EOF
 Get the status of the alter command. Indicates the number of regions of the
 table that have received the updated schema
-Pass table name.
+Pass table name. 
 
 hbase> alter_status 't1'
 EOF

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/scan.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/scan.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/scan.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/scan.rb Wed Mar 12 21:17:13 2014
@@ -33,7 +33,7 @@ members of a column family, leave the qu
 
 The filter can be specified in two ways:
 1. Using a filterString - more information on this is available at:
-http://hbase.apache.org/book.html#thrift.filter-language
+https://our.intern.facebook.com/intern/wiki/index.php/HBase/Filter_Language
 2. Using the entire package name of the filter.
 
 Examples:

Modified: hbase/branches/0.89-fb/src/main/ruby/shell/commands/show_filters.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/ruby/shell/commands/show_filters.rb?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/ruby/shell/commands/show_filters.rb (original)
+++ hbase/branches/0.89-fb/src/main/ruby/shell/commands/show_filters.rb Wed Mar 12 21:17:13 2014
@@ -34,8 +34,8 @@ EOF
       def command( )
         now = Time.now
         formatter.row(["Documentation on filters mentioned below can " +
-                       "be found at: http://hbase.apache.org/" +
-                       "book.html#thrift.filter-language"])
+                       "be found at: https://our.intern.facebook.com/intern/" +
+                       "wiki/index.php/HBase/Filter_Language"])
 
         parseFilter = ParseFilter.new
         supportedFilters = parseFilter.getSupportedFilters

Modified: hbase/branches/0.89-fb/src/saveVersion.sh
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/saveVersion.sh?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/saveVersion.sh (original)
+++ hbase/branches/0.89-fb/src/saveVersion.sh Wed Mar 12 21:17:13 2014
@@ -36,8 +36,8 @@ elif [ -d .git ]; then
   url="git://${hostname}${cwd}"
 else
 	which git  > /dev/null
-	if [ $? == 0 ]; then
-   revision=`git log -1 | grep "git-svn-id:" | awk '{print $2}' | cut -f7 -d\/`
+	if [ $? == 0 ]; then 
+		revision=`git log -1 | grep "git-svn-id:" | awk '{print $2}' | cut -f7 -d\/`
 		hostname=`hostname`
 		if [ "$revision" == "" ]; then
 			git_revision=`git log -1 --pretty=format:"%H"`

Modified: hbase/branches/0.89-fb/src/site/resources/css/site.css
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/resources/css/site.css?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/resources/css/site.css (original)
+++ hbase/branches/0.89-fb/src/site/resources/css/site.css Wed Mar 12 21:17:13 2014
@@ -105,3 +105,5 @@ h4 {
   margin-left: 1.5em;
 }
 */
+
+

Modified: hbase/branches/0.89-fb/src/site/site.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/site.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/site.xml (original)
+++ hbase/branches/0.89-fb/src/site/site.xml Wed Mar 12 21:17:13 2014
@@ -44,3 +44,4 @@
       <artifactId>maven-stylus-skin</artifactId>
     </skin>
 </project>
+

Modified: hbase/branches/0.89-fb/src/site/xdoc/acid-semantics.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/acid-semantics.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/acid-semantics.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/acid-semantics.xml Wed Mar 12 21:17:13 2014
@@ -22,7 +22,7 @@
   xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
   <properties>
-    <title>
+    <title> 
       HBase ACID Properties
     </title>
   </properties>

Modified: hbase/branches/0.89-fb/src/site/xdoc/bulk-loads.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/bulk-loads.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/bulk-loads.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/bulk-loads.xml Wed Mar 12 21:17:13 2014
@@ -18,7 +18,7 @@
   xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
   <properties>
-    <title>
+    <title> 
       Bulk Loads in HBase
     </title>
   </properties>

Modified: hbase/branches/0.89-fb/src/site/xdoc/index.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/index.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/index.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/index.xml Wed Mar 12 21:17:13 2014
@@ -20,7 +20,7 @@
   <properties>
     <title>HBase Home</title>
   </properties>
-
+  
   <body>
     <section name="This is Apache HBase">
     <p>
@@ -54,5 +54,5 @@ HBase includes:
       <p><a href="old_news.html">...</a></p>
     </section>
   </body>
-
+  
 </document>

Modified: hbase/branches/0.89-fb/src/site/xdoc/metrics.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/metrics.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/metrics.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/metrics.xml Wed Mar 12 21:17:13 2014
@@ -18,7 +18,7 @@
   xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
   <properties>
-    <title>
+    <title> 
       HBase Metrics
     </title>
   </properties>
@@ -61,17 +61,17 @@
 
     <section name="Using with JMX">
       <p>
-      In addition to the standard output contexts supported by the Hadoop
-      metrics package, you can also export HBase metrics via Java Management
-      Extensions (JMX).  This will allow viewing HBase stats in JConsole or
+      In addition to the standard output contexts supported by the Hadoop 
+      metrics package, you can also export HBase metrics via Java Management 
+      Extensions (JMX).  This will allow viewing HBase stats in JConsole or 
       any other JMX client.
       </p>
       <section name="Enable HBase stats collection">
       <p>
-      To enable JMX support in HBase, first edit
-      <code>$HBASE_HOME/conf/hadoop-metrics.properties</code> to support
-      metrics refreshing. (If you've already configured
-      <code>hadoop-metrics.properties</code> for another output context,
+      To enable JMX support in HBase, first edit 
+      <code>$HBASE_HOME/conf/hadoop-metrics.properties</code> to support 
+      metrics refreshing. (If you've already configured 
+      <code>hadoop-metrics.properties</code> for another output context, 
       you can skip this step).
       </p>
       <source>
@@ -90,11 +90,11 @@ rpc.period=60
       </section>
       <section name="Setup JMX remote access">
       <p>
-      For remote access, you will need to configure JMX remote passwords
+      For remote access, you will need to configure JMX remote passwords 
       and access profiles.  Create the files:
       </p>
       <dl>
-        <dt><code>$HBASE_HOME/conf/jmxremote.passwd</code> (set permissions
+        <dt><code>$HBASE_HOME/conf/jmxremote.passwd</code> (set permissions 
         to 600)</dt>
         <dd>
         <source>
@@ -102,7 +102,7 @@ monitorRole monitorpass
 controlRole controlpass
         </source>
         </dd>
-
+        
         <dt><code>$HBASE_HOME/conf/jmxremote.access</code></dt>
         <dd>
         <source>
@@ -115,7 +115,7 @@ controlRole readwrite
       <section name="Configure JMX in HBase startup">
       <p>
       Finally, edit the <code>$HBASE_HOME/conf/hbase-env.sh</code>
-      script to add JMX support:
+      script to add JMX support: 
       </p>
       <dl>
         <dt><code>$HBASE_HOME/conf/hbase-env.sh</code></dt>
@@ -132,9 +132,9 @@ export HBASE_REGIONSERVER_OPTS="$HBASE_J
         </dd>
       </dl>
       <p>
-      After restarting the processes you want to monitor, you should now be
-      able to run JConsole (included with the JDK since JDK 5.0) to view
-      the statistics via JMX.  HBase MBeans are exported under the
+      After restarting the processes you want to monitor, you should now be 
+      able to run JConsole (included with the JDK since JDK 5.0) to view 
+      the statistics via JMX.  HBase MBeans are exported under the 
       <strong><code>hadoop</code></strong> domain in JMX.
       </p>
       </section>

Modified: hbase/branches/0.89-fb/src/site/xdoc/old_news.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/old_news.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/old_news.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/old_news.xml Wed Mar 12 21:17:13 2014
@@ -22,7 +22,7 @@
   xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
   <properties>
-    <title>
+    <title> 
       Old News
     </title>
   </properties>
@@ -33,7 +33,7 @@
       <p><a href="http://www.meetup.com/hbaseusergroup/calendar/12689351/">HBase User Group Meeting, HUG9</a> hosted by Mozilla, March 10th, 2010</p>
       <p>Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/12241393/">HBase User Group Meeting, HUG8</a>, January 27th, 2010 at StumbleUpon in SF</p>
       <p>September 8th, 2010: HBase 0.20.0 is faster, stronger, slimmer, and sweeter tasting than any previous HBase release.  Get it off the <a href="releases.html">Releases</a> page.</p>
-      <p><a href="http://dev.us.apachecon.com/c/acus2009/">ApacheCon</a> in Oakland: November 2-6th, 2009:
+      <p><a href="http://dev.us.apachecon.com/c/acus2009/">ApacheCon</a> in Oakland: November 2-6th, 2009: 
       The Apache Foundation will be celebrating its 10th anniversary in beautiful Oakland by the Bay. Lots of good talks and meetups including an HBase presentation by a couple of the lads.</p>
       <p>HBase at Hadoop World in NYC: October 2nd, 2009: A few of us will be talking on Practical HBase out east at <a href="http://www.cloudera.com/hadoop-world-nyc">Hadoop World: NYC</a>.</p>
       <p>HUG7 and HBase Hackathon: August 7th-9th, 2009 at StumbleUpon in SF: Sign up for the <a href="http://www.meetup.com/hbaseusergroup/calendar/10950511/">HBase User Group Meeting, HUG7</a> or for the <a href="http://www.meetup.com/hackathon/calendar/10951718/">Hackathon</a> or for both (all are welcome!).</p>

Modified: hbase/branches/0.89-fb/src/site/xdoc/pseudo-distributed.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/pseudo-distributed.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/pseudo-distributed.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/pseudo-distributed.xml Wed Mar 12 21:17:13 2014
@@ -22,13 +22,13 @@
   xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
   <properties>
-    <title>
+    <title> 
 Running HBase in pseudo-distributed mode
     </title>
   </properties>
 
   <body>
-      <p>This document augments what is described in the HBase 'Getting Started' in the
+      <p>This document augments what is described in the HBase 'Getting Started' in the 
  <a href="http://hbase.apache.org/docs/current/api/overview-summary.html#distributed">Distributed Operation: Pseudo- and Fully-distributed modes</a> section.
  In particular it describes scripts that allow you start extra masters and regionservers when running in pseudo-distributed mode.
  </p>
@@ -74,3 +74,4 @@ Running HBase in pseudo-distributed mode
 </body>
 
 </document>
+

Modified: hbase/branches/0.89-fb/src/site/xdoc/replication.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/site/xdoc/replication.xml?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/site/xdoc/replication.xml (original)
+++ hbase/branches/0.89-fb/src/site/xdoc/replication.xml Wed Mar 12 21:17:13 2014
@@ -63,7 +63,7 @@
         and must be kept in HDFS as long as they are needed to replicate data
         to any slave cluster. Each RS reads from the oldest log it needs to
         replicate and keeps the current position inside ZooKeeper to simplify
-        failure recovery. That position can be different for every slave
+        failure recovery. That position can be different for every slave 
         cluster, same for the queue of HLogs to process.
       </p>
       <p>
@@ -266,7 +266,7 @@
           since ZK doesn't support the rename operation) and will delete all the
           old ones when it's done. The recovered queues' znodes will be named
           with the id of the slave cluster appended with the name of the dead
-          server.
+          server. 
         </p>
         <p>
           Once that is done, the master cluster RS will create one new source thread per

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MultithreadedTestUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MultithreadedTestUtil.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MultithreadedTestUtil.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/MultithreadedTestUtil.java Wed Mar 12 21:17:13 2014
@@ -126,7 +126,7 @@ public abstract class MultithreadedTestU
       this.stopped = true;
     }
   }
-
+  
   /**
    * A test thread that performs a repeating operation.
    */
@@ -134,13 +134,13 @@ public abstract class MultithreadedTestU
     public RepeatingTestThread(TestContext ctx) {
       super(ctx);
     }
-
+    
     public final void doWork() throws Exception {
       while (ctx.shouldRun() && !stopped) {
         doAnAction();
       }
     }
-
+    
     public abstract void doAnAction() throws Exception;
   }
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java Wed Mar 12 21:17:13 2014
@@ -49,16 +49,16 @@ public class TestGlobalMemStoreSize exte
   final byte[] FIVE_HUNDRED_KBYTES;
 
   final byte [] FAMILY_NAME = Bytes.toBytes("col");
-
+  
   private static int regionServerNum =4;
   private static int regionNum = 16;
   // total region num = region num + root and meta region
   private static int totalRegionNum = regionNum +2;
-
+  
   /** constructor */
   public TestGlobalMemStoreSize() {
     super(regionServerNum);
-
+    
     FIVE_HUNDRED_KBYTES = new byte[500 * 1024];
     for (int i = 0; i < 500 * 1024; i++) {
       FIVE_HUNDRED_KBYTES[i] = 'x';
@@ -81,7 +81,7 @@ public class TestGlobalMemStoreSize exte
     }
     startKeys.add(null);
     LOG.debug(startKeys.size() + " start keys generated");
-
+    
     List<HRegion> regions = new ArrayList<HRegion>();
     for (int i = 0; i < regionNum; i++) {
       regions.add(createAregion(startKeys.get(i), startKeys.get(i+1)));
@@ -96,7 +96,7 @@ public class TestGlobalMemStoreSize exte
     }
     closeRootAndMeta();
   }
-
+  
   /**
    * Test the global mem store size in the region server is equal to sum of each
    * region's mem store size
@@ -106,7 +106,7 @@ public class TestGlobalMemStoreSize exte
   public void testGlobalMemStore() throws IOException {
     waitForAllRegionsAssigned();
     assertEquals(getOnlineRegionServers().size(), regionServerNum);
-
+    
     int totalRegionNum = 0;
     for (HRegionServer server : getOnlineRegionServers()) {
       long globalMemStoreSize = 0;
@@ -117,7 +117,7 @@ public class TestGlobalMemStoreSize exte
       assertEquals(server.getGlobalMemstoreSize().get(),globalMemStoreSize);
     }
     assertEquals(totalRegionNum,totalRegionNum);
-
+    
     for (HRegionServer server : getOnlineRegionServers()) {
       for(HRegion region : server.getOnlineRegions()) {
         region.flushcache();
@@ -135,7 +135,7 @@ public class TestGlobalMemStoreSize exte
     }
     return total;
   }
-
+  
   private List<HRegionServer> getOnlineRegionServers() {
     List<HRegionServer> list = new ArrayList<HRegionServer>();
     for (JVMClusterUtil.RegionServerThread rst : cluster.getRegionServerThreads()) {
@@ -174,3 +174,4 @@ public class TestGlobalMemStoreSize exte
     return region;
   }
 }
+

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerAddress.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerAddress.java?rev=1576909&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerAddress.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestHServerAddress.java Wed Mar 12 21:17:13 2014
@@ -0,0 +1,49 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import static org.junit.Assert.*;
+
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.junit.Test;
+
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
+
+public class TestHServerAddress {
+
+  /**
+   * Tests if HServerAddress was correctly serialized and deserialized
+   * @throws Exception
+   */
+  @Test
+  public void testSerializeDeserialize() throws Exception {
+    ThriftCodec<HServerAddress> codec = new ThriftCodecManager()
+        .getCodec(HServerAddress.class);
+    TMemoryBuffer transport = new TMemoryBuffer(1000 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+    HServerAddress serverAddress = new HServerAddress("127.0.0.1", 60020);
+    codec.write(serverAddress, protocol);
+    HServerAddress serverAddrCopy = codec.read(protocol);
+    assertEquals(serverAddress, serverAddrCopy);
+  }
+
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java Wed Mar 12 21:17:13 2014
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.util.Set;
 import java.util.TreeSet;
 
+import com.facebook.swift.codec.ThriftCodec;
+import com.facebook.swift.codec.ThriftCodecManager;
 import junit.framework.TestCase;
 
 import org.apache.commons.logging.Log;
@@ -30,6 +32,8 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
 
 public class TestKeyValue extends TestCase {
   private final Log LOG = LogFactory.getLog(this.getClass().getName());
@@ -364,4 +368,16 @@ public class TestKeyValue extends TestCa
         kv.toString().replaceAll("=[0-9]+$", "=0"));
   }
 
+  public void testSwiftSerDe() throws Exception {
+    ThriftCodec<KeyValue> codec =
+        new ThriftCodecManager().getCodec(KeyValue.class);
+    TMemoryBuffer transport = new TMemoryBuffer(10 * 1024);
+    TCompactProtocol protocol = new TCompactProtocol(transport);
+    KeyValue kv = new KeyValue(Bytes.toBytes("myRow"), Bytes.toBytes("myCF"),
+        Bytes.toBytes("myQualifier"), 12345L, Bytes.toBytes("myValue"));
+    codec.write(kv, protocol);
+    KeyValue kvCopy = codec.read(protocol);
+    assertEquals(kv, kvCopy);
+    assertEquals(kv.toString(), kvCopy.toString());
+  }
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestMultiParallelPut.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestMultiParallelPut.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestMultiParallelPut.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestMultiParallelPut.java Wed Mar 12 21:17:13 2014
@@ -101,10 +101,10 @@ public class TestMultiParallelPut extend
           Bytes.compareTo(VALUE,
               r.getValue(BYTES_FAMILY, QUALIFIER)));
     }
-
+    
     // waiting for load balancer running
     Thread.sleep(MiniHBaseCluster.WAIT_FOR_LOADBALANCER);
-
+    
     HBaseAdmin admin = new HBaseAdmin(conf);
     ClusterStatus cs = admin.getClusterStatus();
     int expectedServerCount = 2;

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestPerColumnFamilyFlush.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestPerColumnFamilyFlush.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestPerColumnFamilyFlush.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestPerColumnFamilyFlush.java Wed Mar 12 21:17:13 2014
@@ -20,8 +20,13 @@
 
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
 import junit.framework.Assert;
 import junit.framework.TestCase;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -36,10 +41,6 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-
 /**
  * This test verifies the correctness of the Per Column Family flushing strategy
  */
@@ -60,9 +61,8 @@ public class TestPerColumnFamilyFlush ex
   public static final byte[] FAMILY2 = families[1];
   public static final byte[] FAMILY3 = families[2];
 
-  private void initHRegion (String callingMethod,
-                            HBaseConfiguration conf)
-          throws IOException {
+  private void initHRegion (String callingMethod, Configuration conf)
+      throws IOException {
     HTableDescriptor htd = new HTableDescriptor(TABLENAME);
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
@@ -105,12 +105,12 @@ public class TestPerColumnFamilyFlush ex
   @Test
   public void testSelectiveFlushWhenEnabled() throws IOException {
     // Set up the configuration
-    HBaseConfiguration conf = new HBaseConfiguration();
+    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200*1024);
     conf.setBoolean(HConstants.HREGION_MEMSTORE_PER_COLUMN_FAMILY_FLUSH, true);
     conf.setLong(HConstants.HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE, 100*1024);
 
-    // Intialize the HRegion
+    // Initialize the HRegion
     initHRegion(getName(), conf);
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
     for (int i = 1; i <= 1200; i++) {
@@ -247,15 +247,123 @@ public class TestPerColumnFamilyFlush ex
     Assert.assertEquals(0, region.getMemstoreSize().get());
   }
 
+  public void testSelectiveFlushWithThreshold(long t) throws IOException {
+    /*         t->|
+     * Phase 1    |     |     |
+     *            +-----+-----+
+     *           FM1   FM2   FM3
+     */
+    // flush all
+    region.flushcache(false);
+    region.put(createPut(2, 1));
+    region.put(createPut(3, 1));
+
+
+    int cnt = 0;
+    while (region.getStore(FAMILY1).getMemStoreSize() <= t) {
+      region.put(createPut(1, cnt));
+      cnt ++;
+    }
+
+    final long sizeFamily1_1 = region.getStore(FAMILY1).getMemStoreSize();
+    final long sizeFamily2_1 = region.getStore(FAMILY2).getMemStoreSize();
+    final long sizeFamily3_1 = region.getStore(FAMILY3).getMemStoreSize();
+
+    Assert.assertTrue(sizeFamily1_1 > t);
+    Assert.assertTrue(sizeFamily2_1 > 0);
+    Assert.assertTrue(sizeFamily3_1 > 0);
+
+    /*         t->
+     * Phase 2          |     |
+     *            +-----+-----+
+     *           FM1   FM2   FM3
+     */
+    // flush, should only flush family 1
+    region.flushcache(true);
+
+    final long sizeFamily1_2 = region.getStore(FAMILY1).getMemStoreSize();
+    final long sizeFamily2_2 = region.getStore(FAMILY2).getMemStoreSize();
+    final long sizeFamily3_2 = region.getStore(FAMILY3).getMemStoreSize();
+
+    // should clear FAMILY1 only
+    Assert.assertEquals("sizeFamily1", 0, sizeFamily1_2);
+    Assert.assertEquals("sizeFamily2", sizeFamily2_1, sizeFamily2_2);
+    Assert.assertEquals("sizeFamily3", sizeFamily3_1, sizeFamily3_2);
+    /*         t->,
+     * Phase 3    |     |     |
+     *            +-----+-----+
+     *           FM1   FM2   FM3
+     */
+    // flush all
+    region.flushcache(false);
+
+    region.put(createPut(2, 1));
+    region.put(createPut(3, 1));
+    // reduce cnt to 1 to make the size less than t
+    cnt--;
+    for (int i = 0; i < cnt; i ++) {
+      region.put(createPut(1, i));
+    }
+
+    final long sizeFamily1_3 = region.getStore(FAMILY1).getMemStoreSize();
+    final long sizeFamily2_3 = region.getStore(FAMILY2).getMemStoreSize();
+    final long sizeFamily3_3 = region.getStore(FAMILY3).getMemStoreSize();
+
+    Assert.assertTrue("sizeFamily1 < t", sizeFamily1_3 < t);
+    Assert.assertEquals("sizeFamily2", sizeFamily2_1, sizeFamily2_3);
+    Assert.assertEquals("sizeFamily3", sizeFamily3_1, sizeFamily3_3);
+    /*         t->
+     * Phase 3
+     *            +-----+-----+
+     *           FM1   FM2   FM3
+     */
+    // flush, should flush all
+    region.flushcache(true);
+
+    final long sizeFamily1_4 = region.getStore(FAMILY1).getMemStoreSize();
+    final long sizeFamily2_4 = region.getStore(FAMILY2).getMemStoreSize();
+    final long sizeFamily3_4 = region.getStore(FAMILY3).getMemStoreSize();
+    Assert.assertEquals("sizeFamily1", 0, sizeFamily1_4);
+    Assert.assertEquals("sizeFamily2", 0, sizeFamily2_4);
+    Assert.assertEquals("sizeFamily3", 0, sizeFamily3_4);
+  }
+
+  @Test
+  public void testConfigueChange() throws IOException {
+    final int T_init = 100*1024;
+    final int T_1 = 200*1024;
+    final int T_2 = 50*1024;
+
+    // Set up the configuration
+    Configuration conf = HBaseConfiguration.create();
+    conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200*1024);
+    conf.setBoolean(HConstants.HREGION_MEMSTORE_PER_COLUMN_FAMILY_FLUSH, true);
+    conf.setLong(HConstants.HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE, T_init);
+
+    // Initialize the HRegion
+    initHRegion(getName(), conf);
+
+    // test for threshold of T_init
+    this.testSelectiveFlushWithThreshold(T_init);
+    // test for threshold of T_1
+    conf.setLong(HConstants.HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE, T_1);
+    HRegionServer.configurationManager.notifyAllObservers(conf);
+    this.testSelectiveFlushWithThreshold(T_1);
+    // test for threshold of T_2
+    conf.setLong(HConstants.HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE, T_2);
+    HRegionServer.configurationManager.notifyAllObservers(conf);
+    this.testSelectiveFlushWithThreshold(T_2);
+  }
+
   @Test
   public void testSelectiveFlushWhenNotEnabled() throws IOException {
     // Set up the configuration
-    HBaseConfiguration conf = new HBaseConfiguration();
+    Configuration conf = HBaseConfiguration.create();
     conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024);
     conf.setBoolean(HConstants.HREGION_MEMSTORE_PER_COLUMN_FAMILY_FLUSH, false);
     conf.setLong(HConstants.HREGION_MEMSTORE_COLUMNFAMILY_FLUSH_SIZE, 100 * 1024);
 
-    // Intialize the HRegion
+    // Initialize the HRegion
     initHRegion(getName(), conf);
     // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
     for (int i = 1; i <= 1200; i++) {

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java Wed Mar 12 21:17:13 2014
@@ -82,7 +82,7 @@ public class TestRegionRebalancing exten
     }
     startKeys.add(null);
     LOG.debug(startKeys.size() + " start keys generated");
-
+    
     List<HRegion> regions = new ArrayList<HRegion>();
     for (int i = 0; i < 60; i++) {
       regions.add(createAregion(startKeys.get(i), startKeys.get(i+1)));
@@ -103,7 +103,7 @@ public class TestRegionRebalancing exten
    * When one of the server shuts down, the avg load is 3.875.
    * When this server comes back, the avg load will be 3.64
    * Set the slot number near 0, so no server's load will large than 4.
-   * The load balance algorithm should handle this case properly.
+   * The load balance algorithm should handle this case properly. 
    */
   @Test(timeout=10000)
   public void testRebalancing() throws IOException {
@@ -113,14 +113,14 @@ public class TestRegionRebalancing exten
       cluster.startRegionServer();
       checkingServerStatus();
     }
-
+   
     setInjectionHandler();
 
     LOG.debug("Restart: killing 1 region server.");
     cluster.stopRegionServer(2, false);
     cluster.waitOnRegionServer(2);
     assertRegionsAreBalanced();
-
+    
     LOG.debug("Restart: adding that region server back");
     cluster.startRegionServer();
     assertRegionsAreBalanced();
@@ -154,7 +154,7 @@ public class TestRegionRebalancing exten
     List<HRegionServer> servers = getOnlineRegionServers();
     double avg = cluster.getMaster().getAverageLoad();
     for (HRegionServer server : servers) {
-      int serverLoad = server.getOnlineRegions().size();
+      int serverLoad = server.getOnlineRegions().size();      
       LOG.debug(server.hashCode() + " Avg: " + avg + " actual: " + serverLoad);
     }
   }
@@ -177,14 +177,14 @@ public class TestRegionRebalancing exten
     boolean success = false;
     float slop = conf.getFloat("hbase.regions.slop", (float)0.1);
     if (slop <= 0) slop = 1;
-
+    
     // waiting for load balancer running
     try {
       Thread.sleep(MiniHBaseCluster.WAIT_FOR_LOADBALANCER);
     } catch (InterruptedException e1) {
       LOG.error("Got InterruptedException when waiting for load balance " + e1);
     }
-
+    
     for (int i = 0; i < 5; i++) {
       success = true;
       // make sure all the regions are reassigned before we test balance
@@ -195,18 +195,18 @@ public class TestRegionRebalancing exten
       double avg = cluster.getMaster().getAverageLoad();
       int avgLoadPlusSlop = (int)Math.ceil(avg * (1 + slop));
       int avgLoadMinusSlop = (int)Math.floor(avg * (1 - slop)) - 1;
-
+     
       LOG.debug("There are " + servers.size() + " servers and " + regionCount
         + " regions. Load Average: " + avg + " low border: " + avgLoadMinusSlop
         + ", up border: " + avgLoadPlusSlop + "; attempt: " + i);
-
+     
       for (HRegionServer server : servers) {
-        int serverLoad = server.getOnlineRegions().size();
-        LOG.debug(server.hashCode() + " Avg: " + avg + " actual: " + serverLoad);
+        int serverLoad = server.getOnlineRegions().size(); 
+        LOG.debug(server.hashCode() + " Avg: " + avg + " actual: " + serverLoad);      
         if (!(avg > 2.0 && serverLoad <= avgLoadPlusSlop
-            && serverLoad >= avgLoadMinusSlop)) {
+            && serverLoad >= avgLoadMinusSlop)) { 
           LOG.debug(server.hashCode() + " Isn't balanced!!! Avg: " + avg +
-              " actual: " + serverLoad + " slop: " + slop);
+              " actual: " + serverLoad + " slop: " + slop);   
           success = false;
         }
       }
@@ -267,3 +267,4 @@ public class TestRegionRebalancing exten
     return region;
   }
 }
+

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed Mar 12 21:17:13 2014
@@ -218,8 +218,8 @@ public class TestZooKeeper {
     LOG.info("Starting testMasterSessionExpired");
     new HTable(conf, HConstants.META_TABLE_NAME);
     TEST_UTIL.expireMasterSession();
-
-    List<RegionServerThread> regionServerThreadList =
+    
+    List<RegionServerThread> regionServerThreadList = 
       TEST_UTIL.getHBaseCluster().getRegionServerThreads();
     for (RegionServerThread regionServerThread : regionServerThreadList) {
       regionServerThread.getRegionServer().kill();

Copied: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/UnstableTestSuite.java (from r1576907, hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java)
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/UnstableTestSuite.java?p2=hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/UnstableTestSuite.java&p1=hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java&r1=1576907&r2=1576909&rev=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/YouAreDeadException.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/UnstableTestSuite.java Wed Mar 12 21:17:13 2014
@@ -1,5 +1,5 @@
 /**
- * Copyright 2010 The Apache Software Foundation
+ * Copyright 2014 The Apache Software Foundation
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -19,16 +19,12 @@
  */
 package org.apache.hadoop.hbase;
 
-import java.io.IOException;
+import org.apache.hadoop.hbase.regionserver.TestHRegionCloseRetry;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+import org.junit.runners.Suite.SuiteClasses;
 
-/**
- * This exception is thrown by the master when a region server reports and is
- * already being processed as dead. This can happen when a region server loses
- * its session but didn't figure it yet.
- */
-public class YouAreDeadException extends IOException {
-
-  public YouAreDeadException(String message) {
-    super(message);
-  }
+@RunWith(Suite.class)
+@SuiteClasses({ TestHRegionCloseRetry.class })
+public class UnstableTestSuite {
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/avro/TestAvroServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/avro/TestAvroServer.java?rev=1576909&r1=1576908&r2=1576909&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/avro/TestAvroServer.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/avro/TestAvroServer.java Wed Mar 12 21:17:13 2014
@@ -1,4 +1,4 @@
-/**
+/** 
  * 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
@@ -152,7 +152,7 @@ public class TestAvroServer {
     GenericArray<AFamilyDescriptor> families = new GenericData.Array<AFamilyDescriptor>(1, familyArraySchema);
     families.add(familyA);
     tableA.families = families;
-    impl.createTable(tableA);
+    impl.createTable(tableA);    
     assertEquals(impl.describeTable(tableAname).families.size(), 1);
 
     impl.disableTable(tableAname);
@@ -198,7 +198,7 @@ public class TestAvroServer {
     column.qualifier = qualifierAname;
     columns.add(column);
     getA.columns = columns;
-
+   
     assertFalse(impl.exists(tableAname, getA));
 
     APut putA = new APut();