HBASE-4605 Constraints (Jesse Yates)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1214889 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-12-15 17:44:59 +00:00
parent 52dbd69707
commit 5166afddc5
29 changed files with 1799 additions and 0 deletions

View File

@ -868,6 +868,11 @@ System.out.println("md5 digest as string length: " + sbDigest.length); // ret
</para>
</section>
<section xml:id="constraints"><title>Constraints</title>
<para>HBase currently supports 'constraints' in traditional (SQL) database parlance. The advised usage for Constraints is in enforcing business rules for attributes in the table (eg. make sure values are in the range 1-10).
Constraints could also be used to enforce referential integrity, but this is strongly discouraged as it will dramatically decrease the write throughput of the tables where integrity checking enabled.</para>
</section>
</chapter> <!-- schema design -->
<chapter xml:id="mapreduce">

View File

@ -477,6 +477,16 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
public void remove(final byte [] key) {
values.remove(new ImmutableBytesWritable(key));
}
/**
* Remove metadata represented by the key from the {@link #values} map
*
* @param key Key whose key and value we're to remove from HTableDescriptor
* parameters.
*/
public void remove(final String key) {
remove(Bytes.toBytes(key));
}
/**
* Check if the readOnly flag of the table is set. If the readOnly flag is
@ -788,6 +798,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
* <em> INTERNAL </em> This method is a part of {@link WritableComparable} interface
* and is used for de-serialization of the HTableDescriptor over RPC
*/
@Override
public void readFields(DataInput in) throws IOException {
int version = in.readInt();
if (version < 3)
@ -822,6 +833,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
* <em> INTERNAL </em> This method is a part of {@link WritableComparable} interface
* and is used for serialization of the HTableDescriptor over RPC
*/
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(TABLE_DESCRIPTOR_VERSION);
Bytes.writeByteArray(out, name);
@ -850,6 +862,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
* @return 0 if the contents of the descriptors are exactly matching,
* 1 if there is a mismatch in the contents
*/
@Override
public int compareTo(final HTableDescriptor other) {
int result = Bytes.compareTo(this.name, other.name);
if (result == 0) {
@ -1052,6 +1065,38 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
return false;
}
/**
* Remove a coprocessor from those set on the table
* @param className Class name of the co-processor
*/
public void removeCoprocessor(String className) {
ImmutableBytesWritable match = null;
Matcher keyMatcher;
Matcher valueMatcher;
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : this.values
.entrySet()) {
keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e
.getKey().get()));
if (!keyMatcher.matches()) {
continue;
}
valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes
.toString(e.getValue().get()));
if (!valueMatcher.matches()) {
continue;
}
// get className and compare
String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
// remove the CP if it is present
if (clazz.equals(className.trim())) {
match = e.getKey();
break;
}
}
// if we found a match, remove it
if (match != null)
this.values.remove(match);
}
/**
* Returns the {@link Path} object representing the table directory under

View File

@ -0,0 +1,41 @@
/**
* 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.constraint;
import org.apache.hadoop.conf.Configuration;
/**
* Base class to use when actually implementing a {@link Constraint}. It takes
* care of getting and setting of configuration for the constraint.
*/
public abstract class BaseConstraint implements
Constraint {
private Configuration conf;
@Override
public void setConf(Configuration conf) {
this.conf = conf;
}
@Override
public Configuration getConf() {
return this.conf;
}
}

View File

@ -0,0 +1,75 @@
/**
* 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.constraint;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Put;
/**
* Apply a {@link Constraint} (in traditional database terminology) to a HTable.
* Any number of {@link Constraint Constraints} can be added to the table, in
* any order.
* <p>
* A {@link Constraint} must be added to a table before the table is loaded via
* {@link Constraints#add(HTableDescriptor, Class...)} or
* {@link Constraints#add(HTableDescriptor, org.apache.hadoop.hbase.util.Pair...)}
* (if you want to add a configuration with the {@link Constraint}). Constraints
* will be run in the order that they are added. Further, a Constraint will be
* configured before it is run (on load).
* <p>
* See {@link Constraints#enableConstraint(HTableDescriptor, Class)} and
* {@link Constraints#disableConstraint(HTableDescriptor, Class)} for
* enabling/disabling of a given {@link Constraint} after it has been added.
* <p>
* If a {@link Put} is not valid, the Constraint should throw some sort
* {@link ConstraintException} indicating that the {@link Put} has failed. When
* this exception is thrown, not further retries of the {@link Put} are
* attempted nor are any other {@link Constraint Constraints} attempted (the
* {@link Put} is clearly not valid). Therefore, there are performance
* implications in the order in which {@link BaseConstraint Constraints} are
* specified.
* <p>
* If a {@link Constraint} fails to fail the {@link Put} via a
* {@link ConstraintException}, but instead throws a {@link RuntimeException},
* the entire constraint processing mechanism ({@link ConstraintProcessor}) will
* be unloaded from the table. This ensures that the region server is still
* functional, but not more {@link Put Puts} will be checked via
* {@link Constraint Constraints}.
* <p>
* Further, {@link Constraint Constraints} probably not be used to enforce
* cross-table references as it will cause tremendous write slowdowns, but it is
* possible.
* <p>
* NOTE: Implementing classes must have a nullary (no-args) constructor
*/
public interface Constraint extends Configurable {
/**
* Check a {@link Put} to ensure it is valid for the table. If the {@link Put}
* is valid, then just return from the method. Otherwise, throw an
* {@link Exception} specifying what happened. This {@link Exception} is
* propagated back to the client so you can see what caused the {@link Put} to
* fail.
* @param p {@link Put} to check
* @throws ConstraintException when the {@link Put} does not match the
* constraint.
*/
public void check(Put p) throws ConstraintException;
}

View File

@ -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.constraint;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.client.Put;
/**
* Exception that a user defined constraint throws on failure of a {@link Put}.
* <p>
* Does <b>NOT</b> attempt the {@link Put} multiple times, since the constraint
* <it>should</it> fail every time for the same {@link Put} (it should be
* idempotent).
*/
public class ConstraintException extends DoNotRetryIOException {
private static final long serialVersionUID = 1197446454511704140L;
public ConstraintException() {
super();
}
public ConstraintException(String msg)
{
super(msg);
}
public ConstraintException(String msg, Throwable cause) {
super(msg, cause);
}
}

