Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 856E7E61C for ; Mon, 25 Feb 2013 22:52:10 +0000 (UTC) Received: (qmail 24434 invoked by uid 500); 25 Feb 2013 22:52:10 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 24384 invoked by uid 500); 25 Feb 2013 22:52:10 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 24376 invoked by uid 99); 25 Feb 2013 22:52:10 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Feb 2013 22:52:10 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Feb 2013 22:52:05 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 5E4722388C64; Mon, 25 Feb 2013 22:50:54 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1449950 [16/35] - in /hbase/trunk: ./ hbase-client/ hbase-client/src/ hbase-client/src/main/ hbase-client/src/main/java/ hbase-client/src/main/java/org/ hbase-client/src/main/java/org/apache/ hbase-client/src/main/java/org/apache/hadoop/ h... Date: Mon, 25 Feb 2013 22:50:29 -0000 To: commits@hbase.apache.org From: eclark@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130225225054.5E4722388C64@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableInfoMissingException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableInfoMissingException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableInfoMissingException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableInfoMissingException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,47 @@ +/** + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * + * Failed to find .tableinfo file under table dir + * + */ +@InterfaceAudience.Private +@SuppressWarnings("serial") +public class TableInfoMissingException extends HBaseIOException { + + public TableInfoMissingException() { + super(); + } + + public TableInfoMissingException( String message ) { + super(message); + } + + public TableInfoMissingException( String message, Throwable t ) { + super(message, t); + } + + public TableInfoMissingException( Throwable t ) { + super(t); + } + +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotDisabledException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotDisabledException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotDisabledException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotDisabledException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,53 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +/** + * Thrown if a table should be offline but is not + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class TableNotDisabledException extends IOException { + private static final long serialVersionUID = 1L << 19 - 1L; + /** default constructor */ + public TableNotDisabledException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public TableNotDisabledException(String s) { + super(s); + } + + /** + * @param tableName Name of table that is not disabled + */ + public TableNotDisabledException(byte[] tableName) { + this(Bytes.toString(tableName)); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotEnabledException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotEnabledException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotEnabledException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotEnabledException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,53 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +/** + * Thrown if a table should be enabled but is not + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class TableNotEnabledException extends IOException { + private static final long serialVersionUID = 262144L; + /** default constructor */ + public TableNotEnabledException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public TableNotEnabledException(String s) { + super(s); + } + + /** + * @param tableName Name of table that is not enabled + */ + public TableNotEnabledException(byte[] tableName) { + this(Bytes.toString(tableName)); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotFoundException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotFoundException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotFoundException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TableNotFoundException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,39 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** Thrown when a table can not be located */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class TableNotFoundException extends RegionException { + private static final long serialVersionUID = 993179627856392526L; + + /** default constructor */ + public TableNotFoundException() { + super(); + } + + /** @param s message */ + public TableNotFoundException(String s) { + super(s); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TablePartiallyOpenException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TablePartiallyOpenException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TablePartiallyOpenException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/TablePartiallyOpenException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,51 @@ +/** + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +/** + * Thrown if a table should be online/offline but is partially open + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class TablePartiallyOpenException extends IOException { + private static final long serialVersionUID = 3571982660065058361L; + + public TablePartiallyOpenException() { + super(); + } + + /** + * @param s message + */ + public TablePartiallyOpenException(String s) { + super(s); + } + + /** + * @param tableName Name of table that is partial open + */ + public TablePartiallyOpenException(byte[] tableName) { + this(Bytes.toString(tableName)); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownProtocolException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,46 @@ +/** + * + * 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.exceptions; + +/** + * An error requesting an RPC protocol that the server is not serving. + */ +@SuppressWarnings("serial") +public class UnknownProtocolException extends org.apache.hadoop.hbase.exceptions.DoNotRetryIOException { + private Class protocol; + + public UnknownProtocolException(String mesg) { + // required for unwrapping from a RemoteException + super(mesg); + } + + public UnknownProtocolException(Class protocol) { + this(protocol, "Server is not handling protocol "+protocol.getName()); + } + + public UnknownProtocolException(Class protocol, String mesg) { + super(mesg); + this.protocol = protocol; + } + + public Class getProtocol() { + return protocol; + } +} \ No newline at end of file Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownRegionException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownRegionException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownRegionException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownRegionException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,35 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when we are asked to operate on a region we know nothing about. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class UnknownRegionException extends RegionException { + private static final long serialVersionUID = 1968858760475205392L; + + public UnknownRegionException(String regionName) { + super(regionName); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownScannerException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownScannerException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownScannerException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownScannerException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,48 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + + +/** + * Thrown if a region server is passed an unknown scanner id. + * Usually means the client has take too long between checkins and so the + * scanner lease on the serverside has expired OR the serverside is closing + * down and has cancelled all leases. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class UnknownScannerException extends DoNotRetryIOException { + private static final long serialVersionUID = 993179627856392526L; + + /** constructor */ + public UnknownScannerException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public UnknownScannerException(String s) { + super(s); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownSnapshotException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownSnapshotException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownSnapshotException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/UnknownSnapshotException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,42 @@ +/** + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Exception thrown when we get a request for a snapshot we don't recognize. + */ +@SuppressWarnings("serial") +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class UnknownSnapshotException extends HBaseSnapshotException { + + /** + * @param msg full information about the failure + */ + public UnknownSnapshotException(String msg) { + super(msg); + } + + public UnknownSnapshotException(String msg, Exception e) { + super(msg, e); + } + +} \ No newline at end of file Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/WrongRegionException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/WrongRegionException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/WrongRegionException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/WrongRegionException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,46 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.IOException; + +/** + * Thrown when a request contains a key which is not part of this region + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class WrongRegionException extends IOException { + private static final long serialVersionUID = 993179627856392526L; + + /** constructor */ + public WrongRegionException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public WrongRegionException(String s) { + super(s); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/YouAreDeadException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/YouAreDeadException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/YouAreDeadException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/YouAreDeadException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,38 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.IOException; + +/** + * 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. + */ +@SuppressWarnings("serial") +@InterfaceAudience.Public +@InterfaceStability.Stable +public class YouAreDeadException extends IOException { + public YouAreDeadException(String message) { + super(message); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ZooKeeperConnectionException.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ZooKeeperConnectionException.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ZooKeeperConnectionException.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ZooKeeperConnectionException.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,53 @@ +/** + * + * 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.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.IOException; + +/** + * Thrown if the client can't connect to zookeeper + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ZooKeeperConnectionException extends IOException { + private static final long serialVersionUID = 1L << 23 - 1L; + /** default constructor */ + public ZooKeeperConnectionException() { + super(); + } + + /** + * Constructor + * @param s message + */ + public ZooKeeperConnectionException(String s) { + super(s); + } + + /** + * Constructor taking another exception. + * @param e Exception to grab data from. + */ + public ZooKeeperConnectionException(String message, Exception e) { + super(message, e); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/EventType.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,115 @@ +/** + * + * 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.executor; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * List of all HBase event handler types. Event types are named by a + * convention: event type names specify the component from which the event + * originated and then where its destined -- e.g. RS2ZK_ prefix means the + * event came from a regionserver destined for zookeeper -- and then what + * the even is; e.g. REGION_OPENING. + * + *

We give the enums indices so we can add types later and keep them + * grouped together rather than have to add them always to the end as we + * would have to if we used raw enum ordinals. + */ +@InterfaceAudience.Private +public enum EventType { + // Messages originating from RS (NOTE: there is NO direct communication from + // RS to Master). These are a result of RS updates into ZK. + // RS_ZK_REGION_CLOSING (1), // It is replaced by M_ZK_REGION_CLOSING(HBASE-4739) + RS_ZK_REGION_CLOSED (2, ExecutorType.MASTER_CLOSE_REGION), // RS has finished closing a region + RS_ZK_REGION_OPENING (3, null), // RS is in process of opening a region + RS_ZK_REGION_OPENED (4, ExecutorType.MASTER_OPEN_REGION), // RS has finished opening a region + RS_ZK_REGION_SPLITTING (5, null), // RS has started a region split + RS_ZK_REGION_SPLIT (6, ExecutorType.MASTER_SERVER_OPERATIONS), // RS split has completed. + RS_ZK_REGION_FAILED_OPEN (7, ExecutorType.MASTER_CLOSE_REGION), // RS failed to open a region + + // Messages originating from Master to RS + M_RS_OPEN_REGION (20, ExecutorType.RS_OPEN_REGION), // Master asking RS to open a region + M_RS_OPEN_ROOT (21, ExecutorType.RS_OPEN_ROOT), // Master asking RS to open root + M_RS_OPEN_META (22, ExecutorType.RS_OPEN_META), // Master asking RS to open meta + M_RS_CLOSE_REGION (23, ExecutorType.RS_CLOSE_REGION), // Master asking RS to close a region + M_RS_CLOSE_ROOT (24, ExecutorType.RS_CLOSE_ROOT), // Master asking RS to close root + M_RS_CLOSE_META (25, ExecutorType.RS_CLOSE_META), // Master asking RS to close meta + + // Messages originating from Client to Master + C_M_DELETE_TABLE (40, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to delete a table + C_M_DISABLE_TABLE (41, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to disable a table + C_M_ENABLE_TABLE (42, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to enable a table + C_M_MODIFY_TABLE (43, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to modify a table + C_M_ADD_FAMILY (44, null), // Client asking Master to add family to table + C_M_DELETE_FAMILY (45, null), // Client asking Master to delete family of table + C_M_MODIFY_FAMILY (46, null), // Client asking Master to modify family of table + C_M_CREATE_TABLE (47, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to create a table + C_M_SNAPSHOT_TABLE (48, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to snapshot an offline table + C_M_RESTORE_SNAPSHOT (49, ExecutorType.MASTER_TABLE_OPERATIONS), // Client asking Master to restore a snapshot + + // Updates from master to ZK. This is done by the master and there is + // nothing to process by either Master or RS + M_ZK_REGION_OFFLINE (50, null), // Master adds this region as offline in ZK + M_ZK_REGION_CLOSING (51, null), // Master adds this region as closing in ZK + + // Master controlled events to be executed on the master + M_SERVER_SHUTDOWN (70, ExecutorType.MASTER_SERVER_OPERATIONS), // Master is processing shutdown of a RS + M_META_SERVER_SHUTDOWN (72, ExecutorType.MASTER_META_SERVER_OPERATIONS), // Master is processing shutdown of RS hosting a meta region (-ROOT- or .META.). + M_MASTER_RECOVERY (73, ExecutorType.MASTER_SERVER_OPERATIONS), // Master is processing recovery of regions found in ZK RIT + + // RS controlled events to be executed on the RS + RS_PARALLEL_SEEK (80, ExecutorType.RS_PARALLEL_SEEK); + + private final int code; + private final ExecutorType executor; + + /** + * Constructor + */ + EventType(final int code, final ExecutorType executor) { + this.code = code; + this.executor = executor; + } + + public int getCode() { + return this.code; + } + + public static EventType get(final int code) { + // Is this going to be slow? Its used rare but still... + for (EventType et: EventType.values()) { + if (et.getCode() == code) return et; + } + throw new IllegalArgumentException("Unknown code " + code); + } + + public boolean isOnlineSchemaChangeSupported() { + return ( + this.equals(EventType.C_M_ADD_FAMILY) || + this.equals(EventType.C_M_DELETE_FAMILY) || + this.equals(EventType.C_M_MODIFY_FAMILY) || + this.equals(EventType.C_M_MODIFY_TABLE) + ); + } + + ExecutorType getExecutorServiceType() { + return this.executor; + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,57 @@ +/** + * + * 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.executor; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * The following is a list of all executor types, both those that run in the + * master and those that run in the regionserver. + */ +@InterfaceAudience.Private +public enum ExecutorType { + + // Master executor services + MASTER_CLOSE_REGION (1), + MASTER_OPEN_REGION (2), + MASTER_SERVER_OPERATIONS (3), + MASTER_TABLE_OPERATIONS (4), + MASTER_RS_SHUTDOWN (5), + MASTER_META_SERVER_OPERATIONS (6), + + // RegionServer executor services + RS_OPEN_REGION (20), + RS_OPEN_ROOT (21), + RS_OPEN_META (22), + RS_CLOSE_REGION (23), + RS_CLOSE_ROOT (24), + RS_CLOSE_META (25), + RS_PARALLEL_SEEK (26); + + ExecutorType(int value) {} + + /** + * @param serverName + * @return Conflation of the executor type and the passed servername. + */ + String getExecutorName(String serverName) { + return this.toString() + "-" + serverName.replace("%", "%%"); + } +} \ No newline at end of file Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,88 @@ +/* + * + * 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.filter; + +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * A binary comparator which lexicographically compares against the specified + * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class BinaryComparator extends ByteArrayComparable { + + /** + * Constructor + * @param value value + */ + public BinaryComparator(byte[] value) { + super(value); + } + + @Override + public int compareTo(byte [] value, int offset, int length) { + return Bytes.compareTo(this.value, 0, this.value.length, value, offset, length); + } + + /** + * @return The comparator serialized using pb + */ + public byte [] toByteArray() { + ComparatorProtos.BinaryComparator.Builder builder = + ComparatorProtos.BinaryComparator.newBuilder(); + builder.setComparable(super.convert()); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link BinaryComparator} instance + * @return An instance of {@link BinaryComparator} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static BinaryComparator parseFrom(final byte [] pbBytes) + throws DeserializationException { + ComparatorProtos.BinaryComparator proto; + try { + proto = ComparatorProtos.BinaryComparator.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new BinaryComparator(proto.getComparable().getValue().toByteArray()); + } + + /** + * @param other + * @return true if and only if the fields of the comparator that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(ByteArrayComparable other) { + if (other == this) return true; + if (!(other instanceof BinaryComparator)) return false; + + return super.areSerializedFieldsEqual(other); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,90 @@ +/* + * + * 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.filter; + +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * A comparator which compares against a specified byte array, but only compares + * up to the length of this byte array. For the rest it is similar to + * {@link BinaryComparator}. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class BinaryPrefixComparator extends ByteArrayComparable { + + /** + * Constructor + * @param value value + */ + public BinaryPrefixComparator(byte[] value) { + super(value); + } + + @Override + public int compareTo(byte [] value, int offset, int length) { + return Bytes.compareTo(this.value, 0, this.value.length, value, offset, + this.value.length <= length ? this.value.length : length); + } + + /** + * @return The comparator serialized using pb + */ + public byte [] toByteArray() { + ComparatorProtos.BinaryPrefixComparator.Builder builder = + ComparatorProtos.BinaryPrefixComparator.newBuilder(); + builder.setComparable(super.convert()); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link BinaryPrefixComparator} instance + * @return An instance of {@link BinaryPrefixComparator} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static BinaryPrefixComparator parseFrom(final byte [] pbBytes) + throws DeserializationException { + ComparatorProtos.BinaryPrefixComparator proto; + try { + proto = ComparatorProtos.BinaryPrefixComparator.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new BinaryPrefixComparator(proto.getComparable().getValue().toByteArray()); + } + + /** + * @param other + * @return true if and only if the fields of the comparator that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(ByteArrayComparable other) { + if (other == this) return true; + if (!(other instanceof BinaryPrefixComparator)) return false; + + return super.areSerializedFieldsEqual(other); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,132 @@ +/* + * + * 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.filter; + +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; + +/** + * A bit comparator which performs the specified bitwise operation on each of the bytes + * with the specified byte array. Then returns whether the result is non-zero. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class BitComparator extends ByteArrayComparable { + + /** Bit operators. */ + public enum BitwiseOp { + /** and */ + AND, + /** or */ + OR, + /** xor */ + XOR + } + protected BitwiseOp bitOperator; + + /** + * Constructor + * @param value value + * @param bitOperator operator to use on the bit comparison + */ + public BitComparator(byte[] value, BitwiseOp bitOperator) { + super(value); + this.bitOperator = bitOperator; + } + + /** + * @return the bitwise operator + */ + public BitwiseOp getOperator() { + return bitOperator; + } + + /** + * @return The comparator serialized using pb + */ + public byte [] toByteArray() { + ComparatorProtos.BitComparator.Builder builder = + ComparatorProtos.BitComparator.newBuilder(); + builder.setComparable(super.convert()); + ComparatorProtos.BitComparator.BitwiseOp bitwiseOpPb = + ComparatorProtos.BitComparator.BitwiseOp.valueOf(bitOperator.name()); + builder.setBitwiseOp(bitwiseOpPb); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link BitComparator} instance + * @return An instance of {@link BitComparator} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static BitComparator parseFrom(final byte [] pbBytes) + throws DeserializationException { + ComparatorProtos.BitComparator proto; + try { + proto = ComparatorProtos.BitComparator.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + BitwiseOp bitwiseOp = BitwiseOp.valueOf(proto.getBitwiseOp().name()); + return new BitComparator(proto.getComparable().getValue().toByteArray(),bitwiseOp); + } + + /** + * @param other + * @return true if and only if the fields of the comparator that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(ByteArrayComparable other) { + if (other == this) return true; + if (!(other instanceof BitComparator)) return false; + + BitComparator comparator = (BitComparator)other; + return super.areSerializedFieldsEqual(other) + && this.getOperator().equals(comparator.getOperator()); + } + + @Override + public int compareTo(byte[] value, int offset, int length) { + if (length != this.value.length) { + return 1; + } + int b = 0; + //Iterating backwards is faster because we can quit after one non-zero byte. + for (int i = length - 1; i >= 0 && b == 0; i--) { + switch (bitOperator) { + case AND: + b = (this.value[i] & value[i+offset]) & 0xff; + break; + case OR: + b = (this.value[i] | value[i+offset]) & 0xff; + break; + case XOR: + b = (this.value[i] ^ value[i+offset]) & 0xff; + break; + } + } + return b == 0 ? 1 : 0; + } +} + Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,98 @@ +/** + * + * 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.filter; + +import com.google.protobuf.ByteString; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; +import org.apache.hadoop.hbase.util.Bytes; + + +/** Base class for byte array comparators */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class ByteArrayComparable implements Comparable { + + byte[] value; + + /** + * Constructor. + * @param value the value to compare against + */ + public ByteArrayComparable(byte [] value) { + this.value = value; + } + + public byte[] getValue() { + return value; + } + + /** + * @return The comparator serialized using pb + */ + public abstract byte [] toByteArray(); + + ComparatorProtos.ByteArrayComparable convert() { + ComparatorProtos.ByteArrayComparable.Builder builder = + ComparatorProtos.ByteArrayComparable.newBuilder(); + if (value != null) builder.setValue(ByteString.copyFrom(value)); + return builder.build(); + } + + /** + * @param pbBytes A pb serialized {@link ByteArrayComparable} instance + * @return An instance of {@link ByteArrayComparable} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static ByteArrayComparable parseFrom(final byte [] pbBytes) + throws DeserializationException { + throw new DeserializationException( + "parseFrom called on base ByteArrayComparable, but should be called on derived type"); + } + + /** + * @param other + * @return true if and only if the fields of the comparator that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(ByteArrayComparable other) { + if (other == this) return true; + + return Bytes.equals(this.getValue(), other.getValue()); + } + + @Override + public int compareTo(byte [] value) { + return compareTo(value, 0, value.length); + } + + /** + * Special compareTo method for subclasses, to avoid + * copying byte[] unnecessarily. + * @param value byte[] to compare + * @param offset offset into value + * @param length number of bytes to compare + * @return a negative integer, zero, or a positive integer as this object + * is less than, equal to, or greater than the specified object. + */ + public abstract int compareTo(byte [] value, int offset, int length); +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,120 @@ +/* + * + * 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.filter; + +import com.google.common.base.Preconditions; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; + +import java.util.ArrayList; + +/** + * Simple filter that returns first N columns on row only. + * This filter was written to test filters in Get and as soon as it gets + * its quota of columns, {@link #filterAllRemaining()} returns true. This + * makes this filter unsuitable as a Scan filter. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ColumnCountGetFilter extends FilterBase { + private int limit = 0; + private int count = 0; + + public ColumnCountGetFilter(final int n) { + Preconditions.checkArgument(n >= 0, "limit be positive %s", n); + this.limit = n; + } + + public int getLimit() { + return limit; + } + + @Override + public boolean filterAllRemaining() { + return this.count > this.limit; + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) { + this.count++; + return filterAllRemaining() ? ReturnCode.NEXT_COL : ReturnCode.INCLUDE_AND_NEXT_COL; + } + + @Override + public void reset() { + this.count = 0; + } + + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 1, + "Expected 1 but got: %s", filterArguments.size()); + int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0)); + return new ColumnCountGetFilter(limit); + } + + /** + * @return The filter serialized using pb + */ + public byte [] toByteArray() { + FilterProtos.ColumnCountGetFilter.Builder builder = + FilterProtos.ColumnCountGetFilter.newBuilder(); + builder.setLimit(this.limit); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link ColumnCountGetFilter} instance + * @return An instance of {@link ColumnCountGetFilter} made from bytes + * @throws org.apache.hadoop.hbase.exceptions.DeserializationException + * @see #toByteArray + */ + public static ColumnCountGetFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.ColumnCountGetFilter proto; + try { + proto = FilterProtos.ColumnCountGetFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new ColumnCountGetFilter(proto.getLimit()); + } + + /** + * @param other + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof ColumnCountGetFilter)) return false; + + ColumnCountGetFilter other = (ColumnCountGetFilter)o; + return this.getLimit() == other.getLimit(); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + this.limit; + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,141 @@ +/* + * + * 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.filter; + +import com.google.common.base.Preconditions; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; + +import java.util.ArrayList; + +/** + * A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset. + * This filter can be used for row-based indexing, where references to other tables are stored across many columns, + * in order to efficient lookups and paginated results for end users. Only most recent versions are considered + * for pagination. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ColumnPaginationFilter extends FilterBase +{ + private int limit = 0; + private int offset = 0; + private int count = 0; + + public ColumnPaginationFilter(final int limit, final int offset) + { + Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit); + Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset); + this.limit = limit; + this.offset = offset; + } + + /** + * @return limit + */ + public int getLimit() { + return limit; + } + + /** + * @return offset + */ + public int getOffset() { + return offset; + } + + @Override + public ReturnCode filterKeyValue(KeyValue v) + { + if(count >= offset + limit) + { + return ReturnCode.NEXT_ROW; + } + + ReturnCode code = count < offset ? ReturnCode.NEXT_COL : + ReturnCode.INCLUDE_AND_NEXT_COL; + count++; + return code; + } + + @Override + public void reset() + { + this.count = 0; + } + + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 2, + "Expected 2 but got: %s", filterArguments.size()); + int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0)); + int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1)); + return new ColumnPaginationFilter(limit, offset); + } + + /** + * @return The filter serialized using pb + */ + public byte [] toByteArray() { + FilterProtos.ColumnPaginationFilter.Builder builder = + FilterProtos.ColumnPaginationFilter.newBuilder(); + builder.setLimit(this.limit); + builder.setOffset(this.offset); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link ColumnPaginationFilter} instance + * @return An instance of {@link ColumnPaginationFilter} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static ColumnPaginationFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.ColumnPaginationFilter proto; + try { + proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new ColumnPaginationFilter(proto.getLimit(),proto.getOffset()); + } + + /** + * @param other + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof ColumnPaginationFilter)) return false; + + ColumnPaginationFilter other = (ColumnPaginationFilter)o; + return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset(); + } + + @Override + public String toString() { + return String.format("%s (%d, %d)", this.getClass().getSimpleName(), + this.limit, this.offset); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,140 @@ +/* + * + * 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.filter; + +import com.google.common.base.Preconditions; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.Bytes; + +import java.util.ArrayList; + +/** + * This filter is used for selecting only those keys with columns that matches + * a particular prefix. For example, if prefix is 'an', it will pass keys with + * columns like 'and', 'anti' but not keys with columns like 'ball', 'act'. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ColumnPrefixFilter extends FilterBase { + protected byte [] prefix = null; + + public ColumnPrefixFilter(final byte [] prefix) { + this.prefix = prefix; + } + + public byte[] getPrefix() { + return prefix; + } + + @Override + public ReturnCode filterKeyValue(KeyValue kv) { + if (this.prefix == null || kv.getBuffer() == null) { + return ReturnCode.INCLUDE; + } else { + return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()); + } + } + + public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) { + if (qualifierLength < prefix.length) { + int cmp = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, this.prefix, 0, + qualifierLength); + if (cmp <= 0) { + return ReturnCode.SEEK_NEXT_USING_HINT; + } else { + return ReturnCode.NEXT_ROW; + } + } else { + int cmp = Bytes.compareTo(buffer, qualifierOffset, this.prefix.length, this.prefix, 0, + this.prefix.length); + if (cmp < 0) { + return ReturnCode.SEEK_NEXT_USING_HINT; + } else if (cmp > 0) { + return ReturnCode.NEXT_ROW; + } else { + return ReturnCode.INCLUDE; + } + } + } + + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 1, + "Expected 1 but got: %s", filterArguments.size()); + byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + return new ColumnPrefixFilter(columnPrefix); + } + + /** + * @return The filter serialized using pb + */ + public byte [] toByteArray() { + FilterProtos.ColumnPrefixFilter.Builder builder = + FilterProtos.ColumnPrefixFilter.newBuilder(); + if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix)); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link ColumnPrefixFilter} instance + * @return An instance of {@link ColumnPrefixFilter} made from bytes + * @throws org.apache.hadoop.hbase.exceptions.DeserializationException + * @see #toByteArray + */ + public static ColumnPrefixFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.ColumnPrefixFilter proto; + try { + proto = FilterProtos.ColumnPrefixFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new ColumnPrefixFilter(proto.getPrefix().toByteArray()); + } + + /** + * @param other + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof ColumnPrefixFilter)) return false; + + ColumnPrefixFilter other = (ColumnPrefixFilter)o; + return Bytes.equals(this.getPrefix(), other.getPrefix()); + } + + public KeyValue getNextKeyHint(KeyValue kv) { + return KeyValue.createFirstOnRow( + kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(), + kv.getFamilyOffset(), kv.getFamilyLength(), prefix, 0, prefix.length); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,228 @@ +/* + * + * 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.filter; + +import com.google.common.base.Preconditions; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.Bytes; + +import java.util.ArrayList; + +/** + * This filter is used for selecting only those keys with columns that are + * between minColumn to maxColumn. For example, if minColumn is 'an', and + * maxColumn is 'be', it will pass keys with columns like 'ana', 'bad', but not + * keys with columns like 'bed', 'eye' + * + * If minColumn is null, there is no lower bound. If maxColumn is null, there is + * no upper bound. + * + * minColumnInclusive and maxColumnInclusive specify if the ranges are inclusive + * or not. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ColumnRangeFilter extends FilterBase { + protected byte[] minColumn = null; + protected boolean minColumnInclusive = true; + protected byte[] maxColumn = null; + protected boolean maxColumnInclusive = false; + + /** + * Create a filter to select those keys with columns that are between minColumn + * and maxColumn. + * @param minColumn minimum value for the column range. If if it's null, + * there is no lower bound. + * @param minColumnInclusive if true, include minColumn in the range. + * @param maxColumn maximum value for the column range. If it's null, + * @param maxColumnInclusive if true, include maxColumn in the range. + * there is no upper bound. + */ + public ColumnRangeFilter(final byte[] minColumn, boolean minColumnInclusive, + final byte[] maxColumn, boolean maxColumnInclusive) { + this.minColumn = minColumn; + this.minColumnInclusive = minColumnInclusive; + this.maxColumn = maxColumn; + this.maxColumnInclusive = maxColumnInclusive; + } + + /** + * @return if min column range is inclusive. + */ + public boolean isMinColumnInclusive() { + return minColumnInclusive; + } + + /** + * @return if max column range is inclusive. + */ + public boolean isMaxColumnInclusive() { + return maxColumnInclusive; + } + + /** + * @return the min column range for the filter + */ + public byte[] getMinColumn() { + return this.minColumn; + } + + /** + * @return true if min column is inclusive, false otherwise + */ + public boolean getMinColumnInclusive() { + return this.minColumnInclusive; + } + + /** + * @return the max column range for the filter + */ + public byte[] getMaxColumn() { + return this.maxColumn; + } + + /** + * @return true if max column is inclusive, false otherwise + */ + public boolean getMaxColumnInclusive() { + return this.maxColumnInclusive; + } + + @Override + public ReturnCode filterKeyValue(KeyValue kv) { + byte[] buffer = kv.getBuffer(); + int qualifierOffset = kv.getQualifierOffset(); + int qualifierLength = kv.getQualifierLength(); + int cmpMin = 1; + + if (this.minColumn != null) { + cmpMin = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, + this.minColumn, 0, this.minColumn.length); + } + + if (cmpMin < 0) { + return ReturnCode.SEEK_NEXT_USING_HINT; + } + + if (!this.minColumnInclusive && cmpMin == 0) { + return ReturnCode.SKIP; + } + + if (this.maxColumn == null) { + return ReturnCode.INCLUDE; + } + + int cmpMax = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, + this.maxColumn, 0, this.maxColumn.length); + + if (this.maxColumnInclusive && cmpMax <= 0 || + !this.maxColumnInclusive && cmpMax < 0) { + return ReturnCode.INCLUDE; + } + + return ReturnCode.NEXT_ROW; + } + + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 4, + "Expected 4 but got: %s", filterArguments.size()); + byte [] minColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + boolean minColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(1)); + byte [] maxColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(2)); + boolean maxColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(3)); + + if (minColumn.length == 0) + minColumn = null; + if (maxColumn.length == 0) + maxColumn = null; + return new ColumnRangeFilter(minColumn, minColumnInclusive, + maxColumn, maxColumnInclusive); + } + + /** + * @return The filter serialized using pb + */ + public byte [] toByteArray() { + FilterProtos.ColumnRangeFilter.Builder builder = + FilterProtos.ColumnRangeFilter.newBuilder(); + if (this.minColumn != null) builder.setMinColumn(ByteString.copyFrom(this.minColumn)); + builder.setMinColumnInclusive(this.minColumnInclusive); + if (this.maxColumn != null) builder.setMaxColumn(ByteString.copyFrom(this.maxColumn)); + builder.setMaxColumnInclusive(this.maxColumnInclusive); + return builder.build().toByteArray(); + } + + /** + * @param pbBytes A pb serialized {@link ColumnRangeFilter} instance + * @return An instance of {@link ColumnRangeFilter} made from bytes + * @throws DeserializationException + * @see #toByteArray + */ + public static ColumnRangeFilter parseFrom(final byte [] pbBytes) + throws DeserializationException { + FilterProtos.ColumnRangeFilter proto; + try { + proto = FilterProtos.ColumnRangeFilter.parseFrom(pbBytes); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return new ColumnRangeFilter(proto.hasMinColumn()?proto.getMinColumn().toByteArray():null, + proto.getMinColumnInclusive(),proto.hasMaxColumn()?proto.getMaxColumn().toByteArray():null, + proto.getMaxColumnInclusive()); + } + + /** + * @param other + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof ColumnRangeFilter)) return false; + + ColumnRangeFilter other = (ColumnRangeFilter)o; + return Bytes.equals(this.getMinColumn(),other.getMinColumn()) + && this.getMinColumnInclusive() == other.getMinColumnInclusive() + && Bytes.equals(this.getMaxColumn(), other.getMaxColumn()) + && this.getMaxColumnInclusive() == other.getMaxColumnInclusive(); + } + + @Override + public KeyValue getNextKeyHint(KeyValue kv) { + return KeyValue.createFirstOnRow(kv.getBuffer(), kv.getRowOffset(), kv + .getRowLength(), kv.getBuffer(), kv.getFamilyOffset(), kv + .getFamilyLength(), this.minColumn, 0, this.minColumn == null ? 0 + : this.minColumn.length); + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + " " + + (this.minColumnInclusive ? "[" : "(") + Bytes.toStringBinary(this.minColumn) + + ", " + Bytes.toStringBinary(this.maxColumn) + + (this.maxColumnInclusive ? "]" : ")"); + } +} Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java?rev=1449950&view=auto ============================================================================== --- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (added) +++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java Mon Feb 25 22:50:17 2013 @@ -0,0 +1,179 @@ +/** + * + * 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.filter; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType; +import org.apache.hadoop.hbase.util.Bytes; + +import java.util.ArrayList; +/** + * This is a generic filter to be used to filter by comparison. It takes an + * operator (equal, greater, not equal, etc) and a byte [] comparator. + *

+ * To filter by row key, use {@link RowFilter}. + *

+ * To filter by column qualifier, use {@link QualifierFilter}. + *

+ * To filter by value, use {@link SingleColumnValueFilter}. + *

+ * These filters can be wrapped with {@link SkipFilter} and {@link WhileMatchFilter} + * to add more control. + *

+ * Multiple filters can be combined using {@link FilterList}. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class CompareFilter extends FilterBase { + + /** Comparison operators. */ + public enum CompareOp { + /** less than */ + LESS, + /** less than or equal to */ + LESS_OR_EQUAL, + /** equals */ + EQUAL, + /** not equal */ + NOT_EQUAL, + /** greater than or equal to */ + GREATER_OR_EQUAL, + /** greater than */ + GREATER, + /** no operation */ + NO_OP, + } + + protected CompareOp compareOp; + protected ByteArrayComparable comparator; + + /** + * Constructor. + * @param compareOp the compare op for row matching + * @param comparator the comparator for row matching + */ + public CompareFilter(final CompareOp compareOp, + final ByteArrayComparable comparator) { + this.compareOp = compareOp; + this.comparator = comparator; + } + + /** + * @return operator + */ + public CompareOp getOperator() { + return compareOp; + } + + /** + * @return the comparator + */ + public ByteArrayComparable getComparator() { + return comparator; + } + + protected boolean doCompare(final CompareOp compareOp, + final ByteArrayComparable comparator, final byte [] data, + final int offset, final int length) { + if (compareOp == CompareOp.NO_OP) { + return true; + } + int compareResult = comparator.compareTo(data, offset, length); + switch (compareOp) { + case LESS: + return compareResult <= 0; + case LESS_OR_EQUAL: + return compareResult < 0; + case EQUAL: + return compareResult != 0; + case NOT_EQUAL: + return compareResult == 0; + case GREATER_OR_EQUAL: + return compareResult > 0; + case GREATER: + return compareResult >= 0; + default: + throw new RuntimeException("Unknown Compare op " + + compareOp.name()); + } + } + + public static ArrayList extractArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 2, + "Expected 2 but got: %s", filterArguments.size()); + CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(0)); + ByteArrayComparable comparator = ParseFilter.createComparator( + ParseFilter.removeQuotesFromByteArray(filterArguments.get(1))); + + if (comparator instanceof RegexStringComparator || + comparator instanceof SubstringComparator) { + if (compareOp != CompareOp.EQUAL && + compareOp != CompareOp.NOT_EQUAL) { + throw new IllegalArgumentException ("A regexstring comparator and substring comparator" + + " can only be used with EQUAL and NOT_EQUAL"); + } + } + ArrayList arguments = new ArrayList(); + arguments.add(compareOp); + arguments.add(comparator); + return arguments; + } + + /** + * @return A pb instance to represent this instance. + */ + FilterProtos.CompareFilter convert() { + FilterProtos.CompareFilter.Builder builder = + FilterProtos.CompareFilter.newBuilder(); + HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name()); + builder.setCompareOp(compareOp); + if (this.comparator != null) builder.setComparator(ProtobufUtil.toComparator(this.comparator)); + return builder.build(); + } + + /** + * + * @param o + * @return true if and only if the fields of the filter that are serialized + * are equal to the corresponding fields in other. Used for testing. + */ + boolean areSerializedFieldsEqual(Filter o) { + if (o == this) return true; + if (!(o instanceof CompareFilter)) return false; + + CompareFilter other = (CompareFilter)o; + return this.getOperator().equals(other.getOperator()) && + (this.getComparator() == other.getComparator() + || this.getComparator().areSerializedFieldsEqual(other.getComparator())); + } + + @Override + public String toString() { + return String.format("%s (%s, %s)", + this.getClass().getSimpleName(), + this.compareOp.name(), + Bytes.toStringBinary(this.comparator.getValue())); + } +}