You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/08/27 07:01:07 UTC

svn commit: r990018 [6/10] - in /hbase/branches/0.90_master_rewrite: ./ bin/ bin/replication/ src/assembly/ src/docbkx/ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/filter/ s...

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceMetrics.java?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceMetrics.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceMetrics.java Fri Aug 27 05:01:02 2010
@@ -0,0 +1,100 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+import org.apache.hadoop.hbase.metrics.MetricsRate;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.Updater;
+import org.apache.hadoop.metrics.jvm.JvmMetrics;
+import org.apache.hadoop.metrics.util.MetricsIntValue;
+import org.apache.hadoop.metrics.util.MetricsLongValue;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+
+/**
+ * This class is for maintaining the various replication statistics
+ * for a source and publishing them through the metrics interfaces.
+ */
+public class ReplicationSourceMetrics implements Updater {
+  private final MetricsRecord metricsRecord;
+  private MetricsRegistry registry = new MetricsRegistry();
+
+  /** Rate of shipped operations by the source */
+  public final MetricsRate shippedOpsRate =
+      new MetricsRate("shippedOpsRate", registry);
+
+  /** Rate of shipped batches by the source */
+  public final MetricsRate shippedBatchesRate =
+      new MetricsRate("shippedBatchesRate", registry);
+
+  /** Rate of log entries (can be multiple Puts) read from the logs */
+  public final MetricsRate logEditsReadRate =
+      new MetricsRate("logEditsReadRate", registry);
+
+  /** Rate of log entries filtered by the source */
+  public final MetricsRate logEditsFilteredRate =
+      new MetricsRate("logEditsFilteredRate", registry);
+
+  /** Age of the last operation that was shipped by the source */
+  private final MetricsLongValue ageOfLastShippedOp =
+      new MetricsLongValue("ageOfLastShippedOp", registry);
+
+  /**
+   * Current size of the queue of logs to replicate,
+   * excluding the one being processed at the moment
+   */
+  public final MetricsIntValue sizeOfLogQueue =
+      new MetricsIntValue("sizeOfLogQueue", registry);
+
+  /**
+   * Constructor used to register the metrics
+   * @param id Name of the source this class is monitoring
+   */
+  public ReplicationSourceMetrics(String id) {
+    MetricsContext context = MetricsUtil.getContext("hbase");
+    String name = Thread.currentThread().getName();
+    metricsRecord = MetricsUtil.createRecord(context, "replication");
+    metricsRecord.setTag("RegionServer", name);
+    context.registerUpdater(this);
+    // export for JMX
+    new ReplicationStatistics(this.registry, "ReplicationSource for " + id);
+  }
+
+  /**
+   * Set the age of the last edit that was shipped
+   * @param timestamp write time of the edit
+   */
+  public void setAgeOfLastShippedOp(long timestamp) {
+    ageOfLastShippedOp.set(System.currentTimeMillis() - timestamp);
+  }
+
+  @Override
+  public void doUpdates(MetricsContext metricsContext) {
+    synchronized (this) {
+      this.shippedOpsRate.pushMetric(this.metricsRecord);
+      this.shippedBatchesRate.pushMetric(this.metricsRecord);
+      this.logEditsReadRate.pushMetric(this.metricsRecord);
+      this.logEditsFilteredRate.pushMetric(this.metricsRecord);
+      this.ageOfLastShippedOp.pushMetric(this.metricsRecord);
+      this.sizeOfLogQueue.pushMetric(this.metricsRecord);
+    }
+    this.metricsRecord.update();
+  }
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationStatistics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationStatistics.java?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationStatistics.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationStatistics.java Fri Aug 27 05:01:02 2010
@@ -0,0 +1,45 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import org.apache.hadoop.hbase.metrics.MetricsMBeanBase;
+import org.apache.hadoop.metrics.util.MBeanUtil;
+import org.apache.hadoop.metrics.util.MetricsRegistry;
+
+import javax.management.ObjectName;
+
+/**
+ * Exports metrics recorded by {@link ReplicationSourceMetrics} as an MBean
+ * for JMX monitoring.
+ */
+public class ReplicationStatistics extends MetricsMBeanBase {
+
+  private final ObjectName mbeanName;
+
+  /**
+   * Constructor to register the MBean
+   * @param registry which rehistry to use
+   * @param name name to get to this bean
+   */
+  public ReplicationStatistics(MetricsRegistry registry, String name) {
+    super(registry, name);
+    mbeanName = MBeanUtil.registerMBean("Replication", name, this);
+  }
+}

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/Main.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/Main.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/Main.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/Main.java Fri Aug 27 05:01:02 2010
@@ -22,9 +22,16 @@ package org.apache.hadoop.hbase.rest;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.Arrays;
+import java.util.List;
+
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.ServletHolder;
@@ -40,17 +47,43 @@ import com.sun.jersey.spi.container.serv
  * </ul>
  */
 public class Main implements Constants {
+  private static final String DEFAULT_LISTEN_PORT = "8080";
 
-  public static void main(String[] args) throws Exception {
-    // process command line
+  private static void printUsageAndExit(Options options, int exitCode) {
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("REST", null, options,
+      "To start the REST server run 'bin/hbase-daemon.sh start rest'\n" +
+      "To shutdown the REST server run 'bin/hbase-daemon.sh stop rest' or" +
+      " send a kill signal to the rest server pid",
+      true);
+    System.exit(exitCode);
+  }
 
+  public static void main(String[] args) throws Exception {
+    Log LOG = LogFactory.getLog("RESTServer");
     Options options = new Options();
-    options.addOption("p", "port", true, "service port");
+    options.addOption("p", "port", true, "Port to bind to [default:" +
+      DEFAULT_LISTEN_PORT + "]");
     CommandLineParser parser = new PosixParser();
     CommandLine cmd = parser.parse(options, args);
-    int port = 8080;
-    if (cmd.hasOption("p")) {
-      port = Integer.valueOf(cmd.getOptionValue("p"));
+    /**
+     * This is so complicated to please both bin/hbase and bin/hbase-daemon.
+     * hbase-daemon provides "start" and "stop" arguments
+     * hbase should print the help if no argument is provided
+     */
+    List<String> commandLine = Arrays.asList(args);
+    boolean stop = commandLine.contains("stop");
+    boolean start = commandLine.contains("start");
+    if (cmd.hasOption("help") || !start || stop) {
+      printUsageAndExit(options, 1);
+    }
+    // Get port to bind to
+    int port = 0;
+    try {
+      port = Integer.parseInt(cmd.getOptionValue("port", DEFAULT_LISTEN_PORT));
+    } catch (NumberFormatException e) {
+      LOG.error("Could not parse the value provided for the port option", e);
+      printUsageAndExit(options, -1);
     }
 
     // set up the Jersey servlet container for Jetty

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/RowResultGenerator.java Fri Aug 27 05:01:02 2010
@@ -24,6 +24,10 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
@@ -31,8 +35,11 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 
 public class RowResultGenerator extends ResultGenerator {
+  private static final Log LOG = LogFactory.getLog(RowResultGenerator.class);
+
   private Iterator<KeyValue> valuesI;
   private KeyValue cache;
 
@@ -67,6 +74,14 @@ public class RowResultGenerator extends 
       if (result != null && !result.isEmpty()) {
         valuesI = result.list().iterator();
       }
+    } catch (NoSuchColumnFamilyException e) {
+      // Warn here because Stargate will return 404 in the case if multiple
+      // column families were specified but one did not exist -- currently
+      // HBase will fail the whole Get.
+      // Specifying multiple columns in a URI should be uncommon usage but
+      // help to avoid confusion by leaving a record of what happened here in
+      // the log.
+      LOG.warn(StringUtils.stringifyException(e));
     } finally {
       pool.putTable(table);
     }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java Fri Aug 27 05:01:02 2010
@@ -143,7 +143,7 @@ public class Client {
     int code = httpClient.executeMethod(method);
     long endTime = System.currentTimeMillis();
     if (LOG.isDebugEnabled()) {
-      LOG.debug(method.getName() + " " + uri + ": " + code + " " +
+      LOG.debug(method.getName() + " " + uri + " " + code + " " +
         method.getStatusText() + " in " + (endTime - startTime) + " ms");
     }
     return code;

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java Fri Aug 27 05:01:02 2010
@@ -24,8 +24,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
-import java.util.Map;
-import java.util.WeakHashMap;
 
 import javax.ws.rs.Produces;
 import javax.ws.rs.WebApplicationException;
@@ -47,7 +45,7 @@ import org.apache.hadoop.hbase.rest.Cons
 public class PlainTextMessageBodyProducer 
   implements MessageBodyWriter<Object> {
 
-  private Map<Object, byte[]> buffer = new WeakHashMap<Object, byte[]>();
+  private ThreadLocal<byte[]> buffer = new ThreadLocal<byte[]>();
 
   @Override
   public boolean isWriteable(Class<?> arg0, Type arg1, Annotation[] arg2,
@@ -58,9 +56,9 @@ public class PlainTextMessageBodyProduce
 	@Override
 	public long getSize(Object object, Class<?> type, Type genericType,
 			Annotation[] annotations, MediaType mediaType) {
-	  byte[] bytes = object.toString().getBytes(); 
-	  buffer.put(object, bytes);
-		return bytes.length;
+    byte[] bytes = object.toString().getBytes(); 
+	  buffer.set(bytes);
+    return bytes.length;
 	}
 
 	@Override
@@ -68,6 +66,8 @@ public class PlainTextMessageBodyProduce
 			Annotation[] annotations, MediaType mediaType,
 			MultivaluedMap<String, Object> httpHeaders, OutputStream outStream)
 			throws IOException, WebApplicationException {
-		outStream.write(buffer.remove(object));
+    byte[] bytes = buffer.get();
+		outStream.write(bytes);
+    buffer.remove();
 	}	
 }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java Fri Aug 27 05:01:02 2010
@@ -25,8 +25,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Type;
-import java.util.Map;
-import java.util.WeakHashMap;
 
 import javax.ws.rs.Produces;
 import javax.ws.rs.WebApplicationException;
@@ -49,7 +47,7 @@ import org.apache.hadoop.hbase.rest.Prot
 public class ProtobufMessageBodyProducer
   implements MessageBodyWriter<ProtobufMessageHandler> {
 
-  private Map<Object, byte[]> buffer = new WeakHashMap<Object, byte[]>();
+  private ThreadLocal<byte[]> buffer = new ThreadLocal<byte[]>();
 
 	@Override
 	public boolean isWriteable(Class<?> type, Type genericType, 
@@ -67,7 +65,7 @@ public class ProtobufMessageBodyProducer
 	    return -1;
 	  }
 	  byte[] bytes = baos.toByteArray();
-	  buffer.put(m, bytes);
+	  buffer.set(bytes);
 	  return bytes.length;
 	}
 
@@ -75,6 +73,8 @@ public class ProtobufMessageBodyProducer
 	    Annotation[] annotations, MediaType mediaType, 
 	    MultivaluedMap<String, Object> httpHeaders, OutputStream entityStream) 
 	    throws IOException, WebApplicationException {
-	  entityStream.write(buffer.remove(m));
+    byte[] bytes = buffer.get();
+	  entityStream.write(bytes);
+    buffer.remove();
 	}
 }

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Fri Aug 27 05:01:02 2010
@@ -843,27 +843,27 @@ public class Bytes {
   /**
    * Lexographically compare two arrays.
    *
-   * @param b1 left operand
-   * @param b2 right operand
-   * @param s1 Where to start comparing in the left buffer
-   * @param s2 Where to start comparing in the right buffer
-   * @param l1 How much to compare from the left buffer
-   * @param l2 How much to compare from the right buffer
+   * @param buffer1 left operand
+   * @param buffer2 right operand
+   * @param offset1 Where to start comparing in the left buffer
+   * @param offset2 Where to start comparing in the right buffer
+   * @param length1 How much to compare from the left buffer
+   * @param length2 How much to compare from the right buffer
    * @return 0 if equal, < 0 if left is less than right, etc.
    */
-  public static int compareTo(byte[] b1, int s1, int l1,
-      byte[] b2, int s2, int l2) {
+  public static int compareTo(byte[] buffer1, int offset1, int length1,
+      byte[] buffer2, int offset2, int length2) {
     // Bring WritableComparator code local
-    int end1 = s1 + l1;
-    int end2 = s2 + l2;
-    for (int i = s1, j = s2; i < end1 && j < end2; i++, j++) {
-      int a = (b1[i] & 0xff);
-      int b = (b2[j] & 0xff);
+    int end1 = offset1 + length1;
+    int end2 = offset2 + length2;
+    for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) {
+      int a = (buffer1[i] & 0xff);
+      int b = (buffer2[j] & 0xff);
       if (a != b) {
         return a - b;
       }
     }
-    return l1 - l2;
+    return length1 - length2;
   }
 
   /**

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/ManualEnvironmentEdge.java Fri Aug 27 05:01:02 2010
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+/**
+ * An environment edge that uses a manually set value. This is useful for testing events that are supposed to
+ * happen in the same millisecond.
+ */
+public class ManualEnvironmentEdge implements EnvironmentEdge {
+
+  // Sometimes 0 ts might have a special value, so lets start with 1
+  protected long value = 1L;
+
+  public void setValue(long newValue) {
+    value = newValue;
+  }
+
+  @Override
+  public long currentTimeMillis() {
+    return this.value;
+  }
+}

Added: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java (added)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/PairOfSameType.java Fri Aug 27 05:01:02 2010
@@ -0,0 +1,112 @@
+/**
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import java.util.Iterator;
+
+import org.apache.commons.lang.NotImplementedException;
+
+/**
+ * A generic, immutable class for pairs of objects both of type <code>T</code>.
+ * @param <T>
+ * @See {@link Pair} if Types differ.
+ */
+public class PairOfSameType<T> implements Iterable<T> {
+  private final T first;
+  private final T second;
+
+  /**
+   * Constructor
+   * @param a operand
+   * @param b operand
+   */
+  public PairOfSameType(T a, T b) {
+    this.first = a;
+    this.second = b;
+  }
+
+  /**
+   * Return the first element stored in the pair.
+   * @return T
+   */
+  public T getFirst() {
+    return first;
+  }
+
+  /**
+   * Return the second element stored in the pair.
+   * @return T
+   */
+  public T getSecond() {
+    return second;
+  }
+
+  private static boolean equals(Object x, Object y) {
+     return (x == null && y == null) || (x != null && x.equals(y));
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean equals(Object other) {
+    return other instanceof PairOfSameType &&
+      equals(first, ((PairOfSameType)other).first) &&
+      equals(second, ((PairOfSameType)other).second);
+  }
+
+  @Override
+  public int hashCode() {
+    if (first == null)
+      return (second == null) ? 0 : second.hashCode() + 1;
+    else if (second == null)
+      return first.hashCode() + 2;
+    else
+      return first.hashCode() * 17 + second.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "{" + getFirst() + "," + getSecond() + "}";
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return new Iterator<T>() {
+      private int returned = 0;
+
+      @Override
+      public boolean hasNext() {
+        return this.returned < 2;
+      }
+
+      @Override
+      public T next() {
+        if (++this.returned == 1) return getFirst();
+        else if (this.returned == 2) return getSecond();
+        else throw new IllegalAccessError("this.returned=" + this.returned);
+      }
+
+      @Override
+      public void remove() {
+        throw new NotImplementedException();
+      }
+    };
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/SoftValueSortedMap.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/SoftValueSortedMap.java?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/SoftValueSortedMap.java (original)
+++ hbase/branches/0.90_master_rewrite/src/main/java/org/apache/hadoop/hbase/util/SoftValueSortedMap.java Fri Aug 27 05:01:02 2010
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.util;
 
 import java.lang.ref.ReferenceQueue;
+import java.lang.ref.SoftReference;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
@@ -72,7 +73,7 @@ public class SoftValueSortedMap<K,V> imp
     for (Object obj; (obj = this.rq.poll()) != null;) {
       i++;
       //noinspection unchecked
-      this.internalMap.remove(((SoftValue<K,V>)obj).getKey());
+      this.internalMap.remove(((SoftValue<K,V>)obj).key);
     }
     return i;
   }
@@ -171,13 +172,7 @@ public class SoftValueSortedMap<K,V> imp
   }
 
   public synchronized Set<Map.Entry<K,V>> entrySet() {
-    checkReferences();
-    Set<Map.Entry<K, SoftValue<K,V>>> entries = this.internalMap.entrySet();
-    Set<Map.Entry<K, V>> real_entries = new TreeSet<Map.Entry<K,V>>();
-    for(Map.Entry<K, SoftValue<K,V>> entry : entries) {
-      real_entries.add(entry.getValue());
-    }
-    return real_entries;
+    throw new RuntimeException("Not implemented");
   }
 
   public synchronized Collection<V> values() {
@@ -189,4 +184,13 @@ public class SoftValueSortedMap<K,V> imp
     }
     return hardValues;
   }
+
+  private static class SoftValue<K,V> extends SoftReference<V> {
+    final K key;
+
+    SoftValue(K key, V value, ReferenceQueue q) {
+      super(value, q);
+      this.key = key;
+    }
+  }
 }

Added: hbase/branches/0.90_master_rewrite/src/main/javadoc/org/apache/hadoop/hbase/replication/package.html
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/javadoc/org/apache/hadoop/hbase/replication/package.html?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/javadoc/org/apache/hadoop/hbase/replication/package.html (added)
+++ hbase/branches/0.90_master_rewrite/src/main/javadoc/org/apache/hadoop/hbase/replication/package.html Fri Aug 27 05:01:02 2010
@@ -0,0 +1,128 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
+<html>
+
+<!--
+   Copyright 2010 The Apache Software Foundation
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<head />
+<body bgcolor="white">
+<h1>Multi Cluster Replication</h1>
+This package provides replication between HBase clusters.
+<p>
+
+<h2>Table Of Contents</h2>
+<ol>
+    <li><a href="#status">Status</a></li>
+    <li><a href="#requirements">Requirements</a></li>
+    <li><a href="#deployment">Deployment</a></li>
+</ol>
+
+<p>
+<a name="status">
+<h2>Status</h2>
+</a>
+<p>
+This package is experimental quality software and is only meant to be a base
+for future developments. The current implementation offers the following
+features:
+
+<ol>
+    <li>Master/Slave replication limited to 1 slave cluster. </li>
+    <li>Replication of scoped families in user tables.</li>
+    <li>Start/stop replication stream.</li>
+    <li>Supports clusters of different sizes.</li>
+    <li>Handling of partitions longer than 10 minutes</li>
+</ol>
+Please report bugs on the project's Jira when found.
+<p>
+<a name="requirements">
+<h2>Requirements</h2>
+</a>
+<p>
+
+Before trying out replication, make sure to review the following requirements:
+
+<ol>
+    <li>Zookeeper should be handled by yourself, not by HBase, and should
+    always be available during the deployment.</li>
+    <li>All machines from both clusters should be able to reach every
+    other machine since replication goes from any region server to any
+    other one on the slave cluster. That also includes the
+    Zookeeper clusters.</li>
+    <li>Both clusters should have the same HBase and Hadoop major revision.
+    For example, having 0.90.1 on the master and 0.90.0 on the slave is
+    correct but not 0.90.1 and 0.89.20100725.</li>
+    <li>Every table that contains families that are scoped for replication
+    should exist on every cluster with the exact same name, same for those
+    replicated families.</li>
+</ol>
+
+<p>
+<a name="deployment">
+<h2>Deployment</h2>
+</a>
+<p>
+
+The following steps describe how to enable replication from a cluster
+to another. This must be done with both clusters offlined.
+<ol>
+    <li>Edit ${HBASE_HOME}/conf/hbase-site.xml on both cluster to add
+    the following configurations:
+        <pre>
+&lt;property&gt;
+  &lt;name&gt;hbase.replication&lt;/name&gt;
+  &lt;value&gt;true&lt;/value&gt;
+&lt;/property&gt;</pre>
+    </li>
+    <li>Run the following command on any cluster:
+    <pre>
+$HBASE_HOME/bin/hbase org.jruby.Main $HBASE_HOME/bin/replication/add_peer.tb</pre>
+    This will show you the help to setup the replication stream between
+    both clusters. If both clusters use the same Zookeeper cluster, you have
+    to use a different <b>zookeeper.znode.parent</b> since they can't
+    write in the same folder.
+    </li>
+    <li>You can now start and stop the clusters with your preferred method.</li>
+</ol>
+
+You can confirm that your setup works by looking at any region server's log
+on the master cluster and look for the following lines;
+
+<pre>
+Considering 1 rs, with ratio 0.1
+Getting 1 rs from peer cluster # 0
+Choosing peer 10.10.1.49:62020</pre>
+
+In this case it indicates that 1 region server from the slave cluster
+was chosen for replication.<br><br>
+
+Should you want to stop the replication while the clusters are running, open
+the shell on the master cluster and issue this command:
+<pre>
+hbase(main):001:0> zk 'set /zookeeper.znode.parent/replication/state false'</pre>
+
+Where you replace the znode parent with the one configured on your master
+cluster. Replication of already queued edits will still happen after you
+issued that command but new entries won't be. To start it back, simply replace
+"false" with "true" in the command.
+
+<p>
+
+</body>
+</html>

Modified: hbase/branches/0.90_master_rewrite/src/main/javadoc/overview.html
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/javadoc/overview.html?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/javadoc/overview.html (original)
+++ hbase/branches/0.90_master_rewrite/src/main/javadoc/overview.html Fri Aug 27 05:01:02 2010
@@ -56,11 +56,13 @@
   <li>Java 1.6.x, preferably from <a href="http://www.java.com/download/">Sun</a>. Use the latest version available except u18 (u19 is fine).</li>
   <li>This version of HBase will only run on <a href="http://hadoop.apache.org/common/releases.html">Hadoop 0.20.x</a>.
  HBase will lose data unless it is running on an HDFS that has a durable sync operation.
- Currently only the <a href="http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-append/">0.20-append</a>
+ Currently only the <a href="http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-append/">branch-0.20-append</a>
  branch has this attribute.  No official releases have been made from this branch as of this writing
  so you will have to build your own Hadoop from the tip of this branch
- (or install Cloudera's <a href="http://archive.cloudera.com/docs/">CDH3b2</a>
- when its available; it will have a durable sync).</li>
+ (or install Cloudera's <a href="http://archive.cloudera.com/docs/">CDH3</a> (as of this writing, it is in beta); it has the
+ 0.20-append patches needed to add a durable sync).
+ See <a href="http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-append/CHANGES.txt">CHANGES.txt</a>
+ in branch-0.20.-append to see list of patches involved.</li>
   <li>
     <em>ssh</em> must be installed and <em>sshd</em> must be running to use Hadoop's scripts to manage remote Hadoop daemons.
    You must be able to ssh to all nodes, including your local node, using passwordless login
@@ -172,7 +174,7 @@ it should run with one replica only (rec
   &lt;property&gt;
     &lt;name&gt;dfs.replication&lt;/name&gt;
     &lt;value&gt;1&lt;/value&gt;
-    &lt;description&gt;The replication count for HLog & HFile storage. Should not be greater than HDFS datanode count.
+    &lt;description&gt;The replication count for HLog &amp; HFile storage. Should not be greater than HDFS datanode count.
     &lt;/description&gt;
   &lt;/property&gt;
   ...
@@ -410,13 +412,21 @@ the HBase version. It does not change yo
 <h2><a name="related" >Related Documentation</a></h2>
 <ul>
   <li><a href="http://hbase.org">HBase Home Page</a>
+  </li>
   <li><a href="http://wiki.apache.org/hadoop/Hbase">HBase Wiki</a>
+  </li>
   <li><a href="http://hadoop.apache.org/">Hadoop Home Page</a>
+  </li>
   <li><a href="http://wiki.apache.org/hadoop/Hbase/MultipleMasters">Setting up Multiple HBase Masters</a>
+  </li>
   <li><a href="http://wiki.apache.org/hadoop/Hbase/RollingRestart">Rolling Upgrades</a>
+  </li>
   <li><a href="org/apache/hadoop/hbase/client/transactional/package-summary.html#package_description">Transactional HBase</a>
+  </li>
   <li><a href="org/apache/hadoop/hbase/client/tableindexed/package-summary.html">Table Indexed HBase</a>
+  </li>
   <li><a href="org/apache/hadoop/hbase/stargate/package-summary.html#package_description">Stargate</a> -- a RESTful Web service front end for HBase.
+  </li>
 </ul>
 
 </body>

Modified: hbase/branches/0.90_master_rewrite/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/resources/hbase-default.xml?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/resources/hbase-default.xml (original)
+++ hbase/branches/0.90_master_rewrite/src/main/resources/hbase-default.xml Fri Aug 27 05:01:02 2010
@@ -186,6 +186,15 @@
     </description>
   </property>
   <property>
+    <name>hbase.regionserver.regionSplitLimit</name>
+    <value>2147483647</value>
+    <description>Limit for the number of regions after which no more region
+    splitting should take place. This is not a hard limit for the number of
+    regions but acts as a guideline for the regionserver to stop splitting after
+    a certain limit. Default is set to MAX_INT.
+    </description>
+  </property>
+  <property>
     <name>hbase.regionserver.logroll.period</name>
     <value>3600000</value>
     <description>Period at which we will roll the commit log.</description>
@@ -290,6 +299,17 @@
     </description>
   </property>
   <property>
+    <name>hbase.master.logcleaner.plugins</name>
+    <value>org.apache.hadoop.hbase.master.TimeToLiveLogCleaner,org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner</value>
+    <description>A comma-separated list of LogCleanerDelegate that are used
+    in LogsCleaner. These log cleaners are called in order, so put the log
+    cleaner that prunes the most log files in the front. To implement your own
+    LogCleanerDelegate, just put it in HBase's classpath and add the fully
+    qualified class name here. Without special reason, you should always add
+    the above default log cleaners in the list.
+    </description>
+  </property>  
+  <property>
     <name>hbase.regions.percheckin</name>
     <value>10</value>
     <description>Maximum number of regions that can be assigned in a single go

Modified: hbase/branches/0.90_master_rewrite/src/main/ruby/shell.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/ruby/shell.rb?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/ruby/shell.rb (original)
+++ hbase/branches/0.90_master_rewrite/src/main/ruby/shell.rb Fri Aug 27 05:01:02 2010
@@ -143,8 +143,6 @@ module Shell
       puts
       puts '-' * 80
       puts
-      puts "Here is the list of groups with their commands:"
-      puts
       ::Shell.command_groups.each do |name, group|
         puts "  " + group[:full_name] + ": "
         puts "    group name: " + name
@@ -162,39 +160,33 @@ module Shell
     end
 
     def help_header
-      return "There are few groups of commands in HBase\n\n" +
-             "Use help 'group_name' (e.g. help 'general') to get help on all commands in a group\n" +
-             "Use help 'command' (e.g. help 'get') to get help on a specific command"
+      return "Enter, help 'COMMAND_GROUP', (e.g. help 'general') to get help on all commands in a group\n" +
+             "Enter, help 'COMMAND', (e.g. help 'get') to get help on a specific command"
     end
 
     def help_footer
-      puts "GENERAL NOTES:"
       puts <<-HERE
-        Quote all names in the hbase shell such as table and column names.  Don't
-        forget commas delimit command parameters.  Type <RETURN> after entering a
-        command to run it.  Dictionaries of configuration used in the creation
-        and alteration of tables are ruby Hashes. They look like this:
-
-          {'key1' => 'value1', 'key2' => 'value2', ...}
-
-        They are opened and closed with curley-braces.  Key/values are delimited
-        by the '=>' character combination.  Usually keys are predefined constants
-        such as NAME, VERSIONS, COMPRESSION, etc.  Constants do not need to be
-        quoted.  Type 'Object.constants' to see a (messy) list of all constants
-        in the environment.
-
-        In case you are using binary keys or values and need to enter them into
-        the shell then use double-quotes to make use of hexadecimal for example:
-
-          hbase> get 't1', "key\\x03\\x3f\\xcd"
-          hbase> get 't1', "key\\003\\023\\011"
-          hbase> put 't1', "test\\xef\\xff", 'f1:', "\\x01\\x33\\x40"
-
-        Using the double-quote notation you can directly use the values output by
-        the shell for example during a "scan" call.
+Quote all names in HBase shell such as table and column names.  Commas delimit
+command parameters.  Type <RETURN> after entering a command to run it.
+Dictionaries of configuration used in the creation and alteration of tables are
+Ruby Hashes. They look like this:
+
+  {'key1' => 'value1', 'key2' => 'value2', ...}
+
+and are opened and closed with curley-braces.  Key/values are delimited by the
+'=>' character combination.  Usually keys are predefined constants such as
+NAME, VERSIONS, COMPRESSION, etc.  Constants do not need to be quoted.  Type
+'Object.constants' to see a (messy) list of all constants in the environment.
+
+If you are using binary keys or values and need to enter them in the shell, use
+double-quote'd hexadecimal representation. For example:
+
+  hbase> get 't1', "key\\x03\\x3f\\xcd"
+  hbase> get 't1', "key\\003\\023\\011"
+  hbase> put 't1', "test\\xef\\xff", 'f1:', "\\x01\\x33\\x40"
 
-        This HBase shell is the JRuby IRB with the above HBase-specific commands
-        added.  For more on the HBase Shell, see http://wiki.apache.org/hadoop/Hbase/Shell
+The HBase shell is the (J)Ruby IRB with the above HBase-specific commands added.
+For more on the HBase Shell, see http://wiki.apache.org/hadoop/Hbase/Shell
       HERE
     end
   end

Modified: hbase/branches/0.90_master_rewrite/src/main/ruby/shell/formatter.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/main/ruby/shell/formatter.rb?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/main/ruby/shell/formatter.rb (original)
+++ hbase/branches/0.90_master_rewrite/src/main/ruby/shell/formatter.rb Fri Aug 27 05:01:02 2010
@@ -30,13 +30,9 @@ module Shell
       end
 
       def refresh_width()
-        @max_width = Java::jline.Terminal.getTerminal().getTerminalWidth()
-
-        # the above doesn't work in some terminals
-        # (such as shell running within emacs)
-        if @max_width == 0
-          @max_width = 100
-        end
+        @max_width = Java::jline.Terminal.getTerminal().getTerminalWidth() if $stdout.tty?
+        # the above doesn't work in some terminals (such as shell running within emacs)
+        @max_width = 100 if @max_width.to_i.zero?
       end
 
       # Takes an output stream and a print width.

Added: hbase/branches/0.90_master_rewrite/src/site/site.vm
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/site/site.vm?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/site/site.vm (added)
+++ hbase/branches/0.90_master_rewrite/src/site/site.vm Fri Aug 27 05:01:02 2010
@@ -0,0 +1,512 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<!-- Generated by Apache Maven Doxia at $dateFormat.format( $currentDate ) -->
+#macro ( link $href $name $target $img $position $alt $border $width $height )
+  #set ( $linkTitle = ' title="' + $name + '"' )
+  #if( $target )
+    #set ( $linkTarget = ' target="' + $target + '"' )
+  #else
+    #set ( $linkTarget = "" )
+  #end
+  #if ( ( $href.toLowerCase().startsWith("http") || $href.toLowerCase().startsWith("https") ) )
+    #set ( $linkClass = ' class="externalLink"' )
+  #else
+    #set ( $linkClass = "" )
+  #end
+  #if ( $img )
+    #if ( $position == "left" )
+      <a href="$href"$linkClass$linkTarget$linkTitle>#image($img $alt $border $width $height)$name</a>
+    #else
+      <a href="$href"$linkClass$linkTarget$linkTitle>$name #image($img $alt $border $width $height)</a>
+    #end
+  #else
+    <a href="$href"$linkClass$linkTarget$linkTitle>$name</a>
+  #end
+#end
+##
+#macro ( image $img $alt $border $width $height )
+  #if( $img )
+    #if ( ! ( $img.toLowerCase().startsWith("http") || $img.toLowerCase().startsWith("https") ) )
+      #set ( $imgSrc = $PathTool.calculateLink( $img, $relativePath ) )
+      #set ( $imgSrc = $imgSrc.replaceAll( "\\", "/" ) )
+      #set ( $imgSrc = ' src="' + $imgSrc + '"' )
+    #else
+      #set ( $imgSrc = ' src="' + $img + '"' )
+    #end
+    #if( $alt )
+      #set ( $imgAlt = ' alt="' + $alt + '"' )
+    #else
+      #set ( $imgAlt = ' alt=""' )
+    #end
+    #if( $border )
+      #set ( $imgBorder = ' border="' + $border + '"' )
+    #else
+      #set ( $imgBorder = "" )
+    #end
+    #if( $width )
+      #set ( $imgWidth = ' width="' + $width + '"' )
+    #else
+      #set ( $imgWidth = "" )
+    #end
+    #if( $height )
+      #set ( $imgHeight = ' height="' + $height + '"' )
+    #else
+      #set ( $imgHeight = "" )
+    #end
+    <img class="imageLink"$imgSrc$imgAlt$imgBorder$imgWidth$imgHeight/>
+  #end
+#end
+#macro ( banner $banner $id )
+  #if ( $banner )
+    #if( $banner.href )
+      <a href="$banner.href" id="$id"#if( $banner.alt ) title="$banner.alt"#end>
+    #else
+        <div id="$id">
+    #end
+##
+    #if( $banner.src )
+        #set ( $src = $banner.src )
+        #if ( ! ( $src.toLowerCase().startsWith("http") || $src.toLowerCase().startsWith("https") ) )
+            #set ( $src = $PathTool.calculateLink( $src, $relativePath ) )
+            #set ( $src = $src.replaceAll( "\\", "/" ) )
+        #end
+        #if ( $banner.alt )
+            #set ( $alt = $banner.alt )
+        #else
+            #set ( $alt = $banner.name )
+        #end
+        <img src="$src" alt="$alt" />
+    #else
+        $banner.name
+    #end
+##
+    #if( $banner.href )
+        </a>
+    #else
+        </div>
+    #end
+  #end
+#end
+##
+#macro ( links $links )
+  #set ( $counter = 0 )
+  #foreach( $item in $links )
+    #set ( $counter = $counter + 1 )
+    #set ( $currentItemHref = $PathTool.calculateLink( $item.href, $relativePath ) )
+    #set ( $currentItemHref = $currentItemHref.replaceAll( "\\", "/" ) )
+    #link( $currentItemHref $item.name $item.target $item.img $item.position $item.alt $item.border $item.width $item.height )
+    #if ( $links.size() > $counter )
+      |
+    #end
+  #end
+#end
+##
+#macro ( breadcrumbs $breadcrumbs )
+  #set ( $counter = 0 )
+  #foreach( $item in $breadcrumbs )
+    #set ( $counter = $counter + 1 )
+    #set ( $currentItemHref = $PathTool.calculateLink( $item.href, $relativePath ) )
+    #set ( $currentItemHref = $currentItemHref.replaceAll( "\\", "/" ) )
+##
+    #if ( $currentItemHref == $alignedFileName || $currentItemHref == "" )
+      $item.name
+    #else
+      #link( $currentItemHref $item.name $item.target $item.img $item.position $item.alt $item.border $item.width $item.height )
+    #end
+    #if ( $breadcrumbs.size() > $counter )
+      &gt;
+    #end
+  #end
+#end
+##
+#macro ( displayTree $display $item )
+  #if ( $item && $item.items && $item.items.size() > 0 )
+    #foreach( $subitem in $item.items )
+      #set ( $subitemHref = $PathTool.calculateLink( $subitem.href, $relativePath ) )
+      #set ( $subitemHref = $subitemHref.replaceAll( "\\", "/" ) )
+      #if ( $alignedFileName == $subitemHref )
+        #set ( $display = true )
+      #end
+##
+      #displayTree( $display $subitem )
+    #end
+  #end
+#end
+##
+#macro ( menuItem $item )
+  #set ( $collapse = "none" )
+  #set ( $currentItemHref = $PathTool.calculateLink( $item.href, $relativePath ) )
+  #set ( $currentItemHref = $currentItemHref.replaceAll( "\\", "/" ) )
+##
+  #if ( $item && $item.items && $item.items.size() > 0 )
+    #if ( $item.collapse == false )
+      #set ( $collapse = "expanded" )
+    #else
+      ## By default collapsed
+      #set ( $collapse = "collapsed" )
+    #end
+##
+    #set ( $display = false )
+    #displayTree( $display $item )
+##
+    #if ( $alignedFileName == $currentItemHref || $display )
+      #set ( $collapse = "expanded" )
+    #end
+  #end
+  <li class="$collapse">
+  #if ( $item.img )
+    #if ( $item.position == "left" )
+      #if ( $alignedFileName == $currentItemHref )
+        <strong>#image($item.img $item.alt $item.border $item.width $item.height) $item.name</strong>
+      #else
+        #link($currentItemHref $item.name $item.target $item.img $item.position $item.alt $item.border $item.width $item.height)
+      #end
+    #else
+      #if ( $alignedFileName == $currentItemHref )
+        <strong>$item.name #image($item.img $item.alt $item.border $item.width $item.height)</strong>
+      #else
+        #link($currentItemHref $item.name $item.target $item.img $item.position $item.alt $item.border $item.width $item.height)
+      #end
+    #end
+  #else
+    #if ( $alignedFileName == $currentItemHref )
+      <strong>$item.name</strong>
+    #else
+      #link( $currentItemHref $item.name $item.target $item.img $item.position $item.alt $item.border $item.width $item.height )
+    #end
+  #end
+  #if ( $item && $item.items && $item.items.size() > 0 )
+    #if ( $collapse == "expanded" )
+      <ul>
+        #foreach( $subitem in $item.items )
+          #menuItem( $subitem )
+        #end
+      </ul>
+    #end
+  #end
+  </li>
+#end
+##
+#macro ( mainMenu $menus )
+  #foreach( $menu in $menus )
+    #if ( $menu.name )
+      #if ( $menu.img )
+        #if( $menu.position )
+          #set ( $position = $menu.position )
+        #else
+          #set ( $position = "left" )
+        #end
+##
+        #if ( ! ( $menu.img.toLowerCase().startsWith("http") || $menu.img.toLowerCase().startsWith("https") ) )
+          #set ( $src = $PathTool.calculateLink( $menu.img, $relativePath ) )
+          #set ( $src = $src.replaceAll( "\\", "/" ) )
+          #set ( $src = ' src="' + $src + '"' )
+        #else
+          #set ( $src = ' src="' + $menu.img + '"' )
+        #end
+##
+        #if( $menu.alt )
+          #set ( $alt = ' alt="' + $menu.alt + '"' )
+        #else
+          #set ( $alt = ' alt="' + $menu.name + '"' )
+        #end
+##
+        #if( $menu.border )
+          #set ( $border = ' border="' + $menu.border + '"' )
+        #else
+          #set ( $border = ' border="0"' )
+        #end
+##
+        #if( $menu.width )
+          #set ( $width = ' width="' + $menu.width + '"' )
+        #else
+          #set ( $width = "" )
+        #end
+        #if( $menu.height )
+          #set ( $height = ' height="' + $menu.height + '"' )
+        #else
+          #set ( $height = "" )
+        #end
+##
+        #set ( $img = '<img class="imageLink"' + $src + $alt + $border + $width + $height + "/>" )
+##
+        #if ( $position == "left" )
+        <h5>$img $menu.name</h5>
+        #else
+        <h5>$menu.name $img</h5>
+        #end
+      #else
+       <h5>$menu.name</h5>
+      #end
+    #end
+    #if ( $menu.items && $menu.items.size() > 0 )
+    <ul>
+      #foreach( $item in $menu.items )
+        #menuItem( $item )
+      #end
+    </ul>
+    #end
+  #end
+#end
+##
+#macro ( copyright )
+  #if ( $project )
+    #if ( ${project.organization} && ${project.organization.name} )
+      #set ( $period = "" )
+    #else
+      #set ( $period = "." )
+   #end
+##
+   #set ( $currentYear = ${currentDate.year} + 1900 )
+##
+    #if ( ${project.inceptionYear} && ( ${project.inceptionYear} != ${currentYear.toString()} ) )
+      ${project.inceptionYear}-${currentYear}${period}
+    #else
+      ${currentYear}${period}
+    #end
+##
+    #if ( ${project.organization} )
+      #if ( ${project.organization.name} && ${project.organization.url} )
+          <a href="$project.organization.url">${project.organization.name}</a>.
+      #elseif ( ${project.organization.name} )
+        ${project.organization.name}.
+      #end
+    #end
+  #end
+#end
+##
+#macro ( publishDate $position $publishDate $version )
+  #if ( $publishDate && $publishDate.format )
+    #set ( $format = $publishDate.format )
+  #else
+    #set ( $format = "yyyy-MM-dd" )
+  #end
+##
+  $dateFormat.applyPattern( $format )
+##
+  #set ( $dateToday = $dateFormat.format( $currentDate ) )
+##
+  #if ( $publishDate && $publishDate.position )
+    #set ( $datePosition = $publishDate.position )
+  #else
+    #set ( $datePosition = "left" )
+  #end
+##
+  #if ( $version )
+    #if ( $version.position )
+      #set ( $versionPosition = $version.position )
+    #else
+      #set ( $versionPosition = "left" )
+    #end
+  #else
+    #set ( $version = "" )
+    #set ( $versionPosition = "left" )
+  #end
+##
+  #set ( $breadcrumbs = $decoration.body.breadcrumbs )
+  #set ( $links = $decoration.body.links )
+
+  #if ( $datePosition.equalsIgnoreCase( "right" ) && $links && $links.size() > 0 )
+    #set ( $prefix = "&nbsp;|" )
+  #else
+    #set ( $prefix = "" )
+  #end
+##
+  #if ( $datePosition.equalsIgnoreCase( $position ) )
+    #if ( ( $datePosition.equalsIgnoreCase( "right" ) ) || ( $datePosition.equalsIgnoreCase( "bottom" ) ) )
+      $prefix <span id="publishDate">$i18n.getString( "site-renderer", $locale, "template.lastpublished" ): $dateToday</span>
+      #if ( $versionPosition.equalsIgnoreCase( $position ) )
+        &nbsp;| <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+      #end
+    #elseif ( ( $datePosition.equalsIgnoreCase( "navigation-bottom" ) ) || ( $datePosition.equalsIgnoreCase( "navigation-top" ) ) )
+      <div id="lastPublished">
+        <span id="publishDate">$i18n.getString( "site-renderer", $locale, "template.lastpublished" ): $dateToday</span>
+        #if ( $versionPosition.equalsIgnoreCase( $position ) )
+          &nbsp;| <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+        #end
+      </div>
+    #elseif ( $datePosition.equalsIgnoreCase("left") )
+      <div class="xleft">
+        <span id="publishDate">$i18n.getString( "site-renderer", $locale, "template.lastpublished" ): $dateToday</span>
+        #if ( $versionPosition.equalsIgnoreCase( $position ) )
+          &nbsp;| <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+        #end
+        #if ( $breadcrumbs && $breadcrumbs.size() > 0 )
+          | #breadcrumbs( $breadcrumbs )
+        #end
+      </div>
+    #end
+  #elseif ( $versionPosition.equalsIgnoreCase( $position ) )
+    #if ( ( $versionPosition.equalsIgnoreCase( "right" ) ) || ( $versionPosition.equalsIgnoreCase( "bottom" ) ) )
+      $prefix <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+    #elseif ( ( $versionPosition.equalsIgnoreCase( "navigation-bottom" ) ) || ( $versionPosition.equalsIgnoreCase( "navigation-top" ) ) )
+      <div id="lastPublished">
+        <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+      </div>
+    #elseif ( $versionPosition.equalsIgnoreCase("left") )
+      <div class="xleft">
+        <span id="projectVersion">$i18n.getString( "site-renderer", $locale, "template.version" ): ${project.version}</span>
+        #if ( $breadcrumbs && $breadcrumbs.size() > 0 )
+          | #breadcrumbs( $breadcrumbs )
+        #end
+      </div>
+    #end
+  #elseif ( $position.equalsIgnoreCase( "left" ) )
+    #if ( $breadcrumbs && $breadcrumbs.size() > 0 )
+      <div class="xleft">
+        #breadcrumbs( $breadcrumbs )
+      </div>
+    #end
+  #end
+#end
+##
+#macro ( poweredByLogo $poweredBy )
+  #if( $poweredBy )
+    #foreach ($item in $poweredBy)
+      #if( $item.href )
+        #set ( $href = $PathTool.calculateLink( $item.href, $relativePath ) )
+        #set ( $href = $href.replaceAll( "\\", "/" ) )
+      #else
+        #set ( $href="http://maven.apache.org/" )
+      #end
+##
+      #if( $item.name )
+        #set ( $name = $item.name )
+      #else
+        #set ( $name = $i18n.getString( "site-renderer", $locale, "template.builtby" )  )
+        #set ( $name = "${name} Maven"  )
+      #end
+##
+      #if( $item.img )
+        #set ( $img = $item.img )
+      #else
+        #set ( $img = "images/logos/maven-feather.png" )
+      #end
+##
+      #if ( ! ( $img.toLowerCase().startsWith("http") || $img.toLowerCase().startsWith("https") ) )
+        #set ( $img = $PathTool.calculateLink( $img, $relativePath ) )
+        #set ( $img = $src.replaceAll( "\\", "/" ) )
+      #end
+##
+      #if( $item.alt )
+        #set ( $alt = ' alt="' + $item.alt + '"' )
+      #else
+        #set ( $alt = ' alt="' + $name + '"' )
+      #end
+##
+      #if( $item.border )
+        #set ( $border = ' border="' + $item.border + '"' )
+      #else
+        #set ( $border = "" )
+      #end
+##
+      #if( $item.width )
+        #set ( $width = ' width="' + $item.width + '"' )
+      #else
+        #set ( $width = "" )
+      #end
+      #if( $item.height )
+        #set ( $height = ' height="' + $item.height + '"' )
+      #else
+        #set ( $height = "" )
+      #end
+##
+      <a href="$href" title="$name" class="poweredBy">
+        <img class="poweredBy" $alt src="$img" $border $width $height />
+      </a>
+    #end
+    #if( $poweredBy.isEmpty() )
+      <a href="http://maven.apache.org/" title="$i18n.getString( "site-renderer", $locale, "template.builtby" ) Maven" class="poweredBy">
+        <img class="poweredBy" alt="$i18n.getString( "site-renderer", $locale, "template.builtby" ) Maven" src="$relativePath/images/logos/maven-feather.png" />
+      </a>
+    #end
+  #else
+    <a href="http://maven.apache.org/" title="$i18n.getString( "site-renderer", $locale, "template.builtby" ) Maven" class="poweredBy">
+      <img class="poweredBy" alt="$i18n.getString( "site-renderer", $locale, "template.builtby" ) Maven" src="$relativePath/images/logos/maven-feather.png" />
+    </a>
+  #end
+#end
+##
+<html xmlns="http://www.w3.org/1999/xhtml"#if ( $locale ) xml:lang="$locale.language" lang="$locale.language"#end>
+  <head>
+    <meta http-equiv="Content-Type" content="text/html; charset=${outputEncoding}" />
+    <title>$title</title>
+    <style type="text/css" media="all">
+      @import url("$relativePath/css/maven-base.css");
+      @import url("$relativePath/css/maven-theme.css");
+      @import url("$relativePath/css/site.css");
+    </style>
+    <link rel="stylesheet" href="$relativePath/css/print.css" type="text/css" media="print" />
+#foreach( $author in $authors )
+      <meta name="author" content="$author" />
+#end
+#if ( $dateCreation )
+    <meta name="Date-Creation-yyyymmdd" content="$dateCreation" />
+#end
+#if ( $dateRevision )
+    <meta name="Date-Revision-yyyymmdd" content="$dateRevision" />
+#end
+#if ( $locale )
+    <meta http-equiv="Content-Language" content="$locale.language" />
+#end
+    #if ( $decoration.body.head )
+      #foreach( $item in $decoration.body.head.getChildren() )
+        ## Workaround for DOXIA-150 due to a non-desired behaviour in p-u
+        ## @see org.codehaus.plexus.util.xml.Xpp3Dom#toString()
+        ## @see org.codehaus.plexus.util.xml.Xpp3Dom#toUnescapedString()
+        #set ( $documentHeader = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" )
+        #set ( $documentHeader = $documentHeader.replaceAll( "\\", "" ) )
+        #if ( $item.name == "script" )
+          $StringUtils.replace( $item.toUnescapedString(), $documentHeader, "" )
+        #else
+          $StringUtils.replace( $item.toString(), $documentHeader, "" )
+        #end
+      #end
+    #end
+    ## $headContent
+  </head>
+  <body class="composite">
+    <div id="banner">
+      #banner( $decoration.bannerLeft "bannerLeft" )
+      #banner( $decoration.bannerRight "bannerRight" )
+      <div class="clear">
+        <hr/>
+      </div>
+    </div>
+    <div id="breadcrumbs">
+      #publishDate( "left" $decoration.publishDate $decoration.version )
+      <div class="xright" style="padding-left: 8px; margin-top: -4px;">
+        <form method="GET" action="http://search-hadoop.com/">
+          <input type="text" style="width: 192px; height: 15px; font-size: inherit; border: 1px solid darkgray" name="q" value="Search wiki, mailing lists, sources & more" onfocus="this.value=''"/>
+          <input type="hidden" name="fc_project" value="HBase"/>
+          <button style="height: 20px; width: 60px;">Search</button>
+        </form>
+      </div>
+      <div class="xright">#links( $decoration.body.links )#publishDate( "right" $decoration.publishDate $decoration.version )</div>
+      <div class="clear">
+        <hr/>
+      </div>
+    </div>
+    <div id="leftColumn">
+      <div id="navcolumn">
+       #publishDate( "navigation-top" $decoration.publishDate $decoration.version )
+       #mainMenu( $decoration.body.menus )
+       #poweredByLogo( $decoration.poweredBy )
+       #publishDate( "navigation-bottom" $decoration.publishDate $decoration.version )
+      </div>
+    </div>
+    <div id="bodyColumn">
+      <div id="contentBox">
+        $bodyContent
+      </div>
+    </div>
+    <div class="clear">
+      <hr/>
+    </div>
+    <div id="footer">
+      <div class="xright">Copyright &#169;#copyright()All Rights Reserved.#publishDate( "bottom" $decoration.publishDate $decoration.version )</div>
+      <div class="clear">
+        <hr/>
+      </div>
+    </div>
+  </body>
+</html>

Modified: hbase/branches/0.90_master_rewrite/src/site/site.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/site/site.xml?rev=990018&r1=990017&r2=990018&view=diff
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/site/site.xml (original)
+++ hbase/branches/0.90_master_rewrite/src/site/site.xml Fri Aug 27 05:01:02 2010
@@ -18,7 +18,7 @@
     <menu name="HBase">
       <item name="Overview" href="index.html"/>
       <item name="License" href="license.html" />
-      <item name="Downloads" href="http://www.apache.org/dyn/closer.cgi/hadoop/hbase/" />
+      <item name="Downloads" href="http://www.apache.org/dyn/closer.cgi/hbase/" />
       <item name="Release Notes" href="https://issues.apache.org/jira/browse/HBASE?report=com.atlassian.jira.plugin.system.project:changelog-panel" />
       <item name="Issue Tracking" href="issue-tracking.html" />
       <item name="Mailing Lists" href="mail-lists.html" />
@@ -35,6 +35,10 @@
       <item name="Bulk Loads" href="bulk-loads.html" />
       <item name="Metrics"      href="metrics.html" />
       <item name="HBase on Windows"      href="cygwin.html" />
+      <item name="Cluster replication"      href="replication.html" />
+      <item name="Pseudo-Distributed HBase"      href="pseudo-distributed.html" />
+      <item name="HBase Book"      href="book.html" />
+      <item name="Example Docbook Article"      href="sample_article.html" />
     </menu>
   </body>
     <skin>

Added: hbase/branches/0.90_master_rewrite/src/site/xdoc/pseudo-distributed.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.90_master_rewrite/src/site/xdoc/pseudo-distributed.xml?rev=990018&view=auto
==============================================================================
--- hbase/branches/0.90_master_rewrite/src/site/xdoc/pseudo-distributed.xml (added)
+++ hbase/branches/0.90_master_rewrite/src/site/xdoc/pseudo-distributed.xml Fri Aug 27 05:01:02 2010
@@ -0,0 +1,77 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Copyright 2010 The Apache Software Foundation
+
+  Licensed 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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document xmlns="http://maven.apache.org/XDOC/2.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
+  <properties>
+    <title> 
+Running HBase in pseudo-distributed mode
+    </title>
+  </properties>
+
+  <body>
+      <p>This document augments what is described in the HBase 'Getting Started' in the 
+ <a href="http://hbase.apache.org/docs/current/api/overview-summary.html#distributed">Distributed Operation: Pseudo- and Fully-distributed modes</a> section.
+ In particular it describes scripts that allow you start extra masters and regionservers when running in pseudo-distributed mode.
+ </p>
+
+ <ol><li>Copy the psuedo-distributed suggested configuration file (feel free to take a peek and understand what it's doing)
+             <source>% cp conf/hbase-site.xml{.psuedo-distributed.template,}</source>
+    </li>
+    <li>(Optional) Start up <a href="http://hadoop.apache.org/common/docs/r0.20.2/quickstart.html#PseudoDistributed">Pseudo-distributed HDFS</a>.
+             <ol><li>If you do, go to conf/hbase-site.xml.  Uncomment the 'hbase.rootdir' property.
+                 </li>
+               <li>Additionally, if you want to test HBase with high data durability enabled, also uncomment the 'dfs.support.append' property.
+               </li>
+       </ol>
+   </li>
+<li>Start up the initial HBase cluster
+                   <source>% bin/start-hbase.sh</source>
+                   <ol>    <li>To start up an extra backup master(s) on the same server run
+                       <source>% bin/local-master-backup.sh start 1</source>
+                       Here the '1' means use ports 60001 &amp; 60011, and this backup master's logfile will be at <i>logs/hbase-${USER}-1-master-${HOSTNAME}.log</i>.
+                       To startup multiple backup masters run <source>% bin/local-master-backup.sh start 2 3</source> You can start up to 9 backup masters (10 total).
+ </li>
+ <li>To start up more regionservers
+     <source>% bin/local-regionservers.sh start 1</source>
+     where '1' means use ports 60201 &amp; 60301 and its logfile will be at <i>logs/hbase-${USER}-1-regionserver-${HOSTNAME}.log</i>.
+     To add 4 more regionservers in addition to the one you just started by running <source>% bin/local-regionservers.sh start 2 3 4 5</source>
+     Supports up to 99 extra regionservers (100 total).
+                    </li>
+                </ol>
+</li>
+<li>To stop the cluster
+    <ol>
+        <li>Assuming you want to stop master backup # 1, run
+            <source>% cat /tmp/hbase-${USER}-1-master.pid |xargs kill -9</source>
+            Note that bin/local-master-backup.sh stop 1 will try to stop the cluster along with the master
+                        </li>
+                        <li>To stop an individual regionserver, run
+                            <source>% bin/local-regionservers.sh stop 1
+                            </source>
+                        </li>
+                    </ol>
+</li>
+</ol>
+</body>
+
+</document>
+