View File

@ -0,0 +1,89 @@
/**
* 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.constraint;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
/***
* Processes multiple {@link Constraint Constraints} on a given table.
* <p>
* This is an ease of use mechanism - all the functionality here could be
* implemented on any given system by a coprocessor.
*/
public class ConstraintProcessor extends BaseRegionObserver {
private static final Log LOG = LogFactory.getLog(ConstraintProcessor.class);
private final ClassLoader classloader;
private List<? extends Constraint> constraints = new ArrayList<Constraint>();
/**
* Create the constraint processor.
* <p>
* Stores the current classloader.
*/
public ConstraintProcessor() {
classloader = this.getClass().getClassLoader();
}
@Override
public void start(CoprocessorEnvironment environment) {
// make sure we are on a region server
if (!(environment instanceof RegionCoprocessorEnvironment)) {
throw new IllegalArgumentException(
"Constraints only act on regions - started in an environment that was not a region");
}
RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) environment;
HTableDescriptor desc = env.getRegion().getTableDesc();
// load all the constraints from the HTD
try {
this.constraints = Constraints.getConstraints(desc, classloader);
} catch (IOException e) {
throw new IllegalArgumentException(e);
}
if (LOG.isInfoEnabled()) {
LOG.info("Finished loading " + constraints.size()
+ " user Constraints on table: " + new String(desc.getName()));
}
}
@Override
public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put,
WALEdit edit, boolean writeToWAL) throws IOException {
// check the put against the stored constraints
for (Constraint c : constraints) {
c.check(put);
}
// if we made it here, then the Put is valid
}
}

View File

@ -0,0 +1,597 @@
/**
* 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.constraint;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
/**
* Utilities for adding/removing constraints from a table.
* <p>
* Constraints can be added on table load time, via the {@link HTableDescriptor}.
* <p>
* NOTE: this class is NOT thread safe. Concurrent setting/enabling/disabling of
* constraints can cause constraints to be run at incorrect times or not at all.
*/
public final class Constraints {
private static final int DEFAULT_PRIORITY = -1;
private Constraints() {
}
private static final Log LOG = LogFactory.getLog(Constraints.class);
private static final String CONSTRAINT_HTD_KEY_PREFIX = "constraint $";
private static final Pattern CONSTRAINT_HTD_ATTR_KEY_PATTERN = Pattern
.compile(CONSTRAINT_HTD_KEY_PREFIX, Pattern.LITERAL);
// configuration key for if the constraint is enabled
private static final String ENABLED_KEY = "_ENABLED";
// configuration key for the priority
private static final String PRIORITY_KEY = "_PRIORITY";
// smallest priority a constraiNt can have
private static final long MIN_PRIORITY = 0L;
// ensure a priority less than the smallest we could intentionally set
private static final long UNSET_PRIORITY = MIN_PRIORITY - 1;
private static String COUNTER_KEY = "hbase.constraint.counter";
/**
* Enable constraints on a table.
* <p>
* Currently, if you attempt to add a constraint to the table, then
* Constraints will automatically be turned on.
*
* @param desc
* table description to add the processor
* @throws IOException
* If the {@link ConstraintProcessor} CP couldn't be added to the
* table.
*/
public static void enable(HTableDescriptor desc) throws IOException {
// if the CP has already been loaded, do nothing
String clazz = ConstraintProcessor.class.getName();
if (desc.hasCoprocessor(clazz))
return;
// add the constrain processor CP to the table
desc.addCoprocessor(clazz);
}
/**
* Turn off processing constraints for a given table, even if constraints have
* been turned on or added.
*
* @param desc
* {@link HTableDescriptor} where to disable {@link Constraint
* Constraints}.
*/
public static void disable(HTableDescriptor desc) {
desc.removeCoprocessor(ConstraintProcessor.class.getName());
}
/**
* Remove all {@link Constraint Constraints} that have been added to the table
* and turn off the constraint processing.
* <p>
* All {@link Configuration Configurations} and their associated
* {@link Constraint} are removed.
*
* @param desc
* {@link HTableDescriptor} to remove {@link Constraint Constraints}
* from.
*/
public static void remove(HTableDescriptor desc) {
// disable constraints
disable(desc);
// remove all the constraint settings
List<ImmutableBytesWritable> keys = new ArrayList<ImmutableBytesWritable>();
// loop through all the key, values looking for constraints
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : desc
.getValues().entrySet()) {
String key = Bytes.toString((e.getKey().get()));
String[] className = CONSTRAINT_HTD_ATTR_KEY_PATTERN.split(key);
if (className.length == 2) {
keys.add(e.getKey());
}
}
// now remove all the keys we found
for (ImmutableBytesWritable key : keys) {
desc.remove(key.get());
}
}
/**
* Check to see if the Constraint is currently set.
*
* @param desc
* {@link HTableDescriptor} to check
* @param clazz
* {@link Constraint} class to check for.
* @return <tt>true</tt> if the {@link Constraint} is present, even if it is
* disabled. <tt>false</tt> otherwise.
*/
public static boolean has(HTableDescriptor desc,
Class<? extends Constraint> clazz) {
return getKeyValueForClass(desc, clazz) != null;
}
/**
* Get the kv {@link Entry} in the descriptor for the specified class
*
* @param desc
* {@link HTableDescriptor} to read
* @param clazz
* to search for
* @return the {@link Pair} of <key, value> in the table, if that class is
* present. <tt>null</tt> otherwise.
*/
private static Pair<String, String> getKeyValueForClass(
HTableDescriptor desc, Class<? extends Constraint> clazz) {
// get the serialized version of the constraint
String key = serializeConstraintClass(clazz);
String value = desc.getValue(key);
if (value == null)
return null;
return new Pair<String, String>(key, value);
}
/**
* Add configuration-less constraints to the table.
* <p>
* This will overwrite any configuration associated with the previous
* constraint of the same class.
*
* @param desc
* {@link HTableDescriptor} to add {@link Constraint Constraints}
* @param constraints
* {@link Constraint Constraints} to add. All constraints are
* considered automatically enabled on add
* @throws IOException
* If constraint could not be serialized/added to table
*/
public static void add(HTableDescriptor desc,
Class<? extends Constraint>... constraints) throws IOException {
// make sure constraints are enabled
enable(desc);
long priority = getNextPriority(desc);
// store each constraint
for (Class<? extends Constraint> clazz : constraints) {
addConstraint(desc, clazz, null, priority++);
}
updateLatestPriority(desc, priority);
}
/**
* Add constraints and their associated configurations to the table.
* <p>
* Adding the same constraint class twice will overwrite the first
* constraint's configuration
*
* @param desc
* {@link HTableDescriptor} to add a {@link Constraint}
* @param constraints
* {@link Pair} of a {@link Constraint} and its associated
* {@link Configuration}. The Constraint will be configured on load
* with the specified configuration.All constraints are considered
* automatically enabled on add
* @throws IOException
* if any constraint could not be deserialized. Assumes if 1
* constraint is not loaded properly, something has gone terribly
* wrong and that all constraints need to be enforced.
*/
public static void add(HTableDescriptor desc,
Pair<Class<? extends Constraint>, Configuration>... constraints)
throws IOException {
enable(desc);
long priority = getNextPriority(desc);
for (Pair<Class<? extends Constraint>, Configuration> pair : constraints) {
addConstraint(desc, pair.getFirst(), pair.getSecond(), priority++);
}
updateLatestPriority(desc, priority);
}
/**
* Add a {@link Constraint} to the table with the given configuration
*
* @param desc
* table descriptor to the constraint to
* @param constraint
* to be added
* @param conf
* configuration associated with the constraint
* @throws IOException
* if any constraint could not be deserialized. Assumes if 1
* constraint is not loaded properly, something has gone terribly
* wrong and that all constraints need to be enforced.
*/
public static void add(HTableDescriptor desc,
Class<? extends Constraint> constraint, Configuration conf)
throws IOException {
enable(desc);
long priority = getNextPriority(desc);
addConstraint(desc, constraint, conf, priority++);
updateLatestPriority(desc, priority);
}
/**
* Write the raw constraint and configuration to the descriptor.
* <p>
* This method takes care of creating a new configuration based on the passed
* in configuration and then updating that with enabled and priority of the
* constraint.
* <p>
* When a constraint is added, it is automatically enabled.
*/
private static void addConstraint(HTableDescriptor desc,
Class<? extends Constraint> clazz, Configuration conf, long priority)
throws IOException {
writeConstraint(desc, serializeConstraintClass(clazz),
configure(conf, true, priority));
}
/**
* Setup the configuration for a constraint as to whether it is enabled and
* its priority
*
* @param conf
* on which to base the new configuration
* @param enabled
* <tt>true</tt> if it should be run
* @param priority
* relative to other constraints
* @returns a new configuration, storable in the {@link HTableDescriptor}
*/
private static Configuration configure(Configuration conf, boolean enabled,
long priority) {
// create the configuration to actually be stored
// clone if possible, but otherwise just create an empty configuration
Configuration toWrite = conf == null ? new Configuration()
: new Configuration(conf);
// update internal properties
toWrite.setBooleanIfUnset(ENABLED_KEY, enabled);
// set if unset long
if (toWrite.getLong(PRIORITY_KEY, UNSET_PRIORITY) == UNSET_PRIORITY) {
toWrite.setLong(PRIORITY_KEY, priority);
}
return toWrite;
}
/**
* Just write the class to the byte [] we are expecting
*
* @param clazz
* @return key to store in the {@link HTableDescriptor}
*/
private static String serializeConstraintClass(
Class<? extends Constraint> clazz) {
String constraintClazz = clazz.getName();
return CONSTRAINT_HTD_KEY_PREFIX + constraintClazz;
}
/**
* Write the given key and associated configuration to the
* {@link HTableDescriptor}
*/
private static void writeConstraint(HTableDescriptor desc, String key,
Configuration conf) throws IOException {
// store the key and conf in the descriptor
desc.setValue(key, serializeConfiguration(conf));
}
/**
* Write the configuration to a String
*
* @param conf
* to write
* @return String representation of that configuration
* @throws IOException
*/
private static String serializeConfiguration(Configuration conf)
throws IOException {
// write the configuration out to the data stream
ByteArrayOutputStream bos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(bos);
conf.writeXml(dos);
dos.flush();
byte[] data = bos.toByteArray();
return Bytes.toString(data);
}
/**
* Read the {@link Configuration} stored in the byte stream.
*
* @param bytes
* to read from
* @return A valid configuration
*/
private static Configuration readConfiguration(byte[] bytes)
throws IOException {
ByteArrayInputStream is = new ByteArrayInputStream(bytes);
Configuration conf = new Configuration();
conf.addResource(is);
return conf;
}
/**
* Read in the configuration from the String encoded configuration
*
* @param bytes
* to read from
* @return A valid configuration
* @throws IOException
* if the configuration could not be read
*/
private static Configuration readConfiguration(String bytes)
throws IOException {
return readConfiguration(Bytes.toBytes(bytes));
}
private static long getNextPriority(HTableDescriptor desc) {
String value = desc.getValue(COUNTER_KEY);
long priority;
// get the current priority
if (value == null) {
priority = MIN_PRIORITY;
} else {
priority = Long.parseLong(value) + 1;
}
return priority;
}
private static void updateLatestPriority(HTableDescriptor desc, long priority) {
// update the max priority
desc.setValue(COUNTER_KEY, Long.toString(priority));
}
/**
* Update the configuration for the {@link Constraint}. Does not change the
* order in which the constraint is run. If the
*
* @param desc
* {@link HTableDescriptor} to update
* @param clazz
* {@link Constraint} to update
* @param configuration
* to update the {@link Constraint} with.
* @throws IOException
* if the Constraint was not stored correctly
* @throws IllegalArgumentException
* if the Constraint was not present on this table.
*/
public static void setConfiguration(HTableDescriptor desc,
Class<? extends Constraint> clazz, Configuration configuration)
throws IOException, IllegalArgumentException {
// get the entry for this class
Pair<String, String> e = getKeyValueForClass(desc, clazz);
if (e == null)
throw new IllegalArgumentException("Constraint: " + clazz.getName()
+ " is not associated with this table.");
// clone over the configuration elements
Configuration conf = new Configuration(configuration);
// read in the previous info about the constraint
Configuration internal = readConfiguration(e.getSecond());
// update the fields based on the previous settings
conf.setIfUnset(ENABLED_KEY, internal.get(ENABLED_KEY));
conf.setIfUnset(PRIORITY_KEY, internal.get(PRIORITY_KEY));
// update the current value
writeConstraint(desc, e.getFirst(), conf);
}
/**
* Remove the constraint (and associated information) for the table
* descriptor.
*
* @param desc
* {@link HTableDescriptor} to modify
* @param clazz
* {@link Constraint} class to remove
*/
public static void remove(HTableDescriptor desc,
Class<? extends Constraint> clazz) {
String key = serializeConstraintClass(clazz);
desc.remove(key);
}
/**
* Enable the given {@link Constraint}. Retains all the information (e.g.
* Configuration) for the {@link Constraint}, but makes sure that it gets
* loaded on the table.
*
* @param desc
* {@link HTableDescriptor} to modify
* @param clazz
* {@link Constraint} to enable
* @throws IOException
* If the constraint cannot be properly deserialized
*/
public static void enableConstraint(HTableDescriptor desc,
Class<? extends Constraint> clazz) throws IOException {
changeConstraintEnabled(desc, clazz, true);
}
/**
* Disable the given {@link Constraint}. Retains all the information (e.g.
* Configuration) for the {@link Constraint}, but it just doesn't load the
* {@link Constraint} on the table.
*
* @param desc
* {@link HTableDescriptor} to modify
* @param clazz
* {@link Constraint} to disable.
* @throws IOException
* if the constraint cannot be found
*/
public static void disableConstraint(HTableDescriptor desc,
Class<? extends Constraint> clazz) throws IOException {
changeConstraintEnabled(desc, clazz, false);
}
/**
* Change the whether the constraint (if it is already present) is enabled or
* disabled.
*/
private static void changeConstraintEnabled(HTableDescriptor desc,
Class<? extends Constraint> clazz, boolean enabled) throws IOException {
// get the original constraint
Pair<String, String> entry = getKeyValueForClass(desc, clazz);
if (entry == null)
throw new IllegalArgumentException("Constraint: " + clazz.getName()
+ " is not associated with this table. You can't enable it!");
// create a new configuration from that conf
Configuration conf = readConfiguration(entry.getSecond());
// set that it is enabled
conf.setBoolean(ENABLED_KEY, enabled);
// write it back out
writeConstraint(desc, entry.getFirst(), conf);
}
/**
* Check to see if the given constraint is enabled.
*
* @param desc
* {@link HTableDescriptor} to check.
* @param clazz
* {@link Constraint} to check for
* @return <tt>true</tt> if the {@link Constraint} is present and enabled.
* <tt>false</tt> otherwise.
* @throws IOException
* If the constraint has improperly stored in the table
*/
public static boolean enabled(HTableDescriptor desc,
Class<? extends Constraint> clazz) throws IOException {
// get the kv
Pair<String, String> entry = getKeyValueForClass(desc, clazz);
// its not enabled so just return false. In fact, its not even present!
if (entry == null)
return false;
// get the info about the constraint
Configuration conf = readConfiguration(entry.getSecond());
return conf.getBoolean(ENABLED_KEY, false);
}
/**
* Get the constraints stored in the table descriptor
*
* @param desc
* To read from
* @param classloader
* To use when loading classes
* @return List of configured {@link Constraint Constraints}
* @throws IOException
* if any part of reading/arguments fails
*/
static List<? extends Constraint> getConstraints(HTableDescriptor desc,
ClassLoader classloader) throws IOException {
List<Constraint> constraints = new ArrayList<Constraint>();
// loop through all the key, values looking for constraints
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : desc
.getValues().entrySet()) {
// read out the constraint
String key = Bytes.toString(e.getKey().get()).trim();
String[] className = CONSTRAINT_HTD_ATTR_KEY_PATTERN.split(key);
if (className.length == 2) {
key = className[1];
if (LOG.isDebugEnabled()) {
LOG.debug("Loading constraint:" + key);
}
// read in the rest of the constraint
Configuration conf;
try {
conf = readConfiguration(e.getValue().get());
} catch (IOException e1) {
// long that we don't have a valid configuration stored, and move on.
LOG.warn("Corrupted configuration found for key:" + key
+ ", skipping it.");
continue;
}
// if it is not enabled, skip it
if (!conf.getBoolean(ENABLED_KEY, false)) {
if (LOG.isDebugEnabled())
LOG.debug("Constraint: " + key + " is DISABLED - skipping it");
// go to the next constraint
continue;
}
try {
// add the constraint, now that we expect it to be valid.
Class<? extends Constraint> clazz = classloader.loadClass(key)
.asSubclass(Constraint.class);
Constraint constraint = clazz.newInstance();
constraint.setConf(conf);
constraints.add(constraint);
} catch (ClassNotFoundException e1) {
throw new IOException(e1);
} catch (InstantiationException e1) {
throw new IOException(e1);
} catch (IllegalAccessException e1) {
throw new IOException(e1);
}
}
}
// sort them, based on the priorities
Collections.sort(constraints, constraintComparator);
return constraints;
}
private static final Comparator<Constraint> constraintComparator = new Comparator<Constraint>() {
@Override
public int compare(Constraint c1, Constraint c2) {
// compare the priorities of the constraints stored in their configuration
return Long.valueOf(c1.getConf().getLong(PRIORITY_KEY, DEFAULT_PRIORITY))
.compareTo(c2.getConf().getLong(PRIORITY_KEY, DEFAULT_PRIORITY));
}
};
}

View File

@ -0,0 +1,218 @@
/*
* 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.
*/
/**
* Restrict the domain of a data attribute, often times to fulfill business rules/requirements.
*
<p>
<h2> Table of Contents</h2>
<ul>
<li><a href="#overview">Overview</a></li>
<li><a href="#concurrency">Concurrency and Atomicity</a></li>
<li><a href="#caveats">Caveats</a></li>
<li><a href="#usage">Example Usage</a></li>
</ul>
</p>
<h2><a name="overview">Overview</a></h2>
Constraints are used to enforce business rules in a database.
By checking all {@link org.apache.hadoop.hbase.client.Put Puts} on a given table, you can enforce very specific data policies.
For instance, you can ensure that a certain column family-column qualifier pair always has a value between 1 and 10.
Otherwise, the {@link org.apache.hadoop.hbase.client.Put} is rejected and the data integrity is maintained.
<p>
Constraints are designed to be configurable, so a constraints can be used across different tables, but implement different behavior depending on the specific configuration given to that constraint.
<p>
By adding a constraint to a table (see <a href="#usage">Example Usage</a>), constraints will automatically enabled.
You also then have the option of disabling (just 'turn off') or remove (delete all associated information) constraints on a table.
If you remove all constraints (see {@link org.apache.hadoop.hbase.constraint.Constraints#remove(org.apache.hadoop.hbase.HTableDescriptor)}, you must re-add any {@link org.apache.hadoop.hbase.constraint.Constraint} you want on that table.
However, if they are just disabled (see {@link org.apache.hadoop.hbase.constraint.Constraints#disable(org.apache.hadoop.hbase.HTableDescriptor)}, all you need to do is enable constraints again, and everything will be turned back on.
Individual constraints can also be individually enabled, disabled or removed without affecting others.
<p>
By default, constraints are disabled on a table.
This means you will not see <i>any</i> slow down on a table if constraints are not enabled.
<p>
<b>NOTES</b>
<ol>
<li>Constraints are run in the order that they are added to a table. This has implications for what order constraints should be added to a table.</li>
<li>There are certain keys that are reserved for the Configuration namespace:
<ul>
<li>_ENABLED - used server-side to determine if a constraint should be run</li>
<li>_PRIORITY - used server-side to determine what order a constraint should be run</li>
</ul>
If these items are set, they will be respected in the constraint configuration, but they are taken care of by default in when adding constraints to an {@link org.apache.hadoop.hbase.HTableDescriptor} via the usual method.</li>
</ol>
<p>
Under the hood, constraints are implemented as a Coprocessor (see {@link org.apache.hadoop.hbase.constraint.ConstraintProcessor} if you are interested).
<h2><a name="concurrency">Concurrency and Atomicity</a></h2>
Currently, no attempts at enforcing correctness in a multi-threaded scenario when modifying a constraint, via {@link org.apache.hadoop.hbase.constraint.Constraints}, to the the {@link org.apache.hadoop.hbase.HTableDescriptor}.
This is particularly important when adding a constraint(s) to the descriptor as it first retrieves the next priority from a custom value set in the descriptor,
adds each constraint (with increasing priority) to the descriptor, and then the next available priority is re-stored back in the {@link org.apache.hadoop.hbase.HTableDescriptor}.
Locking is recommended around each of Constraints add methods: {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class...)},
{@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, org.apache.hadoop.hbase.util.Pair...)}, and {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class, org.apache.hadoop.conf.Configuration)}.
Any changes on <i>a single HTableDescriptor</i> should be serialized, either within a single thread or via external mechanisms.
<p>
Note that having a higher priority means that a constraint will run later; e.g. a constraint with priority 1 will run before a constraint with priority 2.
<p>
Since Constraints currently are designed to just implement simple checks (e.g. is the value in the right range), there will be no atomicity conflicts.
Even if one of the puts finishes the constraint first, the single row will not be corrupted and the 'fastest' write will win; the underlying region takes care of breaking the tie and ensuring that writes get serialized to the table.
So yes, this doesn't ensure that we are going to get specific ordering or even a fully consistent view of the underlying data.
<p>
Each constraint should only use local/instance variables, unless doing more advanced usage. Static variables could cause difficulties when checking concurrent writes to the same region, leading to either highly locked situations (decreasing throughtput) or higher probability of errors.
However, as long as each constraint just uses local variables, each thread interacting with the constraint will be completely fine.
<h2><a name="caveats">Caveats</a></h2>
In traditional (SQL) databases, Constraints are often used to enforce <a href="http://en.wikipedia.org/wiki/Relational_database#Constraints">referential integrity</a>.
However, in HBase, this will likely cause significant overhead and dramatically decrease the number of {@link org.apache.hadoop.hbase.client.Put Puts}/second possible on a table.
This is because to check the referential integrity when making a {@link org.apache.hadoop.hbase.client.Put}, one must block on a scan for the 'remote' table, checking for the valid reference.
For millions of {@link org.apache.hadoop.hbase.client.Put Puts} a second, this will breakdown very quickly.
There are several options around the blocking behavior including, but not limited to:
<ul>
<li>Create a 'pre-join' table where the keys are already denormalized</li>
<li>Designing for 'incorrect' references</li>
<li>Using an external enforcement mechanism</li>
</ul>
<h2><a name="usage">Example usage</a></h2>
First, you must define a {@link org.apache.hadoop.hbase.constraint.Constraint}.
The best way to do this is to extend {@link org.apache.hadoop.hbase.constraint.BaseConstraint}, which takes care of some of the more mundane details of using a {@link org.apache.hadoop.hbase.constraint.Constraint}.
<p>
Let's look at one possible implementation of a constraint - an IntegerConstraint(there are also several simple examples in the tests).
The IntegerConstraint checks to make sure that the value is a String-encoded <code>int</code>.
It is really simple to implement this kind of constraint, the only method needs to be implemented is {@link org.apache.hadoop.hbase.constraint.Constraint#check(org.apache.hadoop.hbase.client.Put)}:
<div style="background-color: #cccccc; padding: 2px">
<blockquote><pre>
public class IntegerConstraint extends BaseConstraint {
public void check(Put p) throws ConstraintException {
Map&ltbyte[], List&ltKeyValue&gt&gt familyMap = p.getFamilyMap();
for (List &ltKeyValue&gt kvs : familyMap.values()) {
for (KeyValue kv : kvs) {
// just make sure that we can actually pull out an int
// this will automatically throw a NumberFormatException if we try to
// store something that isn't an Integer.
try {
Integer.parseInt(new String(kv.getValue()));
} catch (NumberFormatException e) {
throw new ConstraintException("Value in Put (" + p
+ ") was not a String-encoded integer", e);
} } }
</pre></blockquote>
</div>
<p>
Note that all exceptions that you expect to be thrown must be caught and then rethrown as a {@link org.apache.hadoop.hbase.constraint.ConstraintException}.
This way, you can be sue that a {@link org.apache.hadoop.hbase.client.Put} fails for an expected reason, rather than for any reason.
For example, an {@link java.lang.OutOfMemoryError} is probably indicative of an inherent problem in the {@link org.apache.hadoop.hbase.constraint.Constraint}, rather than a failed {@link org.apache.hadoop.hbase.client.Put}.
<p>
If an unexpected exception is thrown (for example, any kind of uncaught {@link java.lang.RuntimeException}), constraint-checking will be 'unloaded' from the regionserver where that error occurred.
This means no further {@link org.apache.hadoop.hbase.constraint.Constraint Constraints} will be checked on that server until it is reloaded. This is done to ensure the system remains as available as possible.
Therefore, be careful when writing your own Constraint.
<p>
So now that we have a Constraint, we want to add it to a table. It's as easy as:
<div style="background-color: #cccccc; padding: 2px">
<blockquote><pre>
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
...
Constraints.add(desc, IntegerConstraint.class);
</pre></blockquote></div>
<p>
Once we added the IntegerConstraint, constraints will be enabled on the table (once it is created) and we will always check to make sure that the value is an String-encoded integer.
<p>
However, suppose we also write our own constraint, <code>MyConstraint.java</code>.
First, you need to make sure this class-files are in the classpath (in a jar) on the regionserver where that constraint will be run.
<p>
Suppose that MyConstraint also uses a Configuration (see {@link org.apache.hadoop.hbase.constraint.Constraint#getConf()}).
Then adding MyConstraint looks like this:
<div style="background-color: #cccccc; padding: 2px">
<blockquote><pre>
HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
Configuration conf = new Configuration();
...
(add values to the conf)
(modify the table descriptor)
...
Constraints.add(desc, new Pair(MyConstraint.class, conf));
</pre></blockquote></div>
<p>
At this point we added both the IntegerConstraint and MyConstraint to the table, the IntegerConstraint <i>will be run first</i>, followed by MyConstraint.
<p>
Suppose we realize that the {@link org.apache.hadoop.conf.Configuration} for MyConstraint is actually wrong when it was added to the table. Note, when it is added to the table, it is <i>not</i> added by reference, but is instead copied into the {@link org.apache.hadoop.hbase.HTableDescriptor}.
Thus, to change the {@link org.apache.hadoop.conf.Configuration} we are using for MyConstraint, we need to do this:
<div style="background-color: #cccccc; padding: 2px">
<blockquote><pre>
(add/modify the conf)
...
Constraints.setConfiguration(desc, MyConstraint.class, conf);
</pre></blockquote></div>
<p>
This will overwrite the previous configuration for MyConstraint, but <i>not</i> change the order of the constraint nor if it is enabled/disabled.
<p>
Note that the same constraint class can be added multiple times to a table without repercussion.
A use case for this is the same constraint working differently based on its configuration.
<p>
Suppose then we want to disable <i>just</i> MyConstraint. Its as easy as:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.disable(desc, MyConstraint.class);
</pre></blockquote></div>
<p>
This just turns off MyConstraint, but retains the position and the configuration associated with MyConstraint.
Now, if we want to re-enable the constraint, its just another one-liner:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.enable(desc, MyConstraint.class);
</pre></blockquote></div>
<p>
Similarly, constraints on the entire table are disabled via:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.disable(desc);
</pre></blockquote></div>
<p>
Or enabled via:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.enable(desc);
</pre></blockquote></div>
<p>
Lastly, suppose you want to remove MyConstraint from the table, including with position it should be run at and its configuration.
This is similarly simple:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.remove(desc, MyConstraint.class);
</pre></blockquote></div>
<p>
Also, removing <i>all</i> constraints from a table is similarly simple:
<div style="background-color: #cccccc">
<blockquote><pre>
Constraints.remove(desc);
</pre></blockquote></div>
This will remove all constraints (and associated information) from the table and turn off the constraint processing.
*/
package org.apache.hadoop.hbase.constraint;

View File

@ -0,0 +1,49 @@
package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test setting values in the descriptor
*/
@Category(SmallTests.class)
public class TestHTableDescriptor {
/**
* Test cps in the table description
* @throws Exception
*/
@Test
public void testGetSetRemoveCP() throws Exception {
HTableDescriptor desc = new HTableDescriptor("table");
// simple CP
String className = BaseRegionObserver.class.getName();
// add and check that it is present
desc.addCoprocessor(className);
assertTrue(desc.hasCoprocessor(className));
// remove it and check that it is gone
desc.removeCoprocessor(className);
assertFalse(desc.hasCoprocessor(className));
}
/**
* Test that we add and remove strings from settings properly.
* @throws Exception
*/
@Test
public void testRemoveString() throws Exception {
HTableDescriptor desc = new HTableDescriptor("table");
String key = "Some";
String value = "value";
desc.setValue(key, value);
assertEquals(value, desc.getValue(key));
desc.remove(key);
assertEquals(null, desc.getValue(key));
}
}

View File

@ -0,0 +1,31 @@
/**
* 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.constraint;
import org.apache.hadoop.hbase.client.Put;
/**
* Always fail the put.
*/
public class AllFailConstraint extends BaseConstraint {
@Override
public void check(Put p) throws ConstraintException {
throw new ConstraintException("AllFailConstraint fails for all puts");
}
}

View File

@ -0,0 +1,32 @@
/**
* 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.constraint;
import org.apache.hadoop.hbase.client.Put;
/**
* Simple test constraint that always allows the put to pass
*/
public class AllPassConstraint extends BaseConstraint {
@Override
public void check(Put p) {
// Do nothing - it passes
}
}

View File

@ -0,0 +1,52 @@
/**
* 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.constraint;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Put;
/**
* Test Constraint to check to make sure the configuration is set
*/
public class CheckConfigurationConstraint extends BaseConstraint {
private static String key = "testKey";
private static String value = "testValue";
public static Configuration getConfiguration() {
Configuration conf = new Configuration();
conf.set(key, value);
return conf;
}
@Override
public void check(Put p) {
// NOOP
}
@Override
public void setConf(Configuration conf) {
String val = conf.get(key);
if (val == null || !val.equals(value))
throw new IllegalArgumentException(
"Configuration was not passed correctly");
super.setConf(conf);
}
}

View File

@ -0,0 +1,16 @@
package org.apache.hadoop.hbase.constraint;
import org.apache.hadoop.hbase.client.Put;
/**
* Always non-gracefully fail on attempt
*/
public class RuntimeFailConstraint extends BaseConstraint {
@Override
public void check(Put p) throws ConstraintException {
throw new RuntimeException(
"RuntimeFailConstraint always throws a runtime exception");
}
}

View File

@ -0,0 +1,260 @@
/**
* 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.constraint;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Do the complex testing of constraints against a minicluster
*/
@Category(MediumTests.class)
public class TestConstraint {
private static final Log LOG = LogFactory
.getLog(TestConstraint.class);
private static HBaseTestingUtility util;
private static final byte[] tableName = Bytes.toBytes("test");
private static final byte[] dummy = Bytes.toBytes("dummy");
private static final byte[] row1 = Bytes.toBytes("r1");
private static final byte[] test = Bytes.toBytes("test");
@BeforeClass
public static void setUpBeforeClass() throws Exception {
util = new HBaseTestingUtility();
util.startMiniCluster();
}
/**
* Test that we run a passing constraint
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Test
public void testConstraintPasses() throws Exception {
// create the table
// it would be nice if this was also a method on the util
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : new byte[][] { dummy, test }) {
desc.addFamily(new HColumnDescriptor(family));
}
// add a constraint
Constraints.add(desc, CheckWasRunConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
table.setAutoFlush(true);
// test that we don't fail on a valid put
Put put = new Put(row1);
byte[] value = Integer.toString(10).getBytes();
put.add(dummy, new byte[0], value);
table.put(put);
assertTrue(CheckWasRunConstraint.wasRun);
}
/**
* Test that constraints will fail properly
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Test(timeout = 10000)
public void testConstraintFails() throws Exception {
// create the table
// it would be nice if this was also a method on the util
HTableDescriptor desc = new HTableDescriptor(tableName);
for (byte[] family : new byte[][] { dummy, test }) {
desc.addFamily(new HColumnDescriptor(family));
}
// add a constraint that is sure to fail
Constraints.add(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
table.setAutoFlush(true);
// test that we do fail on violation
Put put = new Put(row1);
put.add(dummy, new byte[0], "fail".getBytes());
LOG.warn("Doing put in table");
try {
table.put(put);
fail("This put should not have suceeded - AllFailConstraint was not run!");
} catch (RetriesExhaustedWithDetailsException e) {
List<Throwable> causes = e.getCauses();
assertEquals(
"More than one failure cause - should only be the failure constraint exception",
1, causes.size());
Throwable t = causes.get(0);
assertEquals(ConstraintException.class, t.getClass());
}
}
/**
* Check that if we just disable one constraint, then
* @throws Throwable
*/
@SuppressWarnings("unchecked")
@Test
public void testDisableConstraint() throws Throwable {
// create the table
HTableDescriptor desc = new HTableDescriptor(tableName);
// add a family to the table
for (byte[] family : new byte[][] { dummy, test }) {
desc.addFamily(new HColumnDescriptor(family));
}
// add a constraint to make sure it others get run
Constraints.add(desc, CheckWasRunConstraint.class);
// Add Constraint to check
Constraints.add(desc, AllFailConstraint.class);
// and then disable the failing constraint
Constraints.disableConstraint(desc, AllFailConstraint.class);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
table.setAutoFlush(true);
// test that we don't fail because its disabled
Put put = new Put(row1);
put.add(dummy, new byte[0], "pass".getBytes());
table.put(put);
assertTrue(CheckWasRunConstraint.wasRun);
}
/**
* Test that if we disable all constraints, then nothing gets run
* @throws Throwable
*/
@SuppressWarnings("unchecked")
@Test
public void testDisableConstraints() throws Throwable {
// create the table
HTableDescriptor desc = new HTableDescriptor(tableName);
// add a family to the table
for (byte[] family : new byte[][] { dummy, test }) {
desc.addFamily(new HColumnDescriptor(family));
}
// add a constraint to check to see if is run
Constraints.add(desc, CheckWasRunConstraint.class);
// then disable all the constraints
Constraints.disable(desc);
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
table.setAutoFlush(true);
// test that we do fail on violation
Put put = new Put(row1);
put.add(dummy, new byte[0], "pass".getBytes());
LOG.warn("Doing put in table");
table.put(put);
assertFalse(CheckWasRunConstraint.wasRun);
}
/**
* Check to make sure a constraint is unloaded when it fails
* @throws Exception
*/
@Test
public void testIsUnloaded() throws Exception {
// create the table
HTableDescriptor desc = new HTableDescriptor(tableName);
// add a family to the table
for (byte[] family : new byte[][] { dummy, test }) {
desc.addFamily(new HColumnDescriptor(family));
}
// make sure that constraints are unloaded
Constraints.add(desc, RuntimeFailConstraint.class);
// add a constraint to check to see if is run
Constraints.add(desc, CheckWasRunConstraint.class);
CheckWasRunConstraint.wasRun = false;
util.getHBaseAdmin().createTable(desc);
HTable table = new HTable(util.getConfiguration(), tableName);
table.setAutoFlush(true);
// test that we do fail on violation
Put put = new Put(row1);
put.add(dummy, new byte[0], "pass".getBytes());
try{
table.put(put);
fail("RuntimeFailConstraint wasn't triggered - this put shouldn't work!");
} catch (Exception e) {// NOOP
}
// try the put again, this time constraints are not used, so it works
table.put(put);
// and we make sure that constraints were not run...
assertFalse(CheckWasRunConstraint.wasRun);
}
@After
public void cleanup() throws Exception {
// cleanup
CheckWasRunConstraint.wasRun = false;
util.getHBaseAdmin().disableTable(tableName);
util.getHBaseAdmin().deleteTable(tableName);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
util.shutdownMiniCluster();
}
/**
* Constraint to check that it was actually run (or not)
*/
public static class CheckWasRunConstraint extends BaseConstraint {
public static boolean wasRun = false;
@Override
public void check(Put p) {
wasRun = true;
}
}
}

View File

@ -0,0 +1,205 @@
/**
* 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.constraint;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.constraint.TestConstraint.CheckWasRunConstraint;
import org.apache.hadoop.hbase.constraint.WorksConstraint.NameConstraint;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test reading/writing the constraints into the {@link HTableDescriptor}
*/
@Category(SmallTests.class)
public class TestConstraints {
@SuppressWarnings("unchecked")
@Test
public void testSimpleReadWrite() throws Throwable {
HTableDescriptor desc = new HTableDescriptor("table");
Constraints.add(desc, WorksConstraint.class);
List<? extends Constraint> constraints = Constraints.getConstraints(desc,
this.getClass().getClassLoader());
assertEquals(1, constraints.size());
assertEquals(WorksConstraint.class, constraints.get(0).getClass());
// Check that we can add more than 1 constraint and that ordering is
// preserved
Constraints.add(desc, AlsoWorks.class, NameConstraint.class);
constraints = Constraints.getConstraints(desc, this.getClass()
.getClassLoader());
assertEquals(3, constraints.size());
assertEquals(WorksConstraint.class, constraints.get(0).getClass());
assertEquals(AlsoWorks.class, constraints.get(1).getClass());
assertEquals(NameConstraint.class, constraints.get(2).getClass());
}
@SuppressWarnings("unchecked")
@Test
public void testReadWriteWithConf() throws Throwable {
HTableDescriptor desc = new HTableDescriptor("table");
Constraints.add(
desc,
new Pair<Class<? extends Constraint>, Configuration>(
CheckConfigurationConstraint.class, CheckConfigurationConstraint
.getConfiguration()));
List<? extends Constraint> c = Constraints.getConstraints(desc, this
.getClass().getClassLoader());
assertEquals(1, c.size());
assertEquals(CheckConfigurationConstraint.class, c.get(0).getClass());
// check to make sure that we overwrite configurations
Constraints.add(desc, new Pair<Class<? extends Constraint>, Configuration>(
CheckConfigurationConstraint.class, new Configuration()));
try {
Constraints.getConstraints(desc, this.getClass().getClassLoader());
assertTrue("No exception thrown - configuration not overwritten", false);
} catch (IllegalArgumentException e) {
// expect to have the exception, so don't do anything
}
}
/**
* Test that Constraints are properly enabled, disabled, and removed
*
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Test
public void testEnableDisableRemove() throws Exception {
HTableDescriptor desc = new HTableDescriptor("table");
// check general enabling/disabling of constraints
// first add a constraint
Constraints.add(desc, AllPassConstraint.class);
// make sure everything is enabled
assertTrue(Constraints.enabled(desc, AllPassConstraint.class));
assertTrue(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
// check disabling
Constraints.disable(desc);
assertFalse(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
// make sure the added constraints are still present
assertTrue(Constraints.enabled(desc, AllPassConstraint.class));
// check just removing the single constraint
Constraints.remove(desc, AllPassConstraint.class);
assertFalse(Constraints.has(desc, AllPassConstraint.class));
// Add back the single constraint
Constraints.add(desc, AllPassConstraint.class);
// and now check that when we remove constraints, all are gone
Constraints.remove(desc);
assertFalse(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
assertFalse(Constraints.has(desc, AllPassConstraint.class));
}
/**
* Test that when we update a constraint the ordering is not modified.
*
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Test
public void testUpdateConstraint() throws Exception {
HTableDescriptor desc = new HTableDescriptor("table");
Constraints.add(desc, CheckConfigurationConstraint.class,
CheckWasRunConstraint.class);
Constraints.setConfiguration(desc, CheckConfigurationConstraint.class,
CheckConfigurationConstraint.getConfiguration());
List<? extends Constraint> constraints = Constraints.getConstraints(desc,
this.getClass().getClassLoader());
assertEquals(2, constraints.size());
// check to make sure the order didn't change
assertEquals(CheckConfigurationConstraint.class, constraints.get(0)
.getClass());
assertEquals(CheckWasRunConstraint.class, constraints.get(1).getClass());
}
/**
* Test that if a constraint hasn't been set that there are no problems with
* attempting to remove it.
*
* @throws Throwable
* on failure.
*/
@Test
public void testRemoveUnsetConstraint() throws Throwable {
HTableDescriptor desc = new HTableDescriptor("table");
Constraints.remove(desc);
Constraints.remove(desc, AlsoWorks.class);
}
@Test
public void testConfigurationPreserved() throws Throwable {
Configuration conf = new Configuration();
conf.setBoolean("_ENABLED", false);
conf.setLong("_PRIORITY", 10);
HTableDescriptor desc = new HTableDescriptor("table");
Constraints.add(desc, AlsoWorks.class, conf);
Constraints.add(desc, WorksConstraint.class);
assertFalse(Constraints.enabled(desc, AlsoWorks.class));
List<? extends Constraint> constraints = Constraints.getConstraints(desc,
this.getClass().getClassLoader());
for (Constraint c : constraints) {
Configuration storedConf = c.getConf();
if (c instanceof AlsoWorks)
assertEquals(10, storedConf.getLong("_PRIORITY", -1));
// its just a worksconstraint
else
assertEquals(2, storedConf.getLong("_PRIORITY", -1));
}
}
// ---------- Constraints just used for testing
/**
* Also just works
*/
public static class AlsoWorks extends BaseConstraint {
@Override
public void check(Put p) {
// NOOP
}
}
}

View File

@ -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.constraint;
import org.apache.hadoop.hbase.client.Put;
/**
* It just works
*/
public class WorksConstraint extends BaseConstraint {
@Override
public void check(Put p) {
// NOOP
}
/**
* Constraint to check that the naming of constraints doesn't mess up the
* pattern matching.(that constraint $___Constraint$NameConstraint isn't a
* problem)
*/
public static class NameConstraint extends WorksConstraint {
}
}