HADOOP-1957 Web UI with report on cluster state and basic browsing of tables
git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@582442 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
6e4efdf697
commit
6c9ddf9f37
|
@ -22,6 +22,7 @@ Trunk (unreleased changes)
|
||||||
(Inchul Song and Edward Yoon via Stack)
|
(Inchul Song and Edward Yoon via Stack)
|
||||||
HADOOP-1913 Build a Lucene index on an HBase table
|
HADOOP-1913 Build a Lucene index on an HBase table
|
||||||
(Ning Li via Stack)
|
(Ning Li via Stack)
|
||||||
|
HADOOP-1957 Web UI with report on cluster state and basic browsing of tables
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
|
|
|
@ -116,6 +116,9 @@ fi
|
||||||
if [ -d "$HADOOP_HOME/build/classes" ]; then
|
if [ -d "$HADOOP_HOME/build/classes" ]; then
|
||||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
|
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/classes
|
||||||
fi
|
fi
|
||||||
|
if [ -d "$HADOOP_HOME/build/contrib/hbase/webapps" ]; then
|
||||||
|
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build/contrib/hbase/
|
||||||
|
fi
|
||||||
if [ -d "$HADOOP_HOME/build/webapps" ]; then
|
if [ -d "$HADOOP_HOME/build/webapps" ]; then
|
||||||
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
|
CLASSPATH=${CLASSPATH}:$HADOOP_HOME/build
|
||||||
fi
|
fi
|
||||||
|
|
47
build.xml
47
build.xml
|
@ -26,9 +26,32 @@ to call at top-level: ant deploy-contrib compile-core-test
|
||||||
<project name="hbase" default="jar">
|
<project name="hbase" default="jar">
|
||||||
<import file="../build-contrib.xml"/>
|
<import file="../build-contrib.xml"/>
|
||||||
|
|
||||||
<target name="javacc" if="javacc.home">
|
<property name="build.webapps" value="${build.dir}/webapps"/>
|
||||||
<echo message="javacc.home: ${javacc.home}"/>
|
<property name="src.webapps" value="${basedir}/src/webapps" />
|
||||||
<property name="hbaseshell.src.dir"
|
|
||||||
|
<target name="init">
|
||||||
|
<antcall target="hadoopbuildcontrib.init"/>
|
||||||
|
<!--Version is set only if called from hadoop build.xml. Set a default-->
|
||||||
|
<condition property="version" value="0.15.0-dev">
|
||||||
|
<not>
|
||||||
|
<isset property="version" />
|
||||||
|
</not>
|
||||||
|
</condition>
|
||||||
|
<!--Copy webapps over to build dir. Exclude jsp and generated-src java
|
||||||
|
classes -->
|
||||||
|
<mkdir dir="${build.webapps}"/>
|
||||||
|
<copy todir="${build.webapps}">
|
||||||
|
<fileset dir="${src.webapps}">
|
||||||
|
<exclude name="**/*.jsp" />
|
||||||
|
<exclude name="**/.*" />
|
||||||
|
<exclude name="**/*~" />
|
||||||
|
</fileset>
|
||||||
|
</copy>
|
||||||
|
</target>
|
||||||
|
|
||||||
|
<target name="javacc" if="javacc.home">
|
||||||
|
<echo message="javacc.home: ${javacc.home}"/>
|
||||||
|
<property name="hbaseshell.src.dir"
|
||||||
value="${src.dir}/org/apache/hadoop/hbase/shell" />
|
value="${src.dir}/org/apache/hadoop/hbase/shell" />
|
||||||
<mkdir dir="${hbaseshell.src.dir}/generated" />
|
<mkdir dir="${hbaseshell.src.dir}/generated" />
|
||||||
<javacc
|
<javacc
|
||||||
|
@ -36,10 +59,11 @@ to call at top-level: ant deploy-contrib compile-core-test
|
||||||
outputdirectory="${hbaseshell.src.dir}/generated"
|
outputdirectory="${hbaseshell.src.dir}/generated"
|
||||||
javacchome="${javacc.home}"
|
javacchome="${javacc.home}"
|
||||||
/>
|
/>
|
||||||
</target>
|
</target>
|
||||||
|
|
||||||
<target name="compile" depends="init,javacc">
|
<target name="compile" depends="init,javacc">
|
||||||
<echo message="contrib: ${name}"/>
|
<echo message="contrib: ${name}"/>
|
||||||
|
<!--Compile whats under src and generated java classes made from jsp-->
|
||||||
<javac
|
<javac
|
||||||
encoding="${build.encoding}"
|
encoding="${build.encoding}"
|
||||||
srcdir="${src.dir}"
|
srcdir="${src.dir}"
|
||||||
|
@ -50,13 +74,14 @@ to call at top-level: ant deploy-contrib compile-core-test
|
||||||
<classpath refid="classpath"/>
|
<classpath refid="classpath"/>
|
||||||
<classpath path="path"/>
|
<classpath path="path"/>
|
||||||
</javac>
|
</javac>
|
||||||
</target>
|
</target>
|
||||||
|
|
||||||
<!-- Override jar target to specify main class -->
|
<!-- Override jar target to specify main class -->
|
||||||
<target name="jar" depends="compile">
|
<target name="jar" depends="compile">
|
||||||
<jar jarfile="${build.dir}/hadoop-${version}-${name}.jar"
|
<jar jarfile="${build.dir}/hadoop-${version}-${name}.jar"
|
||||||
basedir="${build.classes}" >
|
basedir="${build.classes}" >
|
||||||
<fileset file="${root}/conf/hbase-default.xml"/>
|
<fileset file="${root}/conf/hbase-default.xml"/>
|
||||||
|
<zipfileset dir="${build.webapps}" prefix="webapps"/>
|
||||||
</jar>
|
</jar>
|
||||||
</target>
|
</target>
|
||||||
|
|
||||||
|
@ -85,21 +110,17 @@ to call at top-level: ant deploy-contrib compile-core-test
|
||||||
</target>
|
</target>
|
||||||
|
|
||||||
<!-- the unit test classpath
|
<!-- the unit test classpath
|
||||||
Copied from ../build-contrib.xml so can add
|
Copied from ../build-contrib.xml so can add to it.
|
||||||
-->
|
-->
|
||||||
<path id="test.classpath">
|
<path id="test.classpath">
|
||||||
<pathelement location="${build.test}" />
|
<pathelement location="${build.test}" />
|
||||||
|
<pathelement location="${src.test}"/>
|
||||||
<pathelement location="${hadoop.root}/build/test/classes"/>
|
<pathelement location="${hadoop.root}/build/test/classes"/>
|
||||||
<pathelement location="${hadoop.root}/src/contrib/test"/>
|
<pathelement location="${hadoop.root}/src/contrib/test"/>
|
||||||
<pathelement location="${conf.dir}"/>
|
<pathelement location="${conf.dir}"/>
|
||||||
<pathelement location="${hadoop.root}/build"/>
|
<pathelement location="${hadoop.root}/build"/>
|
||||||
<pathelement location="${src.test}"/>
|
|
||||||
<pathelement location="${root}/conf"/>
|
<pathelement location="${root}/conf"/>
|
||||||
|
<pathelement location="${build.dir}"/>
|
||||||
<path refid="classpath"/>
|
<path refid="classpath"/>
|
||||||
</path>
|
</path>
|
||||||
|
|
||||||
<target name="test">
|
|
||||||
<antcall target="hadoopbuildcontrib.test"/>
|
|
||||||
</target>
|
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -29,12 +29,38 @@
|
||||||
TODO: Support 'local' (All running in single context).
|
TODO: Support 'local' (All running in single context).
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.master.info.port</name>
|
||||||
|
<value>60010</value>
|
||||||
|
<description>The port for the hbase master web UI
|
||||||
|
Set to -1 if you do not want the info server to run.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.master.info.bindAddress</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<description>The address for the hbase master web UI
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.regionserver</name>
|
<name>hbase.regionserver</name>
|
||||||
<value>0.0.0.0:60010</value>
|
<value>0.0.0.0:60020</value>
|
||||||
<description>The host and port a HBase region server runs at.
|
<description>The host and port a HBase region server runs at.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.info.port</name>
|
||||||
|
<value>60030</value>
|
||||||
|
<description>The port for the hbase regionserver web UI
|
||||||
|
Set to -1 if you do not want the info server to run.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.info.bindAddress</name>
|
||||||
|
<value>0.0.0.0</value>
|
||||||
|
<description>The address for the hbase regionserver web UI
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.regionserver.class</name>
|
<name>hbase.regionserver.class</name>
|
||||||
<value>org.apache.hadoop.hbase.HRegionInterface</value>
|
<value>org.apache.hadoop.hbase.HRegionInterface</value>
|
||||||
|
@ -176,4 +202,10 @@
|
||||||
if true, enable audible keyboard bells if an alert is required.
|
if true, enable audible keyboard bells if an alert is required.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbaseshell.formatter</name>
|
||||||
|
<value>org.apache.hadoop.hbase.shell.formatter.AsciiTableFormatter</value>
|
||||||
|
<description>TableFormatter to use outputting HQL result sets.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
</configuration>
|
</configuration>
|
||||||
|
|
|
@ -47,11 +47,15 @@ public interface HConstants {
|
||||||
/** Default master address */
|
/** Default master address */
|
||||||
static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":60000";
|
static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":60000";
|
||||||
|
|
||||||
|
static final int DEFAULT_MASTER_INFOPORT = 60010;
|
||||||
|
|
||||||
/** Parameter name for hbase.regionserver address. */
|
/** Parameter name for hbase.regionserver address. */
|
||||||
static final String REGIONSERVER_ADDRESS = "hbase.regionserver";
|
static final String REGIONSERVER_ADDRESS = "hbase.regionserver";
|
||||||
|
|
||||||
/** Default region server address */
|
/** Default region server address */
|
||||||
static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60010";
|
static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020";
|
||||||
|
|
||||||
|
static final int DEFAULT_REGIONSERVER_INFOPORT = 60030;
|
||||||
|
|
||||||
/** Parameter name for what region server interface to use. */
|
/** Parameter name for what region server interface to use. */
|
||||||
static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
|
static final String REGION_SERVER_CLASS = "hbase.regionserver.class";
|
||||||
|
|
|
@ -51,6 +51,13 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.PathFilter;
|
import org.apache.hadoop.fs.PathFilter;
|
||||||
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.InfoServer;
|
||||||
|
import org.apache.hadoop.hbase.util.Sleeper;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.MapWritable;
|
import org.apache.hadoop.io.MapWritable;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
|
@ -58,13 +65,6 @@ import org.apache.hadoop.ipc.RPC;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.ipc.Server;
|
import org.apache.hadoop.ipc.Server;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.io.BatchUpdate;
|
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.Sleeper;
|
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HMaster is the "master server" for a HBase.
|
* HMaster is the "master server" for a HBase.
|
||||||
|
@ -121,6 +121,11 @@ HMasterRegionInterface {
|
||||||
// A Sleeper that sleeps for threadWakeFrequency
|
// A Sleeper that sleeps for threadWakeFrequency
|
||||||
protected Sleeper sleeper;
|
protected Sleeper sleeper;
|
||||||
|
|
||||||
|
// Default access so accesible from unit tests. MASTER is name of the webapp
|
||||||
|
// and the attribute name used stuffing this instance into web context.
|
||||||
|
InfoServer infoServer;
|
||||||
|
public static final String MASTER = "master";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base HRegion scanner class. Holds utilty common to <code>ROOT</code> and
|
* Base HRegion scanner class. Holds utilty common to <code>ROOT</code> and
|
||||||
* <code>META</code> HRegion scanners.
|
* <code>META</code> HRegion scanners.
|
||||||
|
@ -538,7 +543,7 @@ HMasterRegionInterface {
|
||||||
Integer rootScannerLock = new Integer(0);
|
Integer rootScannerLock = new Integer(0);
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
static class MetaRegion implements Comparable {
|
public static class MetaRegion implements Comparable {
|
||||||
private HServerAddress server;
|
private HServerAddress server;
|
||||||
private Text regionName;
|
private Text regionName;
|
||||||
private Text startKey;
|
private Text startKey;
|
||||||
|
@ -960,12 +965,47 @@ HMasterRegionInterface {
|
||||||
return address;
|
return address;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Hbase root dir.
|
||||||
|
*/
|
||||||
|
public Path getRootDir() {
|
||||||
|
return this.dir;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Read-only map of servers to serverinfo.
|
||||||
|
*/
|
||||||
|
public Map<String, HServerInfo> getServersToServerInfo() {
|
||||||
|
return Collections.unmodifiableMap(this.serversToServerInfo);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Read-only map of servers to load.
|
||||||
|
*/
|
||||||
|
public Map<String, HServerLoad> getServersToLoad() {
|
||||||
|
return Collections.unmodifiableMap(this.serversToLoad);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Location of the <code>-ROOT-</code> region.
|
||||||
|
*/
|
||||||
|
public HServerAddress getRootRegionLocation() {
|
||||||
|
return this.rootRegionLocation.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Read-only map of online regions.
|
||||||
|
*/
|
||||||
|
public Map<Text, MetaRegion> getOnlineMetaRegions() {
|
||||||
|
return Collections.unmodifiableSortedMap(this.onlineMetaRegions);
|
||||||
|
}
|
||||||
|
|
||||||
/** Main processing loop */
|
/** Main processing loop */
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
final String threadName = "HMaster";
|
final String threadName = "HMaster";
|
||||||
Thread.currentThread().setName(threadName);
|
Thread.currentThread().setName(threadName);
|
||||||
startAllServices();
|
startServiceThreads();
|
||||||
/*
|
/*
|
||||||
* Main processing loop
|
* Main processing loop
|
||||||
*/
|
*/
|
||||||
|
@ -1042,11 +1082,18 @@ HMasterRegionInterface {
|
||||||
synchronized(metaScannerLock) {
|
synchronized(metaScannerLock) {
|
||||||
metaScannerThread.interrupt(); // Wake meta scanner
|
metaScannerThread.interrupt(); // Wake meta scanner
|
||||||
}
|
}
|
||||||
|
if (this.infoServer != null) {
|
||||||
|
LOG.info("Stopping infoServer");
|
||||||
|
try {
|
||||||
|
this.infoServer.stop();
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
ex.printStackTrace();
|
||||||
|
}
|
||||||
|
}
|
||||||
server.stop(); // Stop server
|
server.stop(); // Stop server
|
||||||
serverLeases.close(); // Turn off the lease monitor
|
serverLeases.close(); // Turn off the lease monitor
|
||||||
|
|
||||||
// Join up with all threads
|
// Join up with all threads
|
||||||
|
|
||||||
try {
|
try {
|
||||||
rootScannerThread.join(); // Wait for the root scanner to finish.
|
rootScannerThread.join(); // Wait for the root scanner to finish.
|
||||||
} catch (Exception iex) {
|
} catch (Exception iex) {
|
||||||
|
@ -1067,7 +1114,7 @@ HMasterRegionInterface {
|
||||||
* as OOMEs; it should be lightly loaded. See what HRegionServer does if
|
* as OOMEs; it should be lightly loaded. See what HRegionServer does if
|
||||||
* need to install an unexpected exception handler.
|
* need to install an unexpected exception handler.
|
||||||
*/
|
*/
|
||||||
private void startAllServices() {
|
private void startServiceThreads() {
|
||||||
String threadName = Thread.currentThread().getName();
|
String threadName = Thread.currentThread().getName();
|
||||||
try {
|
try {
|
||||||
Threads.setDaemonThreadRunning(this.rootScannerThread,
|
Threads.setDaemonThreadRunning(this.rootScannerThread,
|
||||||
|
@ -1077,7 +1124,15 @@ HMasterRegionInterface {
|
||||||
// Leases are not the same as Chore threads. Set name differently.
|
// Leases are not the same as Chore threads. Set name differently.
|
||||||
this.serverLeases.setName(threadName + ".leaseChecker");
|
this.serverLeases.setName(threadName + ".leaseChecker");
|
||||||
this.serverLeases.start();
|
this.serverLeases.start();
|
||||||
// Start the server last so everything else is running before we start
|
// Put up info server.
|
||||||
|
int port = this.conf.getInt("hbase.master.info.port", 60010);
|
||||||
|
if (port >= 0) {
|
||||||
|
String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
|
||||||
|
this.infoServer = new InfoServer(MASTER, a, port, false);
|
||||||
|
this.infoServer.setAttribute(MASTER, this);
|
||||||
|
this.infoServer.start();
|
||||||
|
}
|
||||||
|
// Start the server so everything else is running before we start
|
||||||
// receiving requests.
|
// receiving requests.
|
||||||
this.server.start();
|
this.server.start();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
|
|
@ -481,14 +481,13 @@ public class HMemcache {
|
||||||
|
|
||||||
/** Shut down map iterators, and release the lock */
|
/** Shut down map iterators, and release the lock */
|
||||||
public void close() {
|
public void close() {
|
||||||
if(! scannerClosed) {
|
if (!scannerClosed) {
|
||||||
try {
|
try {
|
||||||
for(int i = 0; i < keys.length; i++) {
|
for (int i = 0; i < keys.length; i++) {
|
||||||
if(keyIterators[i] != null) {
|
if(keyIterators[i] != null) {
|
||||||
closeSubScanner(i);
|
closeSubScanner(i);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
lock.releaseReadLock();
|
lock.releaseReadLock();
|
||||||
scannerClosed = true;
|
scannerClosed = true;
|
||||||
|
|
|
@ -314,7 +314,7 @@ public class HRegion implements HConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Returns a HRegionInfo object for this region */
|
/** Returns a HRegionInfo object for this region */
|
||||||
HRegionInfo getRegionInfo() {
|
public HRegionInfo getRegionInfo() {
|
||||||
return this.regionInfo;
|
return this.regionInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.io.BatchOperation;
|
||||||
import org.apache.hadoop.hbase.io.BatchUpdate;
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.hadoop.hbase.util.InfoServer;
|
||||||
import org.apache.hadoop.hbase.util.Sleeper;
|
import org.apache.hadoop.hbase.util.Sleeper;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
|
@ -117,6 +118,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
// A sleeper that sleeps for msgInterval.
|
// A sleeper that sleeps for msgInterval.
|
||||||
private final Sleeper sleeper;
|
private final Sleeper sleeper;
|
||||||
|
|
||||||
|
// Info server. Default access so can be used by unit tests. REGIONSERVER
|
||||||
|
// is name of the webapp and the attribute name used stuffing this instance
|
||||||
|
// into web context.
|
||||||
|
InfoServer infoServer;
|
||||||
|
public static final String REGIONSERVER = "regionserver";
|
||||||
|
|
||||||
// Check to see if regions should be split
|
// Check to see if regions should be split
|
||||||
private final Thread splitOrCompactCheckerThread;
|
private final Thread splitOrCompactCheckerThread;
|
||||||
// Needed at shutdown. On way out, if can get this lock then we are not in
|
// Needed at shutdown. On way out, if can get this lock then we are not in
|
||||||
|
@ -408,7 +415,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
false, conf);
|
false, conf);
|
||||||
this.serverInfo = new HServerInfo(new HServerAddress(
|
this.serverInfo = new HServerInfo(new HServerAddress(
|
||||||
new InetSocketAddress(getThisIP(),
|
new InetSocketAddress(getThisIP(),
|
||||||
this.server.getListenerAddress().getPort())), this.rand.nextLong());
|
this.server.getListenerAddress().getPort())), this.rand.nextLong(),
|
||||||
|
this.conf.getInt("hbase.regionserver.info.port", 60030));
|
||||||
this.leases = new Leases(
|
this.leases = new Leases(
|
||||||
conf.getInt("hbase.regionserver.lease.period", 3 * 60 * 1000),
|
conf.getInt("hbase.regionserver.lease.period", 3 * 60 * 1000),
|
||||||
this.threadWakeFrequency);
|
this.threadWakeFrequency);
|
||||||
|
@ -546,6 +554,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
this.leases.closeAfterLeasesExpire();
|
this.leases.closeAfterLeasesExpire();
|
||||||
this.worker.stop();
|
this.worker.stop();
|
||||||
this.server.stop();
|
this.server.stop();
|
||||||
|
if (this.infoServer != null) {
|
||||||
|
LOG.info("Stopping infoServer");
|
||||||
|
try {
|
||||||
|
this.infoServer.stop();
|
||||||
|
} catch (InterruptedException ex) {
|
||||||
|
ex.printStackTrace();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Send interrupts to wake up threads if sleeping so they notice shutdown.
|
// Send interrupts to wake up threads if sleeping so they notice shutdown.
|
||||||
// TODO: Should we check they are alive? If OOME could have exited already
|
// TODO: Should we check they are alive? If OOME could have exited already
|
||||||
|
@ -689,6 +705,14 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
// an unhandled exception, it will just exit.
|
// an unhandled exception, it will just exit.
|
||||||
this.leases.setName(n + ".leaseChecker");
|
this.leases.setName(n + ".leaseChecker");
|
||||||
this.leases.start();
|
this.leases.start();
|
||||||
|
// Put up info server.
|
||||||
|
int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
|
||||||
|
if (port >= 0) {
|
||||||
|
String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
|
||||||
|
this.infoServer = new InfoServer("regionserver", a, port, false);
|
||||||
|
this.infoServer.setAttribute("regionserver", this);
|
||||||
|
this.infoServer.start();
|
||||||
|
}
|
||||||
// Start Server. This service is like leases in that it internally runs
|
// Start Server. This service is like leases in that it internally runs
|
||||||
// a thread.
|
// a thread.
|
||||||
this.server.start();
|
this.server.start();
|
||||||
|
@ -1050,7 +1074,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
if (s == null) {
|
if (s == null) {
|
||||||
throw new UnknownScannerException("Name: " + scannerName);
|
throw new UnknownScannerException("Name: " + scannerName);
|
||||||
}
|
}
|
||||||
leases.renewLease(scannerId, scannerId);
|
this.leases.renewLease(scannerId, scannerId);
|
||||||
|
|
||||||
// Collect values to be returned here
|
// Collect values to be returned here
|
||||||
MapWritable values = new MapWritable();
|
MapWritable values = new MapWritable();
|
||||||
|
@ -1131,11 +1155,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
// remote scanner interface
|
// remote scanner interface
|
||||||
//
|
//
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
public long openScanner(Text regionName, Text[] cols, Text firstRow,
|
public long openScanner(Text regionName, Text[] cols, Text firstRow,
|
||||||
final long timestamp, final RowFilterInterface filter)
|
final long timestamp, final RowFilterInterface filter)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
checkOpen();
|
checkOpen();
|
||||||
requestCount.incrementAndGet();
|
requestCount.incrementAndGet();
|
||||||
try {
|
try {
|
||||||
|
@ -1148,7 +1170,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
synchronized(scanners) {
|
synchronized(scanners) {
|
||||||
scanners.put(scannerName, s);
|
scanners.put(scannerName, s);
|
||||||
}
|
}
|
||||||
leases.createLease(scannerId, scannerId, new ScannerListener(scannerName));
|
this.leases.
|
||||||
|
createLease(scannerId, scannerId, new ScannerListener(scannerName));
|
||||||
return scannerId;
|
return scannerId;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")",
|
LOG.error("Error opening scanner (fsOk: " + this.fsOk + ")",
|
||||||
|
@ -1172,8 +1195,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
throw new UnknownScannerException(scannerName);
|
throw new UnknownScannerException(scannerName);
|
||||||
}
|
}
|
||||||
s.close();
|
s.close();
|
||||||
leases.cancelLease(scannerId, scannerId);
|
this.leases.cancelLease(scannerId, scannerId);
|
||||||
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
checkFileSystem();
|
checkFileSystem();
|
||||||
throw e;
|
throw e;
|
||||||
|
@ -1250,6 +1272,24 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
||||||
region.commit(lockid, timestamp);
|
region.commit(lockid, timestamp);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Info on this server.
|
||||||
|
*/
|
||||||
|
public HServerInfo getServerInfo() {
|
||||||
|
return this.serverInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Immutable list of this servers regions.
|
||||||
|
*/
|
||||||
|
public SortedMap<Text, HRegion> getOnlineRegions() {
|
||||||
|
return Collections.unmodifiableSortedMap(this.onlineRegions);
|
||||||
|
}
|
||||||
|
|
||||||
|
public AtomicInteger getRequestCount() {
|
||||||
|
return this.requestCount;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protected utility method for safely obtaining an HRegion handle.
|
* Protected utility method for safely obtaining an HRegion handle.
|
||||||
* @param regionName Name of online {@link HRegion} to return
|
* @param regionName Name of online {@link HRegion} to return
|
||||||
|
|
|
@ -34,23 +34,25 @@ public class HServerInfo implements Writable {
|
||||||
private HServerAddress serverAddress;
|
private HServerAddress serverAddress;
|
||||||
private long startCode;
|
private long startCode;
|
||||||
private HServerLoad load;
|
private HServerLoad load;
|
||||||
|
private int infoPort;
|
||||||
|
|
||||||
/** default constructor - used by Writable */
|
/** default constructor - used by Writable */
|
||||||
public HServerInfo() {
|
public HServerInfo() {
|
||||||
this.serverAddress = new HServerAddress();
|
this(new HServerAddress(), 0, HConstants.DEFAULT_REGIONSERVER_INFOPORT);
|
||||||
this.startCode = 0;
|
|
||||||
this.load = new HServerLoad();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor
|
* Constructor
|
||||||
* @param serverAddress
|
* @param serverAddress
|
||||||
* @param startCode
|
* @param startCode
|
||||||
|
* @param infoPort Port the info server is listening on.
|
||||||
*/
|
*/
|
||||||
public HServerInfo(HServerAddress serverAddress, long startCode) {
|
public HServerInfo(HServerAddress serverAddress, long startCode,
|
||||||
this.serverAddress = new HServerAddress(serverAddress);
|
final int infoPort) {
|
||||||
|
this.serverAddress = serverAddress;
|
||||||
this.startCode = startCode;
|
this.startCode = startCode;
|
||||||
this.load = new HServerLoad();
|
this.load = new HServerLoad();
|
||||||
|
this.infoPort = infoPort;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -61,6 +63,7 @@ public class HServerInfo implements Writable {
|
||||||
this.serverAddress = new HServerAddress(other.getServerAddress());
|
this.serverAddress = new HServerAddress(other.getServerAddress());
|
||||||
this.startCode = other.getStartCode();
|
this.startCode = other.getStartCode();
|
||||||
this.load = other.getLoad();
|
this.load = other.getLoad();
|
||||||
|
this.infoPort = other.getInfoPort();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -87,6 +90,13 @@ public class HServerInfo implements Writable {
|
||||||
return startCode;
|
return startCode;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Port the info server is listening on.
|
||||||
|
*/
|
||||||
|
public int getInfoPort() {
|
||||||
|
return this.infoPort;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param startCode the startCode to set
|
* @param startCode the startCode to set
|
||||||
*/
|
*/
|
||||||
|
@ -101,13 +111,40 @@ public class HServerInfo implements Writable {
|
||||||
+ ", load: (" + this.load.toString() + ")";
|
+ ", load: (" + this.load.toString() + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
// Writable
|
@Override
|
||||||
|
public boolean equals(Object obj) {
|
||||||
|
if (!(obj instanceof HServerInfo)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
HServerInfo that = (HServerInfo)obj;
|
||||||
|
if (!this.serverAddress.equals(that.serverAddress)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (this.infoPort != that.infoPort) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (this.startCode != that.startCode) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
int result = this.serverAddress.hashCode();
|
||||||
|
result ^= this.infoPort;
|
||||||
|
result ^= this.startCode;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
// Writable
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
public void readFields(DataInput in) throws IOException {
|
public void readFields(DataInput in) throws IOException {
|
||||||
this.serverAddress.readFields(in);
|
this.serverAddress.readFields(in);
|
||||||
this.startCode = in.readLong();
|
this.startCode = in.readLong();
|
||||||
this.load.readFields(in);
|
this.load.readFields(in);
|
||||||
|
this.infoPort = in.readInt();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
|
@ -115,5 +152,6 @@ public class HServerInfo implements Writable {
|
||||||
this.serverAddress.write(out);
|
this.serverAddress.write(out);
|
||||||
out.writeLong(this.startCode);
|
out.writeLong(this.startCode);
|
||||||
this.load.write(out);
|
this.load.write(out);
|
||||||
|
out.writeInt(this.infoPort);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -87,7 +87,8 @@ public class Leases {
|
||||||
public void closeAfterLeasesExpire() {
|
public void closeAfterLeasesExpire() {
|
||||||
synchronized(this.leases) {
|
synchronized(this.leases) {
|
||||||
while (this.leases.size() > 0) {
|
while (this.leases.size() > 0) {
|
||||||
LOG.info(Integer.toString(leases.size()) + " lease(s) " +
|
LOG.info(Thread.currentThread().getName() + " " +
|
||||||
|
Integer.toString(leases.size()) + " lease(s) " +
|
||||||
"outstanding. Waiting for them to expire.");
|
"outstanding. Waiting for them to expire.");
|
||||||
try {
|
try {
|
||||||
this.leases.wait(this.leaseCheckFrequency);
|
this.leases.wait(this.leaseCheckFrequency);
|
||||||
|
@ -105,7 +106,7 @@ public class Leases {
|
||||||
* without any cancellation calls.
|
* without any cancellation calls.
|
||||||
*/
|
*/
|
||||||
public void close() {
|
public void close() {
|
||||||
LOG.info("closing leases");
|
LOG.info(Thread.currentThread().getName() + " closing leases");
|
||||||
this.stop.set(true);
|
this.stop.set(true);
|
||||||
try {
|
try {
|
||||||
this.leaseMonitorThread.interrupt();
|
this.leaseMonitorThread.interrupt();
|
||||||
|
@ -119,7 +120,7 @@ public class Leases {
|
||||||
sortedLeases.clear();
|
sortedLeases.clear();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
LOG.info("leases closed");
|
LOG.info(Thread.currentThread().getName() + " closed leases");
|
||||||
}
|
}
|
||||||
|
|
||||||
/* A client obtains a lease... */
|
/* A client obtains a lease... */
|
||||||
|
@ -330,8 +331,8 @@ public class Leases {
|
||||||
}
|
}
|
||||||
|
|
||||||
void expired() {
|
void expired() {
|
||||||
LOG.info("Lease expired " + getLeaseName());
|
LOG.info(Thread.currentThread().getName() + " lease expired " +
|
||||||
|
getLeaseName());
|
||||||
listener.leaseExpired();
|
listener.leaseExpired();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -20,13 +20,16 @@
|
||||||
package org.apache.hadoop.hbase;
|
package org.apache.hadoop.hbase;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import jline.ConsoleReader;
|
import jline.ConsoleReader;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.shell.Command;
|
import org.apache.hadoop.hbase.shell.Command;
|
||||||
import org.apache.hadoop.hbase.shell.HelpManager;
|
import org.apache.hadoop.hbase.shell.HelpCommand;
|
||||||
import org.apache.hadoop.hbase.shell.ReturnMsg;
|
import org.apache.hadoop.hbase.shell.ReturnMsg;
|
||||||
|
import org.apache.hadoop.hbase.shell.TableFormatterFactory;
|
||||||
import org.apache.hadoop.hbase.shell.generated.ParseException;
|
import org.apache.hadoop.hbase.shell.generated.ParseException;
|
||||||
import org.apache.hadoop.hbase.shell.generated.Parser;
|
import org.apache.hadoop.hbase.shell.generated.Parser;
|
||||||
import org.apache.hadoop.hbase.shell.generated.TokenMgrError;
|
import org.apache.hadoop.hbase.shell.generated.TokenMgrError;
|
||||||
|
@ -40,6 +43,30 @@ public class Shell {
|
||||||
/** audible keyboard bells */
|
/** audible keyboard bells */
|
||||||
public static final boolean DEFAULT_BELL_ENABLED = true;
|
public static final boolean DEFAULT_BELL_ENABLED = true;
|
||||||
|
|
||||||
|
|
||||||
|
/** Return the boolean value indicating whether end of command or not */
|
||||||
|
static boolean isEndOfCommand(String line) {
|
||||||
|
return (line.lastIndexOf(';') > -1) ? true : false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return the string of prompt start string */
|
||||||
|
private static String getPrompt(final StringBuilder queryStr) {
|
||||||
|
return (queryStr.toString().equals("")) ? "Hbase> " : " --> ";
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param watch true if execution time should be computed and returned
|
||||||
|
* @param start start of time interval
|
||||||
|
* @param end end of time interval
|
||||||
|
* @return a string of code execution time. */
|
||||||
|
public static String executeTime(boolean watch, long start, long end) {
|
||||||
|
return watch?
|
||||||
|
" (" + String.format("%.2f", Double.valueOf((end - start) * 0.001)) +
|
||||||
|
" sec)":
|
||||||
|
"";
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Main method
|
* Main method
|
||||||
* @param args not used
|
* @param args not used
|
||||||
|
@ -51,7 +78,9 @@ public class Shell {
|
||||||
ConsoleReader reader = new ConsoleReader();
|
ConsoleReader reader = new ConsoleReader();
|
||||||
reader.setBellEnabled(conf.getBoolean("hbaseshell.jline.bell.enabled",
|
reader.setBellEnabled(conf.getBoolean("hbaseshell.jline.bell.enabled",
|
||||||
DEFAULT_BELL_ENABLED));
|
DEFAULT_BELL_ENABLED));
|
||||||
HelpManager help = new HelpManager();
|
Writer out = new OutputStreamWriter(System.out, "UTF-8");
|
||||||
|
TableFormatterFactory tff = new TableFormatterFactory(out, conf);
|
||||||
|
HelpCommand help = new HelpCommand(out, tff.get());
|
||||||
help.printVersion();
|
help.printVersion();
|
||||||
StringBuilder queryStr = new StringBuilder();
|
StringBuilder queryStr = new StringBuilder();
|
||||||
String extendedLine;
|
String extendedLine;
|
||||||
|
@ -59,7 +88,7 @@ public class Shell {
|
||||||
if (isEndOfCommand(extendedLine)) {
|
if (isEndOfCommand(extendedLine)) {
|
||||||
queryStr.append(" " + extendedLine);
|
queryStr.append(" " + extendedLine);
|
||||||
long start = System.currentTimeMillis();
|
long start = System.currentTimeMillis();
|
||||||
Parser parser = new Parser(queryStr.toString());
|
Parser parser = new Parser(queryStr.toString(), out, tff.get());
|
||||||
ReturnMsg rs = null;
|
ReturnMsg rs = null;
|
||||||
try {
|
try {
|
||||||
Command cmd = parser.terminatedCommand();
|
Command cmd = parser.terminatedCommand();
|
||||||
|
@ -85,26 +114,4 @@ public class Shell {
|
||||||
}
|
}
|
||||||
System.out.println();
|
System.out.println();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Return the boolean value indicating whether end of command or not */
|
|
||||||
static boolean isEndOfCommand(String line) {
|
|
||||||
return (line.lastIndexOf(';') > -1) ? true : false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Return the string of prompt start string */
|
|
||||||
private static String getPrompt(final StringBuilder queryStr) {
|
|
||||||
return (queryStr.toString().equals("")) ? "Hbase> " : " --> ";
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param watch true if execution time should be computed and returned
|
|
||||||
* @param start start of time interval
|
|
||||||
* @param end end of time interval
|
|
||||||
* @return a string of code execution time. */
|
|
||||||
public static String executeTime(boolean watch, long start, long end) {
|
|
||||||
return watch?
|
|
||||||
" (" + String.format("%.2f", Double.valueOf((end - start) * 0.001)) +
|
|
||||||
" sec)":
|
|
||||||
"";
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -33,7 +34,6 @@ import org.apache.hadoop.io.Text;
|
||||||
* Alters tables.
|
* Alters tables.
|
||||||
*/
|
*/
|
||||||
public class AlterCommand extends SchemaModificationCommand {
|
public class AlterCommand extends SchemaModificationCommand {
|
||||||
|
|
||||||
public enum OperationType {ADD, DROP, CHANGE, NOOP}
|
public enum OperationType {ADD, DROP, CHANGE, NOOP}
|
||||||
private OperationType operationType = OperationType.NOOP;
|
private OperationType operationType = OperationType.NOOP;
|
||||||
private Map<String, Map<String, Object>> columnSpecMap =
|
private Map<String, Map<String, Object>> columnSpecMap =
|
||||||
|
@ -41,6 +41,10 @@ public class AlterCommand extends SchemaModificationCommand {
|
||||||
private String table;
|
private String table;
|
||||||
private String column; // column to be dropped
|
private String column; // column to be dropped
|
||||||
|
|
||||||
|
public AlterCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
try {
|
try {
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
|
@ -52,7 +56,7 @@ public class AlterCommand extends SchemaModificationCommand {
|
||||||
columns = columnSpecMap.keySet();
|
columns = columnSpecMap.keySet();
|
||||||
for (String c : columns) {
|
for (String c : columns) {
|
||||||
columnDesc = getColumnDescriptor(c, columnSpecMap.get(c));
|
columnDesc = getColumnDescriptor(c, columnSpecMap.get(c));
|
||||||
System.out.println("Adding " + c + " to " + table +
|
println("Adding " + c + " to " + table +
|
||||||
"... Please wait.");
|
"... Please wait.");
|
||||||
admin.addColumn(new Text(table), columnDesc);
|
admin.addColumn(new Text(table), columnDesc);
|
||||||
}
|
}
|
||||||
|
@ -60,7 +64,7 @@ public class AlterCommand extends SchemaModificationCommand {
|
||||||
break;
|
break;
|
||||||
case DROP:
|
case DROP:
|
||||||
disableTable(admin, table);
|
disableTable(admin, table);
|
||||||
System.out.println("Dropping " + column + " from " + table +
|
println("Dropping " + column + " from " + table +
|
||||||
"... Please wait.");
|
"... Please wait.");
|
||||||
column = appendDelimiter(column);
|
column = appendDelimiter(column);
|
||||||
admin.deleteColumn(new Text(table), new Text(column));
|
admin.deleteColumn(new Text(table), new Text(column));
|
||||||
|
@ -79,12 +83,12 @@ public class AlterCommand extends SchemaModificationCommand {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void disableTable(HBaseAdmin admin, String t) throws IOException {
|
private void disableTable(HBaseAdmin admin, String t) throws IOException {
|
||||||
System.out.println("Disabling " + t + "... Please wait.");
|
println("Disabling " + t + "... Please wait.");
|
||||||
admin.disableTable(new Text(t));
|
admin.disableTable(new Text(t));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void enableTable(HBaseAdmin admin, String t) throws IOException {
|
private void enableTable(HBaseAdmin admin, String t) throws IOException {
|
||||||
System.out.println("Enabling " + t + "... Please wait.");
|
println("Enabling " + t + "... Please wait.");
|
||||||
admin.enableTable(new Text(t));
|
admin.enableTable(new Text(t));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -124,4 +128,9 @@ public class AlterCommand extends SchemaModificationCommand {
|
||||||
public void setOperationType(OperationType operationType) {
|
public void setOperationType(OperationType operationType) {
|
||||||
this.operationType = operationType;
|
this.operationType = operationType;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DDL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -19,10 +19,31 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Takes the lowest-common-denominator {@link Writer} doing its own printlns,
|
||||||
|
* etc.
|
||||||
* @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseShell">HBaseShell</a>
|
* @see <a href="http://wiki.apache.org/lucene-hadoop/Hbase/HbaseShell">HBaseShell</a>
|
||||||
*/
|
*/
|
||||||
public abstract class BasicCommand implements Command, CommandFactory {
|
public abstract class BasicCommand implements Command, CommandFactory {
|
||||||
|
private final Writer out;
|
||||||
|
public final String LINE_SEPARATOR = System.getProperty("line.separator");
|
||||||
|
|
||||||
|
// Shutdown constructor.
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
private BasicCommand() {
|
||||||
|
this(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor
|
||||||
|
* @param o A Writer.
|
||||||
|
*/
|
||||||
|
public BasicCommand(final Writer o) {
|
||||||
|
this.out = o;
|
||||||
|
}
|
||||||
|
|
||||||
public BasicCommand getBasicCommand() {
|
public BasicCommand getBasicCommand() {
|
||||||
return this;
|
return this;
|
||||||
|
@ -51,4 +72,25 @@ public abstract class BasicCommand implements Command, CommandFactory {
|
||||||
return (!column.endsWith(FAMILY_INDICATOR))?
|
return (!column.endsWith(FAMILY_INDICATOR))?
|
||||||
column + FAMILY_INDICATOR: column;
|
column + FAMILY_INDICATOR: column;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Writer to use outputting.
|
||||||
|
*/
|
||||||
|
public Writer getOut() {
|
||||||
|
return this.out;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void print(final String msg) throws IOException {
|
||||||
|
this.out.write(msg);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void println(final String msg) throws IOException {
|
||||||
|
print(msg);
|
||||||
|
print(LINE_SEPARATOR);
|
||||||
|
this.out.flush();
|
||||||
|
}
|
||||||
|
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.SELECT;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
|
||||||
|
@ -27,21 +28,38 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
* Clears the console screen.
|
* Clears the console screen.
|
||||||
*/
|
*/
|
||||||
public class ClearCommand extends BasicCommand {
|
public class ClearCommand extends BasicCommand {
|
||||||
|
public ClearCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
||||||
clear();
|
clear();
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
static void clear() {
|
private void clear() {
|
||||||
String osName = System.getProperty("os.name");
|
String osName = System.getProperty("os.name");
|
||||||
if (osName.length() > 7 && osName.subSequence(0, 7).equals("Windows")) {
|
if (osName.length() > 7 && osName.subSequence(0, 7).equals("Windows")) {
|
||||||
try {
|
try {
|
||||||
Runtime.getRuntime().exec("cmd /C cls");
|
Runtime.getRuntime().exec("cmd /C cls");
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
System.out.println("Can't clear." + e.toString());
|
try {
|
||||||
|
println("Can't clear." + e.toString());
|
||||||
|
} catch (IOException e1) {
|
||||||
|
e1.printStackTrace();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
System.out.print("\033c");
|
try {
|
||||||
|
print("\033c");
|
||||||
|
} catch (IOException e) {
|
||||||
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.SHELL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -25,9 +25,17 @@ public interface Command {
|
||||||
/** family indicator */
|
/** family indicator */
|
||||||
public static final String FAMILY_INDICATOR = ":";
|
public static final String FAMILY_INDICATOR = ":";
|
||||||
|
|
||||||
|
public enum CommandType {DDL, UPDATE, SELECT, INSERT, DELETE, SHELL}
|
||||||
|
|
||||||
/** Execute a command
|
/** Execute a command
|
||||||
* @param conf Configuration
|
* @param conf Configuration
|
||||||
* @return Result of command execution
|
* @return Result of command execution
|
||||||
*/
|
*/
|
||||||
public ReturnMsg execute(Configuration conf);
|
public ReturnMsg execute(final Configuration conf);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Type of this command whether DDL, SELECT, INSERT, UPDATE, DELETE,
|
||||||
|
* or SHELL.
|
||||||
|
*/
|
||||||
|
public CommandType getCommandType();
|
||||||
}
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -32,11 +33,14 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
* Creates tables.
|
* Creates tables.
|
||||||
*/
|
*/
|
||||||
public class CreateCommand extends SchemaModificationCommand {
|
public class CreateCommand extends SchemaModificationCommand {
|
||||||
|
|
||||||
private String tableName;
|
private String tableName;
|
||||||
private Map<String, Map<String, Object>> columnSpecMap =
|
private Map<String, Map<String, Object>> columnSpecMap =
|
||||||
new HashMap<String, Map<String, Object>>();
|
new HashMap<String, Map<String, Object>>();
|
||||||
|
|
||||||
|
public CreateCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
try {
|
try {
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
|
@ -48,7 +52,7 @@ public class CreateCommand extends SchemaModificationCommand {
|
||||||
tableDesc.addFamily(columnDesc);
|
tableDesc.addFamily(columnDesc);
|
||||||
}
|
}
|
||||||
|
|
||||||
System.out.println("Creating table... Please wait.");
|
println("Creating table... Please wait.");
|
||||||
|
|
||||||
admin.createTable(tableDesc);
|
admin.createTable(tableDesc);
|
||||||
return new ReturnMsg(0, "Table created successfully.");
|
return new ReturnMsg(0, "Table created successfully.");
|
||||||
|
@ -73,4 +77,9 @@ public class CreateCommand extends SchemaModificationCommand {
|
||||||
public void addColumnSpec(String column, Map<String, Object> columnSpec) {
|
public void addColumnSpec(String column, Map<String, Object> columnSpec) {
|
||||||
columnSpecMap.put(column, columnSpec);
|
columnSpecMap.put(column, columnSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DDL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -32,6 +33,9 @@ import org.apache.hadoop.io.Text;
|
||||||
* Deletes values from tables.
|
* Deletes values from tables.
|
||||||
*/
|
*/
|
||||||
public class DeleteCommand extends BasicCommand {
|
public class DeleteCommand extends BasicCommand {
|
||||||
|
public DeleteCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
private String tableName;
|
private String tableName;
|
||||||
private String rowKey;
|
private String rowKey;
|
||||||
|
@ -100,4 +104,9 @@ public class DeleteCommand extends BasicCommand {
|
||||||
}
|
}
|
||||||
return columns;
|
return columns;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DELETE;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
@ -35,8 +36,20 @@ public class DescCommand extends BasicCommand {
|
||||||
private static final String [] HEADER =
|
private static final String [] HEADER =
|
||||||
new String [] {"Column Family Descriptor"};
|
new String [] {"Column Family Descriptor"};
|
||||||
private Text tableName;
|
private Text tableName;
|
||||||
|
private final TableFormatter formatter;
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
// Not instantiable
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
private DescCommand() {
|
||||||
|
this(null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public DescCommand(final Writer o, final TableFormatter f) {
|
||||||
|
super(o);
|
||||||
|
this.formatter = f;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ReturnMsg execute(final Configuration conf) {
|
||||||
if (this.tableName == null)
|
if (this.tableName == null)
|
||||||
return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax");
|
return new ReturnMsg(0, "Syntax error : Please check 'Describe' syntax");
|
||||||
try {
|
try {
|
||||||
|
@ -53,7 +66,6 @@ public class DescCommand extends BasicCommand {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
TableFormatter formatter = TableFormatterFactory.get();
|
|
||||||
formatter.header(HEADER);
|
formatter.header(HEADER);
|
||||||
// Do a toString on the HColumnDescriptors
|
// Do a toString on the HColumnDescriptors
|
||||||
String [] columnStrs = new String[columns.length];
|
String [] columnStrs = new String[columns.length];
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HBaseAdmin;
|
import org.apache.hadoop.hbase.HBaseAdmin;
|
||||||
|
@ -31,6 +32,10 @@ import org.apache.hadoop.io.Text;
|
||||||
public class DisableCommand extends BasicCommand {
|
public class DisableCommand extends BasicCommand {
|
||||||
private String tableName;
|
private String tableName;
|
||||||
|
|
||||||
|
public DisableCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
assert tableName != null;
|
assert tableName != null;
|
||||||
|
|
||||||
|
@ -48,4 +53,9 @@ public class DisableCommand extends BasicCommand {
|
||||||
public void setTable(String table) {
|
public void setTable(String table) {
|
||||||
this.tableName = table;
|
this.tableName = table;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DDL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -32,6 +33,10 @@ import org.apache.hadoop.io.Text;
|
||||||
public class DropCommand extends BasicCommand {
|
public class DropCommand extends BasicCommand {
|
||||||
private List<String> tableList;
|
private List<String> tableList;
|
||||||
|
|
||||||
|
public DropCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
if (tableList == null) {
|
if (tableList == null) {
|
||||||
throw new IllegalArgumentException("List of tables is null");
|
throw new IllegalArgumentException("List of tables is null");
|
||||||
|
@ -41,7 +46,7 @@ public class DropCommand extends BasicCommand {
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
|
|
||||||
for (String table : tableList) {
|
for (String table : tableList) {
|
||||||
System.out.println("Dropping " + table + "... Please wait.");
|
println("Dropping " + table + "... Please wait.");
|
||||||
admin.deleteTable(new Text(table));
|
admin.deleteTable(new Text(table));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,4 +59,9 @@ public class DropCommand extends BasicCommand {
|
||||||
public void setTableList(List<String> tableList) {
|
public void setTableList(List<String> tableList) {
|
||||||
this.tableList = tableList;
|
this.tableList = tableList;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DDL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HBaseAdmin;
|
import org.apache.hadoop.hbase.HBaseAdmin;
|
||||||
|
@ -31,12 +32,15 @@ import org.apache.hadoop.io.Text;
|
||||||
public class EnableCommand extends BasicCommand {
|
public class EnableCommand extends BasicCommand {
|
||||||
private String tableName;
|
private String tableName;
|
||||||
|
|
||||||
|
public EnableCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
assert tableName != null;
|
assert tableName != null;
|
||||||
try {
|
try {
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||||
admin.enableTable(new Text(tableName));
|
admin.enableTable(new Text(tableName));
|
||||||
|
|
||||||
return new ReturnMsg(1, "Table enabled successfully.");
|
return new ReturnMsg(1, "Table enabled successfully.");
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
String[] msg = e.getMessage().split("[\n]");
|
String[] msg = e.getMessage().split("[\n]");
|
||||||
|
@ -47,4 +51,9 @@ public class EnableCommand extends BasicCommand {
|
||||||
public void setTable(String table) {
|
public void setTable(String table) {
|
||||||
this.tableName = table;
|
this.tableName = table;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.DDL;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -19,9 +19,14 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
|
||||||
public class ExitCommand extends BasicCommand {
|
public class ExitCommand extends BasicCommand {
|
||||||
|
public ExitCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
||||||
// TOD: Is this the best way to exit? Would be a problem if shell is run
|
// TOD: Is this the best way to exit? Would be a problem if shell is run
|
||||||
|
@ -29,4 +34,9 @@ public class ExitCommand extends BasicCommand {
|
||||||
System.exit(1);
|
System.exit(1);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.SHELL;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -31,7 +32,12 @@ import org.apache.hadoop.util.ToolRunner;
|
||||||
public class FsCommand extends BasicCommand {
|
public class FsCommand extends BasicCommand {
|
||||||
private List<String> query;
|
private List<String> query;
|
||||||
|
|
||||||
|
public FsCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
||||||
|
// This commmand will write the
|
||||||
FsShell shell = new FsShell();
|
FsShell shell = new FsShell();
|
||||||
try {
|
try {
|
||||||
ToolRunner.run(shell, getQuery());
|
ToolRunner.run(shell, getQuery());
|
||||||
|
@ -49,4 +55,9 @@ public class FsCommand extends BasicCommand {
|
||||||
private String[] getQuery() {
|
private String[] getQuery() {
|
||||||
return query.toArray(new String[] {});
|
return query.toArray(new String[] {});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.SHELL;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -32,6 +32,7 @@ import java.util.Map;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.io.StringReader;
|
import java.io.StringReader;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shell.*;
|
import org.apache.hadoop.hbase.shell.*;
|
||||||
|
|
||||||
|
@ -40,10 +41,14 @@ import org.apache.hadoop.hbase.shell.*;
|
||||||
*/
|
*/
|
||||||
public class Parser {
|
public class Parser {
|
||||||
private String QueryString;
|
private String QueryString;
|
||||||
|
private TableFormatter formatter;
|
||||||
|
private Writer out;
|
||||||
|
|
||||||
public Parser(String query) {
|
public Parser(final String query, final Writer o, final TableFormatter f) {
|
||||||
this((Reader)(new StringReader(query)));
|
this((Reader)(new StringReader(query)));
|
||||||
this.QueryString = query;
|
this.QueryString = query;
|
||||||
|
this.formatter = f;
|
||||||
|
this.out = o;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getQueryStr() {
|
public String getQueryStr() {
|
||||||
|
@ -174,7 +179,7 @@ Command cmdStatement() :
|
||||||
|
|
||||||
ExitCommand exitCommand() :
|
ExitCommand exitCommand() :
|
||||||
{
|
{
|
||||||
ExitCommand exit = new ExitCommand();
|
ExitCommand exit = new ExitCommand(this.out);
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
<EXIT> { return exit; }
|
<EXIT> { return exit; }
|
||||||
|
@ -183,7 +188,7 @@ ExitCommand exitCommand() :
|
||||||
FsCommand fsCommand() :
|
FsCommand fsCommand() :
|
||||||
{
|
{
|
||||||
Token t = null;
|
Token t = null;
|
||||||
FsCommand fs = new FsCommand();
|
FsCommand fs = new FsCommand(this.out);
|
||||||
List<String> query = new ArrayList<String>();
|
List<String> query = new ArrayList<String>();
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -202,7 +207,7 @@ FsCommand fsCommand() :
|
||||||
JarCommand jarCommand() :
|
JarCommand jarCommand() :
|
||||||
{
|
{
|
||||||
Token t = null;
|
Token t = null;
|
||||||
JarCommand jar = new JarCommand();
|
JarCommand jar = new JarCommand(this.out);
|
||||||
List<String> query = new ArrayList<String>();
|
List<String> query = new ArrayList<String>();
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -221,7 +226,7 @@ JarCommand jarCommand() :
|
||||||
HelpCommand helpCommand() :
|
HelpCommand helpCommand() :
|
||||||
{
|
{
|
||||||
Token t = null;
|
Token t = null;
|
||||||
HelpCommand help = new HelpCommand();
|
HelpCommand help = new HelpCommand(this.out, this.formatter);
|
||||||
String argument = "";
|
String argument = "";
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -252,7 +257,7 @@ HelpCommand helpCommand() :
|
||||||
|
|
||||||
ShowCommand showCommand() :
|
ShowCommand showCommand() :
|
||||||
{
|
{
|
||||||
ShowCommand show = new ShowCommand();
|
ShowCommand show = new ShowCommand(this.out, this.formatter);
|
||||||
String argument = null;
|
String argument = null;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -268,7 +273,7 @@ ShowCommand showCommand() :
|
||||||
|
|
||||||
DescCommand descCommand() :
|
DescCommand descCommand() :
|
||||||
{
|
{
|
||||||
DescCommand desc = new DescCommand();
|
DescCommand desc = new DescCommand(this.out, this.formatter);
|
||||||
String argument = null;
|
String argument = null;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -347,7 +352,7 @@ Map<String, Object> ColumnSpec() :
|
||||||
|
|
||||||
CreateCommand createCommand() :
|
CreateCommand createCommand() :
|
||||||
{
|
{
|
||||||
CreateCommand createCommand = new CreateCommand();
|
CreateCommand createCommand = new CreateCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
Map<String, Object> columnSpec = null;
|
Map<String, Object> columnSpec = null;
|
||||||
String column = null;
|
String column = null;
|
||||||
|
@ -383,7 +388,7 @@ CreateCommand createCommand() :
|
||||||
|
|
||||||
AlterCommand alterCommand() :
|
AlterCommand alterCommand() :
|
||||||
{
|
{
|
||||||
AlterCommand alterCommand = new AlterCommand();
|
AlterCommand alterCommand = new AlterCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
String column = null;
|
String column = null;
|
||||||
Map<String, Object> columnSpec = null;
|
Map<String, Object> columnSpec = null;
|
||||||
|
@ -439,7 +444,7 @@ AlterCommand alterCommand() :
|
||||||
|
|
||||||
DropCommand dropCommand() :
|
DropCommand dropCommand() :
|
||||||
{
|
{
|
||||||
DropCommand drop = new DropCommand();
|
DropCommand drop = new DropCommand(this.out);
|
||||||
List<String> tableList = null;
|
List<String> tableList = null;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -454,7 +459,7 @@ DropCommand dropCommand() :
|
||||||
|
|
||||||
InsertCommand insertCommand() :
|
InsertCommand insertCommand() :
|
||||||
{
|
{
|
||||||
InsertCommand in = new InsertCommand();
|
InsertCommand in = new InsertCommand(this.out);
|
||||||
List<String> columnfamilies = null;
|
List<String> columnfamilies = null;
|
||||||
List<String> values = null;
|
List<String> values = null;
|
||||||
String table = null;
|
String table = null;
|
||||||
|
@ -490,7 +495,7 @@ InsertCommand insertCommand() :
|
||||||
|
|
||||||
DeleteCommand deleteCommand() :
|
DeleteCommand deleteCommand() :
|
||||||
{
|
{
|
||||||
DeleteCommand deleteCommand = new DeleteCommand();
|
DeleteCommand deleteCommand = new DeleteCommand(this.out);
|
||||||
List<String> columnList = null;
|
List<String> columnList = null;
|
||||||
Token t = null;
|
Token t = null;
|
||||||
String table = null;
|
String table = null;
|
||||||
|
@ -519,7 +524,7 @@ DeleteCommand deleteCommand() :
|
||||||
|
|
||||||
SelectCommand selectCommand() :
|
SelectCommand selectCommand() :
|
||||||
{
|
{
|
||||||
SelectCommand select = new SelectCommand();
|
SelectCommand select = new SelectCommand(this.out, this.formatter);
|
||||||
List<String> columns = null;
|
List<String> columns = null;
|
||||||
String rowKey = "";
|
String rowKey = "";
|
||||||
String timestamp = null;
|
String timestamp = null;
|
||||||
|
@ -573,7 +578,7 @@ SelectCommand selectCommand() :
|
||||||
|
|
||||||
EnableCommand enableCommand() :
|
EnableCommand enableCommand() :
|
||||||
{
|
{
|
||||||
EnableCommand enableCommand = new EnableCommand();
|
EnableCommand enableCommand = new EnableCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -587,7 +592,7 @@ EnableCommand enableCommand() :
|
||||||
|
|
||||||
DisableCommand disableCommand() :
|
DisableCommand disableCommand() :
|
||||||
{
|
{
|
||||||
DisableCommand disableCommand = new DisableCommand();
|
DisableCommand disableCommand = new DisableCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
|
@ -601,7 +606,7 @@ DisableCommand disableCommand() :
|
||||||
|
|
||||||
ClearCommand clearCommand() :
|
ClearCommand clearCommand() :
|
||||||
{
|
{
|
||||||
ClearCommand clear = new ClearCommand();
|
ClearCommand clear = new ClearCommand(this.out);
|
||||||
}
|
}
|
||||||
{
|
{
|
||||||
<CLEAR>
|
<CLEAR>
|
||||||
|
|
|
@ -19,17 +19,164 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
|
import java.io.UnsupportedEncodingException;
|
||||||
|
import java.io.Writer;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
|
||||||
public class HelpCommand extends BasicCommand {
|
public class HelpCommand extends BasicCommand {
|
||||||
private String argument;
|
private String argument;
|
||||||
|
private static final String [] HEADER =
|
||||||
|
new String [] {"Command", "Description", "Example"};
|
||||||
|
|
||||||
|
/** application name */
|
||||||
|
public static final String APP_NAME = "Hbase Shell";
|
||||||
|
|
||||||
|
/** version of the code */
|
||||||
|
public static final String APP_VERSION = "0.0.2";
|
||||||
|
|
||||||
|
/** help contents map */
|
||||||
|
public final Map<String, String[]> help =
|
||||||
|
new HashMap<String, String[]>();
|
||||||
|
|
||||||
|
private final TableFormatter formatter;
|
||||||
|
|
||||||
|
public HelpCommand(final Writer o, final TableFormatter f) {
|
||||||
|
super(o);
|
||||||
|
this.help.putAll(load());
|
||||||
|
this.formatter = f;
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
||||||
HelpManager.printHelp(this.argument);
|
try {
|
||||||
|
printHelp(this.argument);
|
||||||
|
} catch (IOException e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setArgument(String argument) {
|
public void setArgument(String argument) {
|
||||||
this.argument = argument;
|
this.argument = argument;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* add help contents
|
||||||
|
*/
|
||||||
|
private Map<? extends String, ? extends String[]> load() {
|
||||||
|
Map<String, String[]> load = new HashMap<String, String[]>();
|
||||||
|
load.put("SHOW", new String[] {"List all user tables", "SHOW TABLES;"});
|
||||||
|
|
||||||
|
load.put("FS", new String[] { "Hadoop FsShell; entering a lone 'FS;' " +
|
||||||
|
"will emit usage",
|
||||||
|
"FS -copyFromLocal /home/user/backup.dat fs/user/backup;"});
|
||||||
|
|
||||||
|
load.put("JAR", new String[] { "Hadoop RunJar util",
|
||||||
|
"JAR ./build/hadoop-examples.jar pi 10 10;"});
|
||||||
|
load.put("CLEAR", new String[] {"Clear the screen", "CLEAR;"} );
|
||||||
|
|
||||||
|
load.put("DESCRIBE", new String[] { "Print table information",
|
||||||
|
"[DESCRIBE|DESC] table_name;"});
|
||||||
|
|
||||||
|
load.put("CREATE", new String[] {
|
||||||
|
"Create tables",
|
||||||
|
"CREATE TABLE table_name (column_family_name [MAX_VERSIONS=n] " +
|
||||||
|
"[MAX_LENGTH=n] [COMPRESSION=NONE|RECORD|BLOCK] [IN_MEMORY] " +
|
||||||
|
"[BLOOMFILTER=NONE|BLOOM|COUNTING|RETOUCHED VECTOR_SIZE=n NUM_HASH=n], " +
|
||||||
|
"...)"
|
||||||
|
});
|
||||||
|
load.put("DROP", new String[] {
|
||||||
|
"Drop tables",
|
||||||
|
"DROP TABLE table_name [, table_name] ...;"});
|
||||||
|
|
||||||
|
load.put("INSERT", new String[] {
|
||||||
|
"Insert values into table",
|
||||||
|
"INSERT INTO table_name (column_name, ...) " +
|
||||||
|
"VALUES ('value', ...) WHERE row='row_key';"
|
||||||
|
});
|
||||||
|
|
||||||
|
load.put("DELETE", new String[] {
|
||||||
|
"Delete table data",
|
||||||
|
"DELETE {column_name, [, column_name] ... | *} FROM table_name " +
|
||||||
|
"WHERE row='row-key';"
|
||||||
|
});
|
||||||
|
|
||||||
|
load.put("SELECT",
|
||||||
|
new String[] {
|
||||||
|
"Select values from table",
|
||||||
|
"SELECT {column_name, [, column_name] ... | *} FROM table_name " +
|
||||||
|
"[WHERE row='row_key' | STARTING FROM 'row-key'] " +
|
||||||
|
"[NUM_VERSIONS = version_count] " +
|
||||||
|
"[TIMESTAMP 'timestamp'] " +
|
||||||
|
"[LIMIT = row_count] " +
|
||||||
|
"[INTO FILE 'file_name'];"
|
||||||
|
});
|
||||||
|
|
||||||
|
load.put("ALTER",
|
||||||
|
new String[] {
|
||||||
|
"Alter structure of table",
|
||||||
|
"ALTER TABLE table_name ADD column_spec | " +
|
||||||
|
"ADD (column_spec, column_spec, ...) | " +
|
||||||
|
"DROP column_family_name | " +
|
||||||
|
"CHANGE column_spec;"
|
||||||
|
});
|
||||||
|
|
||||||
|
load.put("EXIT", new String[] { "Exit shell", "EXIT;"});
|
||||||
|
|
||||||
|
return load;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Print out the program version.
|
||||||
|
* @throws IOException */
|
||||||
|
public void printVersion() throws IOException {
|
||||||
|
println(APP_NAME + ", " + APP_VERSION + " version.\n"
|
||||||
|
+ "Copyright (c) 2007 by udanax, "
|
||||||
|
+ "licensed to Apache Software Foundation.\n"
|
||||||
|
+ "Type 'help;' for usage.\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
public void printHelp(final String cmd) throws IOException {
|
||||||
|
if (cmd.equals("")) {
|
||||||
|
println("Type 'help COMMAND;' to see command-specific usage.");
|
||||||
|
printHelp(this.help);
|
||||||
|
} else {
|
||||||
|
if (this.help.containsKey(cmd.toUpperCase())) {
|
||||||
|
final Map<String, String []> m = new HashMap<String, String []>();
|
||||||
|
m.put(cmd.toUpperCase(), this.help.get(cmd.toUpperCase()));
|
||||||
|
printHelp(m);
|
||||||
|
} else {
|
||||||
|
println("Unknown Command : Type 'help;' for usage.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void printHelp(final Map<String, String []> m) throws IOException {
|
||||||
|
this.formatter.header(HEADER);
|
||||||
|
for (Map.Entry<String, String []> e: m.entrySet()) {
|
||||||
|
String [] value = e.getValue();
|
||||||
|
if (value.length == 2) {
|
||||||
|
this.formatter.row(new String [] {e.getKey().toUpperCase(), value[0],
|
||||||
|
value[1]});
|
||||||
|
} else {
|
||||||
|
throw new IOException("Value has too many elements:" + value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.formatter.footer();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws UnsupportedEncodingException {
|
||||||
|
HBaseConfiguration conf = new HBaseConfiguration();
|
||||||
|
Writer out = new OutputStreamWriter(System.out, "UTF-8");
|
||||||
|
TableFormatterFactory tff = new TableFormatterFactory(out, conf);
|
||||||
|
HelpCommand cmd = new HelpCommand(out, tff.get());
|
||||||
|
cmd.setArgument("");
|
||||||
|
cmd.execute(conf);
|
||||||
|
cmd.setArgument("select");
|
||||||
|
cmd.execute(conf);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,112 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright 2007 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.shell;
|
|
||||||
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class HelpContents {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* add help contents
|
|
||||||
*/
|
|
||||||
public static Map<? extends String, ? extends String[]> Load() {
|
|
||||||
Map<String, String[]> load = new HashMap<String, String[]>();
|
|
||||||
|
|
||||||
String columnName = "column_name: "
|
|
||||||
+ "\n\t column_family_name"
|
|
||||||
+ "\n\t| column_family_name:column_label_name";
|
|
||||||
String columnList = "{column_name, [, column_name] ... | *}";
|
|
||||||
|
|
||||||
load.put("SHOW", new String[] {"List all available tables", "SHOW TABLES;"});
|
|
||||||
|
|
||||||
load.put("FS", new String[] { "Hadoop FsShell operations",
|
|
||||||
"FS -copyFromLocal /home/user/backup.dat fs/user/backup;" });
|
|
||||||
|
|
||||||
load.put("JAR", new String[] { "Hadoop RunJar util",
|
|
||||||
"JAR ./build/hadoop-examples.jar pi 10 10;" });
|
|
||||||
|
|
||||||
load.put("CLEAR", new String[] {"Clear the screen", "CLEAR;"} );
|
|
||||||
|
|
||||||
load.put("DESCRIBE", new String[] { "Print information about tables",
|
|
||||||
"[DESCRIBE|DESC] table_name;" });
|
|
||||||
|
|
||||||
load.put("CREATE", new String[] {
|
|
||||||
"Create tables",
|
|
||||||
"CREATE TABLE table_name"
|
|
||||||
+ "\n\t(column_family_spec [, column_family_spec] ...);"
|
|
||||||
+ "\n\n"
|
|
||||||
+ "column_family_spec:"
|
|
||||||
+ "\n\tcolumn_family_name"
|
|
||||||
+ "\n\t[MAX_VERSIONS=n]"
|
|
||||||
+ "\n\t[MAX_LENGTH=n]"
|
|
||||||
+ "\n\t[COMPRESSION=NONE|RECORD|BLOCK]"
|
|
||||||
+ "\n\t[IN_MEMORY]"
|
|
||||||
+ "\n\t[BLOOMFILTER=NONE|BLOOM|COUNTING|RETOUCHED VECTOR_SIZE=n NUM_HASH=n]"
|
|
||||||
});
|
|
||||||
|
|
||||||
load.put("DROP", new String[] {
|
|
||||||
"Drop tables",
|
|
||||||
"DROP TABLE table_name [, table_name] ...;" });
|
|
||||||
|
|
||||||
load.put("INSERT", new String[] {
|
|
||||||
"Insert values into tables",
|
|
||||||
"INSERT INTO table_name"
|
|
||||||
+ "\n\t(column_name, ...) VALUES ('value', ...)"
|
|
||||||
+ "\n\tWHERE row='row_key';"
|
|
||||||
+ "\n\n" + columnName
|
|
||||||
});
|
|
||||||
|
|
||||||
load.put("DELETE", new String[] {
|
|
||||||
"Delete a subset of the data in a table",
|
|
||||||
"DELETE " + columnList
|
|
||||||
+ "\n\tFROM table_name"
|
|
||||||
+ "\n\tWHERE row='row-key';"
|
|
||||||
+ "\n\n"
|
|
||||||
+ columnName
|
|
||||||
});
|
|
||||||
|
|
||||||
load.put("SELECT",
|
|
||||||
new String[] {
|
|
||||||
"Select values from tables",
|
|
||||||
"SELECT " + columnList + " FROM table_name"
|
|
||||||
+ "\n\t[WHERE row='row_key' | STARTING FROM 'row-key']"
|
|
||||||
+ "\n\t[NUM_VERSIONS = version_count]"
|
|
||||||
+ "\n\t[TIMESTAMP 'timestamp']"
|
|
||||||
+ "\n\t[LIMIT = row_count]"
|
|
||||||
+ "\n\t[INTO FILE 'file_name'];"
|
|
||||||
});
|
|
||||||
|
|
||||||
load.put("ALTER",
|
|
||||||
new String[] {
|
|
||||||
"Alter the structure of a table",
|
|
||||||
"ALTER TABLE table_name"
|
|
||||||
+ "\n\t ADD column_spec"
|
|
||||||
+ "\n\t| ADD (column_spec, column_spec, ...)"
|
|
||||||
+ "\n\t| DROP column_family_name"
|
|
||||||
+ "\n\t| CHANGE column_spec;"
|
|
||||||
});
|
|
||||||
|
|
||||||
load.put("EXIT", new String[] { "Exit shell", "EXIT;" });
|
|
||||||
|
|
||||||
return load;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -1,82 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright 2007 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.shell;
|
|
||||||
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Prints a usage message for the program to the given stream.
|
|
||||||
*/
|
|
||||||
public class HelpManager {
|
|
||||||
/** application name */
|
|
||||||
public static final String APP_NAME = "Hbase Shell";
|
|
||||||
|
|
||||||
/** version of the code */
|
|
||||||
public static final String APP_VERSION = "0.0.2";
|
|
||||||
|
|
||||||
/** help contents map */
|
|
||||||
public static final Map<String, String[]> help =
|
|
||||||
new HashMap<String, String[]>();
|
|
||||||
|
|
||||||
public HelpManager() {
|
|
||||||
help.putAll(HelpContents.Load());
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Print out the program version. */
|
|
||||||
public void printVersion() {
|
|
||||||
System.out.println(APP_NAME + ", " + APP_VERSION + " version.\n"
|
|
||||||
+ "Copyright (c) 2007 by udanax, "
|
|
||||||
+ "licensed to Apache Software Foundation.\n"
|
|
||||||
+ "Type 'help;' for usage.\n");
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void printHelp(String cmd) {
|
|
||||||
if (cmd.equals("")) {
|
|
||||||
System.out.println("Type 'help <command>;' to see command-specific "
|
|
||||||
+ "usage.\n");
|
|
||||||
for (Map.Entry<String, String[]> helpMap : help.entrySet()) {
|
|
||||||
wrapping(helpMap.getKey(), helpMap.getValue(), false);
|
|
||||||
}
|
|
||||||
System.out.println();
|
|
||||||
} else {
|
|
||||||
if (help.containsKey(cmd.toUpperCase())) {
|
|
||||||
String[] msg = help.get(cmd.toUpperCase());
|
|
||||||
wrapping(cmd.toUpperCase(), msg, true);
|
|
||||||
} else {
|
|
||||||
System.out.println("Unknown Command : Type 'help;' for usage.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void wrapping(String cmd, String[] cmdType, boolean example) {
|
|
||||||
System.out.printf("%-10s", cmd);
|
|
||||||
if (cmdType[0].length() > 55) {
|
|
||||||
System.out.println(cmdType[0].substring(0, 55));
|
|
||||||
System.out.printf("%13s", "");
|
|
||||||
System.out.println(cmdType[0].substring(55, cmdType[1].length()));
|
|
||||||
} else {
|
|
||||||
System.out.println(cmdType[0]);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (example)
|
|
||||||
System.out.println("\nSyntax:\n" + cmdType[1] + "\n");
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -30,12 +31,15 @@ import org.apache.hadoop.io.Text;
|
||||||
* Inserts values into tables.
|
* Inserts values into tables.
|
||||||
*/
|
*/
|
||||||
public class InsertCommand extends BasicCommand {
|
public class InsertCommand extends BasicCommand {
|
||||||
|
|
||||||
private Text tableName;
|
private Text tableName;
|
||||||
private List<String> columnfamilies;
|
private List<String> columnfamilies;
|
||||||
private List<String> values;
|
private List<String> values;
|
||||||
private String rowKey;
|
private String rowKey;
|
||||||
|
|
||||||
|
public InsertCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
public ReturnMsg execute(Configuration conf) {
|
||||||
if (this.tableName == null || this.values == null || this.rowKey == null)
|
if (this.tableName == null || this.values == null || this.rowKey == null)
|
||||||
return new ReturnMsg(0, "Syntax error : Please check 'Insert' syntax.");
|
return new ReturnMsg(0, "Syntax error : Please check 'Insert' syntax.");
|
||||||
|
@ -92,4 +96,9 @@ public class InsertCommand extends BasicCommand {
|
||||||
public byte[] getValue(int i) {
|
public byte[] getValue(int i) {
|
||||||
return this.values.get(i).getBytes();
|
return this.values.get(i).getBytes();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.INSERT;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
import java.lang.reflect.Array;
|
import java.lang.reflect.Array;
|
||||||
import java.lang.reflect.InvocationTargetException;
|
import java.lang.reflect.InvocationTargetException;
|
||||||
import java.lang.reflect.Method;
|
import java.lang.reflect.Method;
|
||||||
|
@ -40,9 +41,12 @@ import org.apache.hadoop.util.RunJar;
|
||||||
* Run hadoop jar commands.
|
* Run hadoop jar commands.
|
||||||
*/
|
*/
|
||||||
public class JarCommand extends BasicCommand {
|
public class JarCommand extends BasicCommand {
|
||||||
|
|
||||||
private List<String> query;
|
private List<String> query;
|
||||||
|
|
||||||
|
public JarCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
public ReturnMsg execute(@SuppressWarnings("unused") Configuration conf) {
|
||||||
|
|
||||||
|
@ -98,6 +102,7 @@ public class JarCommand extends BasicCommand {
|
||||||
try {
|
try {
|
||||||
FileUtil.fullyDelete(workDir);
|
FileUtil.fullyDelete(workDir);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -122,7 +127,7 @@ public class JarCommand extends BasicCommand {
|
||||||
Method main = mainClass.getMethod("main", new Class[] {
|
Method main = mainClass.getMethod("main", new Class[] {
|
||||||
Array.newInstance(String.class, 0).getClass()
|
Array.newInstance(String.class, 0).getClass()
|
||||||
});
|
});
|
||||||
String[] newArgs = (String[])Arrays.asList(args)
|
String[] newArgs = Arrays.asList(args)
|
||||||
.subList(firstArg, args.length).toArray(new String[0]);
|
.subList(firstArg, args.length).toArray(new String[0]);
|
||||||
try {
|
try {
|
||||||
main.invoke(null, new Object[] { newArgs });
|
main.invoke(null, new Object[] { newArgs });
|
||||||
|
@ -144,4 +149,9 @@ public class JarCommand extends BasicCommand {
|
||||||
private String[] getQuery() {
|
private String[] getQuery() {
|
||||||
return query.toArray(new String[] {});
|
return query.toArray(new String[] {});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CommandType getCommandType() {
|
||||||
|
return CommandType.SHELL;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,4 +44,9 @@ public class ReturnMsg {
|
||||||
public int getType() {
|
public int getType() {
|
||||||
return this.type;
|
return this.type;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return this.msg;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -32,7 +33,6 @@ import org.apache.hadoop.io.Text;
|
||||||
* Command. Provides utility methods for alteration operations.
|
* Command. Provides utility methods for alteration operations.
|
||||||
*/
|
*/
|
||||||
public abstract class SchemaModificationCommand extends BasicCommand {
|
public abstract class SchemaModificationCommand extends BasicCommand {
|
||||||
|
|
||||||
private int maxVersions;
|
private int maxVersions;
|
||||||
private int maxLength;
|
private int maxLength;
|
||||||
private HColumnDescriptor.CompressionType compression;
|
private HColumnDescriptor.CompressionType compression;
|
||||||
|
@ -43,6 +43,10 @@ public abstract class SchemaModificationCommand extends BasicCommand {
|
||||||
private int numHash;
|
private int numHash;
|
||||||
private int numEntries;
|
private int numEntries;
|
||||||
|
|
||||||
|
public SchemaModificationCommand(Writer o) {
|
||||||
|
super(o);
|
||||||
|
}
|
||||||
|
|
||||||
private void initOptions() {
|
private void initOptions() {
|
||||||
maxVersions = HColumnDescriptor.DEFAULT_N_VERSIONS;
|
maxVersions = HColumnDescriptor.DEFAULT_N_VERSIONS;
|
||||||
maxLength = HColumnDescriptor.DEFAULT_MAX_VALUE_LENGTH;
|
maxLength = HColumnDescriptor.DEFAULT_MAX_VALUE_LENGTH;
|
||||||
|
|
|
@ -20,6 +20,8 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
|
import java.io.Writer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -34,7 +36,6 @@ import org.apache.hadoop.hbase.HScannerInterface;
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
import org.apache.hadoop.hbase.HTable;
|
import org.apache.hadoop.hbase.HTable;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.shell.generated.ParseException;
|
|
||||||
import org.apache.hadoop.hbase.shell.generated.Parser;
|
import org.apache.hadoop.hbase.shell.generated.Parser;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
|
@ -60,8 +61,22 @@ public class SelectCommand extends BasicCommand {
|
||||||
new String [] {"Column", "Cell"};
|
new String [] {"Column", "Cell"};
|
||||||
private static final String [] HEADER =
|
private static final String [] HEADER =
|
||||||
new String [] {"Row", "Column", "Cell"};
|
new String [] {"Row", "Column", "Cell"};
|
||||||
|
private static final String STAR = "*";
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
private final TableFormatter formatter;
|
||||||
|
|
||||||
|
// Not instantiable
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
private SelectCommand() {
|
||||||
|
this(null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public SelectCommand(final Writer o, final TableFormatter f) {
|
||||||
|
super(o);
|
||||||
|
this.formatter = f;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ReturnMsg execute(final Configuration conf) {
|
||||||
if (this.tableName.equals("") || this.rowKey == null ||
|
if (this.tableName.equals("") || this.rowKey == null ||
|
||||||
this.columns.size() == 0) {
|
this.columns.size() == 0) {
|
||||||
return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
|
return new ReturnMsg(0, "Syntax error : Please check 'Select' syntax.");
|
||||||
|
@ -84,7 +99,6 @@ public class SelectCommand extends BasicCommand {
|
||||||
|
|
||||||
private int compoundWherePrint(HTable table, HBaseAdmin admin) {
|
private int compoundWherePrint(HTable table, HBaseAdmin admin) {
|
||||||
int count = 0;
|
int count = 0;
|
||||||
TableFormatter formatter = TableFormatterFactory.get();
|
|
||||||
try {
|
try {
|
||||||
if (this.version != 0) {
|
if (this.version != 0) {
|
||||||
// A number of versions has been specified.
|
// A number of versions has been specified.
|
||||||
|
@ -113,7 +127,8 @@ public class SelectCommand extends BasicCommand {
|
||||||
formatter.header(isMultiple()? HEADER_COLUMN_CELL: null);
|
formatter.header(isMultiple()? HEADER_COLUMN_CELL: null);
|
||||||
for (Map.Entry<Text, byte[]> e: table.getRow(this.rowKey).entrySet()) {
|
for (Map.Entry<Text, byte[]> e: table.getRow(this.rowKey).entrySet()) {
|
||||||
Text key = e.getKey();
|
Text key = e.getKey();
|
||||||
if (!this.columns.contains(key.toString())) {
|
String keyStr = key.toString();
|
||||||
|
if (!this.columns.contains(STAR) && !this.columns.contains(keyStr)) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
String cellData = toString(key, e.getValue());
|
String cellData = toString(key, e.getValue());
|
||||||
|
@ -173,7 +188,8 @@ public class SelectCommand extends BasicCommand {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private int scanPrint(HTable table, HBaseAdmin admin) {
|
private int scanPrint(HTable table,
|
||||||
|
HBaseAdmin admin) {
|
||||||
int count = 0;
|
int count = 0;
|
||||||
HScannerInterface scan = null;
|
HScannerInterface scan = null;
|
||||||
try {
|
try {
|
||||||
|
@ -186,7 +202,6 @@ public class SelectCommand extends BasicCommand {
|
||||||
}
|
}
|
||||||
HStoreKey key = new HStoreKey();
|
HStoreKey key = new HStoreKey();
|
||||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||||
TableFormatter formatter = TableFormatterFactory.get();
|
|
||||||
// If only one column in query, then don't print out the column.
|
// If only one column in query, then don't print out the column.
|
||||||
formatter.header((parsedColumns.isMultiple())? HEADER: HEADER_ROW_CELL);
|
formatter.header((parsedColumns.isMultiple())? HEADER: HEADER_ROW_CELL);
|
||||||
while (scan.next(key, results) && checkLimit(count)) {
|
while (scan.next(key, results) && checkLimit(count)) {
|
||||||
|
@ -265,7 +280,7 @@ public class SelectCommand extends BasicCommand {
|
||||||
* @return True if query contains multiple columns.
|
* @return True if query contains multiple columns.
|
||||||
*/
|
*/
|
||||||
private boolean isMultiple() {
|
private boolean isMultiple() {
|
||||||
return this.columns.size() > 1 || this.columns.contains("*");
|
return this.columns.size() > 1 || this.columns.contains(STAR);
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean checkLimit(int count) {
|
private boolean checkLimit(int count) {
|
||||||
|
@ -307,10 +322,17 @@ public class SelectCommand extends BasicCommand {
|
||||||
this.version = version;
|
this.version = version;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws ParseException {
|
public static void main(String[] args) throws Exception {
|
||||||
|
Writer out = new OutputStreamWriter(System.out, "UTF-8");
|
||||||
|
HBaseConfiguration c = new HBaseConfiguration();
|
||||||
// For debugging
|
// For debugging
|
||||||
Parser parser = new Parser("select * from -ROOT-;");
|
TableFormatterFactory tff =
|
||||||
|
new TableFormatterFactory(out, c);
|
||||||
|
Parser parser = new Parser("select * from 'x' where row='x';", out, tff.get());
|
||||||
Command cmd = parser.terminatedCommand();
|
Command cmd = parser.terminatedCommand();
|
||||||
ReturnMsg rm = cmd.execute(new HBaseConfiguration());
|
|
||||||
|
ReturnMsg rm = cmd.execute(c);
|
||||||
|
out.write(rm == null? "": rm.toString());
|
||||||
|
out.flush();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HBaseAdmin;
|
import org.apache.hadoop.hbase.HBaseAdmin;
|
||||||
|
@ -29,10 +30,28 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
* Shows all available tables.
|
* Shows all available tables.
|
||||||
*/
|
*/
|
||||||
public class ShowCommand extends BasicCommand {
|
public class ShowCommand extends BasicCommand {
|
||||||
private static final String [] HEADER = new String [] {"Table Name"};
|
private static final String [] HEADER = new String [] {"Name", "Descriptor"};
|
||||||
private String command;
|
private String command;
|
||||||
|
private final TableFormatter formatter;
|
||||||
|
|
||||||
public ReturnMsg execute(Configuration conf) {
|
// Not instantiable
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
private ShowCommand() {
|
||||||
|
this(null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ShowCommand(final Writer o, final TableFormatter f) {
|
||||||
|
this(o, f, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ShowCommand(final Writer o, final TableFormatter f,
|
||||||
|
final String argument) {
|
||||||
|
super(o);
|
||||||
|
this.formatter = f;
|
||||||
|
this.command = argument;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ReturnMsg execute(final Configuration conf) {
|
||||||
if (this.command == null) {
|
if (this.command == null) {
|
||||||
return new ReturnMsg(0, "Syntax error : Please check 'Show' syntax");
|
return new ReturnMsg(0, "Syntax error : Please check 'Show' syntax");
|
||||||
}
|
}
|
||||||
|
@ -43,13 +62,12 @@ public class ShowCommand extends BasicCommand {
|
||||||
HTableDescriptor[] tables = admin.listTables();
|
HTableDescriptor[] tables = admin.listTables();
|
||||||
tableLength = tables.length;
|
tableLength = tables.length;
|
||||||
if (tableLength == 0) {
|
if (tableLength == 0) {
|
||||||
return new ReturnMsg(0, "Table not found");
|
return new ReturnMsg(0, "No tables found");
|
||||||
}
|
}
|
||||||
TableFormatter formatter = TableFormatterFactory.get();
|
|
||||||
formatter.header(HEADER);
|
formatter.header(HEADER);
|
||||||
for (int i = 0; i < tableLength; i++) {
|
for (int i = 0; i < tableLength; i++) {
|
||||||
String tableName = tables[i].getName().toString();
|
String tableName = tables[i].getName().toString();
|
||||||
formatter.row(new String [] {tableName});
|
formatter.row(new String [] {tableName, tables[i].toString()});
|
||||||
}
|
}
|
||||||
formatter.footer();
|
formatter.footer();
|
||||||
return new ReturnMsg(1, tableLength + " table(s) in set");
|
return new ReturnMsg(1, tableLength + " table(s) in set");
|
||||||
|
|
|
@ -19,24 +19,41 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.shell.formatter.AsciiTableFormatter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Interface implemented by table formatters outputting select results.
|
* Interface implemented by table formatters outputting select results.
|
||||||
|
* Implementations must have a constructor that takes a Writer.
|
||||||
|
* @see AsciiTableFormatter
|
||||||
*/
|
*/
|
||||||
public interface TableFormatter {
|
public interface TableFormatter {
|
||||||
/**
|
/**
|
||||||
* Output header.
|
* Output header.
|
||||||
* @param titles Titles to emit.
|
* @param titles Titles to emit.
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void header(final String [] titles);
|
public void header(final String [] titles) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Output footer.
|
* Output footer.
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void footer();
|
public void footer() throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Output a row.
|
* Output a row.
|
||||||
* @param cells
|
* @param cells
|
||||||
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void row(final String [] cells);
|
public void row(final String [] cells) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Output stream being used (This is in interface to enforce fact
|
||||||
|
* that formatters use Writers -- that they operate on character streams
|
||||||
|
* rather than on byte streams).
|
||||||
|
*/
|
||||||
|
public Writer getOut();
|
||||||
}
|
}
|
|
@ -19,153 +19,65 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.shell;
|
package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.PrintStream;
|
import java.io.Writer;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.lang.reflect.Constructor;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.shell.formatter.AsciiTableFormatter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Table formatter.
|
* Table formatter.
|
||||||
* TODO: Make a factory that chooses the formatter to use based off
|
* Specify formatter by setting "hbaseshell.formatter" property in
|
||||||
* configuration. Read a property from hbase-site or from System properties.
|
* <code>hbase-site.xml</code> or by setting system property
|
||||||
* For now, default is the internal AsciiTableFormatter.
|
* <code>hbaseshell.formatter</code>. System property setting prevails over all
|
||||||
* TODO: Mysql has --skip-column-names and --silent which inserts a tab as
|
* other configurations. Outputs UTF-8 encoded Strings even if
|
||||||
|
* original data is binary. On static initialization, changes System.out to be
|
||||||
|
* a UTF-8 output stream.
|
||||||
|
* .
|
||||||
|
* <p>TODO: Mysql has --skip-column-names and --silent which inserts a tab as
|
||||||
* separator. Also has --html and --xml.
|
* separator. Also has --html and --xml.
|
||||||
|
* <p>To use the html formatter, currently set HBASE_OPTS as in:
|
||||||
|
* <code>$ HBASE_OPTS="-Dhbaseshell.formatter=org.apache.hadoop.hbase.shell.formatter.HtmlTableFormatter" ./bin/hbase shell</code>
|
||||||
|
* </p>
|
||||||
*/
|
*/
|
||||||
public class TableFormatterFactory {
|
public class TableFormatterFactory {
|
||||||
private static final TableFormatterFactory factory =
|
private static final Log LOG =
|
||||||
new TableFormatterFactory();
|
LogFactory.getLog(TableFormatterFactory.class.getName());
|
||||||
|
private static final String FORMATTER_KEY = "hbaseshell.formatter";
|
||||||
private final TableFormatter formatter;
|
private final TableFormatter formatter;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Not instantiable
|
||||||
|
*/
|
||||||
|
@SuppressWarnings({ "unchecked", "unused" })
|
||||||
private TableFormatterFactory() {
|
private TableFormatterFactory() {
|
||||||
this.formatter = new AsciiTableFormatter();
|
this(null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@SuppressWarnings("unchecked")
|
||||||
* @return Configured table formatter.
|
public TableFormatterFactory(final Writer out, final Configuration c) {
|
||||||
*/
|
String className = System.getProperty(FORMATTER_KEY);
|
||||||
public static TableFormatter get() {
|
if (className == null) {
|
||||||
return factory.formatter;
|
className = c.get(FORMATTER_KEY, AsciiTableFormatter.class.getName());
|
||||||
}
|
}
|
||||||
|
LOG.debug("Table formatter class: " + className);
|
||||||
/*
|
|
||||||
* Formmatter that outputs data in UTF-8 inside an ASCII table on STDOUT.
|
|
||||||
* If only a single cell result, then no formatting is done. Presumption is
|
|
||||||
* that client manages serial access outputting tables.
|
|
||||||
*/
|
|
||||||
private class AsciiTableFormatter implements TableFormatter {
|
|
||||||
private PrintStream out;
|
|
||||||
private static final String COLUMN_DELIMITER = "| ";
|
|
||||||
private static final String COLUMN_CLOSER = "|";
|
|
||||||
private static final int DEFAULT_COLUMN_WIDTH = 26;
|
|
||||||
// Width is a line of content + delimiter
|
|
||||||
private int columnWidth = DEFAULT_COLUMN_WIDTH;
|
|
||||||
// Amount of width to use for a line of content.
|
|
||||||
private int columnContentWidth =
|
|
||||||
DEFAULT_COLUMN_WIDTH - COLUMN_DELIMITER.length();
|
|
||||||
// COLUMN_LINE is put at head and foot of a column and per column, is drawn
|
|
||||||
// as row delimiter
|
|
||||||
private String columnHorizLine;
|
|
||||||
private final String COLUMN_HORIZ_LINE_CLOSER = "+";
|
|
||||||
// Used padding content to fill column
|
|
||||||
private final String PADDING_CHAR = " ";
|
|
||||||
// True if we are to output no formatting.
|
|
||||||
private boolean noFormatting = false;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Constructor.
|
|
||||||
*/
|
|
||||||
protected AsciiTableFormatter() {
|
|
||||||
try {
|
try {
|
||||||
this.out = new PrintStream(System.out, true, "UTF-8");
|
Class<TableFormatter> clazz =
|
||||||
} catch (UnsupportedEncodingException e) {
|
(Class<TableFormatter>) Class.forName(className);
|
||||||
throw new RuntimeException("Failed setting output to UTF-8", e);
|
Constructor<?> constructor = clazz.getConstructor(Writer.class);
|
||||||
|
this.formatter = (TableFormatter)constructor.newInstance(out);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException("Failed instantiation of " + className, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param titles List of titles. Pass null if no formatting (i.e.
|
* @return The table formatter instance
|
||||||
* no header, no footer, etc.
|
|
||||||
*/
|
*/
|
||||||
public void header(String[] titles) {
|
@SuppressWarnings("unchecked")
|
||||||
if (titles == null) {
|
public TableFormatter get() {
|
||||||
// print nothing.
|
return this.formatter;
|
||||||
this.noFormatting = true;
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
// Calculate width of columns.
|
|
||||||
this.columnWidth = titles.length == 1? 3 * DEFAULT_COLUMN_WIDTH:
|
|
||||||
titles.length == 2? 39: DEFAULT_COLUMN_WIDTH;
|
|
||||||
this.columnContentWidth = this.columnWidth - COLUMN_DELIMITER.length();
|
|
||||||
// Create the horizontal line to draw across the top of each column.
|
|
||||||
this.columnHorizLine = calculateColumnHorizLine(this.columnWidth);
|
|
||||||
// Print out a column topper per column.
|
|
||||||
printRowDelimiter(titles.length);
|
|
||||||
row(titles);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void row(String [] cells) {
|
|
||||||
if (this.noFormatting) {
|
|
||||||
this.out.print(cells[0]);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
// Ok. Output cells a line at a time w/ delimiters between cells.
|
|
||||||
int [] indexes = new int[cells.length];
|
|
||||||
for (int i = 0; i < indexes.length; i++) {
|
|
||||||
indexes[i] = 0;
|
|
||||||
}
|
|
||||||
int allFinished = 0;
|
|
||||||
while (allFinished < indexes.length) {
|
|
||||||
StringBuffer sb = new StringBuffer();
|
|
||||||
for (int i = 0; i < cells.length; i++) {
|
|
||||||
sb.append(COLUMN_DELIMITER);
|
|
||||||
int offset = indexes[i];
|
|
||||||
if (offset + this.columnContentWidth >= cells[i].length()) {
|
|
||||||
String substr = cells[i].substring(offset);
|
|
||||||
if (substr.length() > 0) {
|
|
||||||
// This column is finished
|
|
||||||
allFinished++;
|
|
||||||
sb.append(substr);
|
|
||||||
}
|
|
||||||
for (int j = 0; j < this.columnContentWidth - substr.length(); j++) {
|
|
||||||
sb.append(PADDING_CHAR);
|
|
||||||
}
|
|
||||||
indexes[i] = cells[i].length();
|
|
||||||
} else {
|
|
||||||
String substr = cells[i].substring(indexes[i],
|
|
||||||
indexes[i] + this.columnContentWidth);
|
|
||||||
indexes[i] += this.columnContentWidth;
|
|
||||||
sb.append(substr);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
sb.append(COLUMN_CLOSER);
|
|
||||||
this.out.println(sb.toString());
|
|
||||||
}
|
|
||||||
printRowDelimiter(cells.length);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void footer() {
|
|
||||||
if (this.noFormatting) {
|
|
||||||
// If no formatting, output a newline to delimit cell and the
|
|
||||||
// result summary output at end of every command.
|
|
||||||
this.out.println();
|
|
||||||
}
|
|
||||||
// We're done. Clear flag.
|
|
||||||
this.noFormatting = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void printRowDelimiter(final int columnCount) {
|
|
||||||
for (int i = 0; i < columnCount; i++) {
|
|
||||||
this.out.print(this.columnHorizLine);
|
|
||||||
}
|
|
||||||
this.out.println(COLUMN_HORIZ_LINE_CLOSER);
|
|
||||||
}
|
|
||||||
|
|
||||||
private String calculateColumnHorizLine(final int width) {
|
|
||||||
StringBuffer sb = new StringBuffer();
|
|
||||||
sb.append("+");
|
|
||||||
for (int i = 1; i < width; i++) {
|
|
||||||
sb.append("-");
|
|
||||||
}
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -27,6 +27,7 @@ import java.util.Map;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.io.StringReader;
|
import java.io.StringReader;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shell.*;
|
import org.apache.hadoop.hbase.shell.*;
|
||||||
|
|
||||||
|
@ -35,10 +36,14 @@ import org.apache.hadoop.hbase.shell.*;
|
||||||
*/
|
*/
|
||||||
public class Parser implements ParserConstants {
|
public class Parser implements ParserConstants {
|
||||||
private String QueryString;
|
private String QueryString;
|
||||||
|
private TableFormatter formatter;
|
||||||
|
private Writer out;
|
||||||
|
|
||||||
public Parser(String query) {
|
public Parser(final String query, final Writer o, final TableFormatter f) {
|
||||||
this((Reader)(new StringReader(query)));
|
this((Reader)(new StringReader(query)));
|
||||||
this.QueryString = query;
|
this.QueryString = query;
|
||||||
|
this.formatter = f;
|
||||||
|
this.out = o;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getQueryStr() {
|
public String getQueryStr() {
|
||||||
|
@ -164,7 +169,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public ExitCommand exitCommand() throws ParseException {
|
final public ExitCommand exitCommand() throws ParseException {
|
||||||
ExitCommand exit = new ExitCommand();
|
ExitCommand exit = new ExitCommand(this.out);
|
||||||
jj_consume_token(EXIT);
|
jj_consume_token(EXIT);
|
||||||
{if (true) return exit;}
|
{if (true) return exit;}
|
||||||
throw new Error("Missing return statement in function");
|
throw new Error("Missing return statement in function");
|
||||||
|
@ -172,7 +177,7 @@ public class Parser implements ParserConstants {
|
||||||
|
|
||||||
final public FsCommand fsCommand() throws ParseException {
|
final public FsCommand fsCommand() throws ParseException {
|
||||||
Token t = null;
|
Token t = null;
|
||||||
FsCommand fs = new FsCommand();
|
FsCommand fs = new FsCommand(this.out);
|
||||||
List<String> query = new ArrayList<String>();
|
List<String> query = new ArrayList<String>();
|
||||||
jj_consume_token(FS);
|
jj_consume_token(FS);
|
||||||
label_1:
|
label_1:
|
||||||
|
@ -195,7 +200,7 @@ public class Parser implements ParserConstants {
|
||||||
|
|
||||||
final public JarCommand jarCommand() throws ParseException {
|
final public JarCommand jarCommand() throws ParseException {
|
||||||
Token t = null;
|
Token t = null;
|
||||||
JarCommand jar = new JarCommand();
|
JarCommand jar = new JarCommand(this.out);
|
||||||
List<String> query = new ArrayList<String>();
|
List<String> query = new ArrayList<String>();
|
||||||
jj_consume_token(JAR);
|
jj_consume_token(JAR);
|
||||||
label_2:
|
label_2:
|
||||||
|
@ -230,7 +235,7 @@ public class Parser implements ParserConstants {
|
||||||
|
|
||||||
final public HelpCommand helpCommand() throws ParseException {
|
final public HelpCommand helpCommand() throws ParseException {
|
||||||
Token t = null;
|
Token t = null;
|
||||||
HelpCommand help = new HelpCommand();
|
HelpCommand help = new HelpCommand(this.out, this.formatter);
|
||||||
String argument = "";
|
String argument = "";
|
||||||
jj_consume_token(HELP);
|
jj_consume_token(HELP);
|
||||||
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
||||||
|
@ -304,7 +309,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public ShowCommand showCommand() throws ParseException {
|
final public ShowCommand showCommand() throws ParseException {
|
||||||
ShowCommand show = new ShowCommand();
|
ShowCommand show = new ShowCommand(this.out, this.formatter);
|
||||||
String argument = null;
|
String argument = null;
|
||||||
jj_consume_token(SHOW);
|
jj_consume_token(SHOW);
|
||||||
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
||||||
|
@ -323,7 +328,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public DescCommand descCommand() throws ParseException {
|
final public DescCommand descCommand() throws ParseException {
|
||||||
DescCommand desc = new DescCommand();
|
DescCommand desc = new DescCommand(this.out, this.formatter);
|
||||||
String argument = null;
|
String argument = null;
|
||||||
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
switch ((jj_ntk==-1)?jj_ntk():jj_ntk) {
|
||||||
case DESCRIBE:
|
case DESCRIBE:
|
||||||
|
@ -450,7 +455,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public CreateCommand createCommand() throws ParseException {
|
final public CreateCommand createCommand() throws ParseException {
|
||||||
CreateCommand createCommand = new CreateCommand();
|
CreateCommand createCommand = new CreateCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
Map<String, Object> columnSpec = null;
|
Map<String, Object> columnSpec = null;
|
||||||
String column = null;
|
String column = null;
|
||||||
|
@ -483,7 +488,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public AlterCommand alterCommand() throws ParseException {
|
final public AlterCommand alterCommand() throws ParseException {
|
||||||
AlterCommand alterCommand = new AlterCommand();
|
AlterCommand alterCommand = new AlterCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
String column = null;
|
String column = null;
|
||||||
Map<String, Object> columnSpec = null;
|
Map<String, Object> columnSpec = null;
|
||||||
|
@ -547,7 +552,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public DropCommand dropCommand() throws ParseException {
|
final public DropCommand dropCommand() throws ParseException {
|
||||||
DropCommand drop = new DropCommand();
|
DropCommand drop = new DropCommand(this.out);
|
||||||
List<String> tableList = null;
|
List<String> tableList = null;
|
||||||
jj_consume_token(DROP);
|
jj_consume_token(DROP);
|
||||||
jj_consume_token(TABLE);
|
jj_consume_token(TABLE);
|
||||||
|
@ -558,7 +563,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public InsertCommand insertCommand() throws ParseException {
|
final public InsertCommand insertCommand() throws ParseException {
|
||||||
InsertCommand in = new InsertCommand();
|
InsertCommand in = new InsertCommand(this.out);
|
||||||
List<String> columnfamilies = null;
|
List<String> columnfamilies = null;
|
||||||
List<String> values = null;
|
List<String> values = null;
|
||||||
String table = null;
|
String table = null;
|
||||||
|
@ -593,7 +598,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public DeleteCommand deleteCommand() throws ParseException {
|
final public DeleteCommand deleteCommand() throws ParseException {
|
||||||
DeleteCommand deleteCommand = new DeleteCommand();
|
DeleteCommand deleteCommand = new DeleteCommand(this.out);
|
||||||
List<String> columnList = null;
|
List<String> columnList = null;
|
||||||
Token t = null;
|
Token t = null;
|
||||||
String table = null;
|
String table = null;
|
||||||
|
@ -624,7 +629,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public SelectCommand selectCommand() throws ParseException {
|
final public SelectCommand selectCommand() throws ParseException {
|
||||||
SelectCommand select = new SelectCommand();
|
SelectCommand select = new SelectCommand(this.out, this.formatter);
|
||||||
List<String> columns = null;
|
List<String> columns = null;
|
||||||
String rowKey = "";
|
String rowKey = "";
|
||||||
String timestamp = null;
|
String timestamp = null;
|
||||||
|
@ -704,7 +709,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public EnableCommand enableCommand() throws ParseException {
|
final public EnableCommand enableCommand() throws ParseException {
|
||||||
EnableCommand enableCommand = new EnableCommand();
|
EnableCommand enableCommand = new EnableCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
jj_consume_token(ENABLE);
|
jj_consume_token(ENABLE);
|
||||||
table = Identifier();
|
table = Identifier();
|
||||||
|
@ -714,7 +719,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public DisableCommand disableCommand() throws ParseException {
|
final public DisableCommand disableCommand() throws ParseException {
|
||||||
DisableCommand disableCommand = new DisableCommand();
|
DisableCommand disableCommand = new DisableCommand(this.out);
|
||||||
String table = null;
|
String table = null;
|
||||||
jj_consume_token(DISABLE);
|
jj_consume_token(DISABLE);
|
||||||
table = Identifier();
|
table = Identifier();
|
||||||
|
@ -724,7 +729,7 @@ public class Parser implements ParserConstants {
|
||||||
}
|
}
|
||||||
|
|
||||||
final public ClearCommand clearCommand() throws ParseException {
|
final public ClearCommand clearCommand() throws ParseException {
|
||||||
ClearCommand clear = new ClearCommand();
|
ClearCommand clear = new ClearCommand(this.out);
|
||||||
jj_consume_token(CLEAR);
|
jj_consume_token(CLEAR);
|
||||||
{if (true) return clear;}
|
{if (true) return clear;}
|
||||||
throw new Error("Missing return statement in function");
|
throw new Error("Missing return statement in function");
|
||||||
|
@ -970,6 +975,16 @@ public class Parser implements ParserConstants {
|
||||||
finally { jj_save(0, xla); }
|
finally { jj_save(0, xla); }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final private boolean jj_3R_12() {
|
||||||
|
Token xsp;
|
||||||
|
xsp = jj_scanpos;
|
||||||
|
if (jj_scan_token(60)) {
|
||||||
|
jj_scanpos = xsp;
|
||||||
|
if (jj_scan_token(61)) return true;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
final private boolean jj_3R_11() {
|
final private boolean jj_3R_11() {
|
||||||
if (jj_scan_token(ID)) return true;
|
if (jj_scan_token(ID)) return true;
|
||||||
return false;
|
return false;
|
||||||
|
@ -991,16 +1006,6 @@ public class Parser implements ParserConstants {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
final private boolean jj_3R_12() {
|
|
||||||
Token xsp;
|
|
||||||
xsp = jj_scanpos;
|
|
||||||
if (jj_scan_token(60)) {
|
|
||||||
jj_scanpos = xsp;
|
|
||||||
if (jj_scan_token(61)) return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ParserTokenManager token_source;
|
public ParserTokenManager token_source;
|
||||||
SimpleCharStream jj_input_stream;
|
SimpleCharStream jj_input_stream;
|
||||||
public Token token, jj_nt;
|
public Token token, jj_nt;
|
||||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.io.StringReader;
|
import java.io.StringReader;
|
||||||
import java.io.Reader;
|
import java.io.Reader;
|
||||||
|
import java.io.Writer;
|
||||||
import org.apache.hadoop.hbase.shell.*;
|
import org.apache.hadoop.hbase.shell.*;
|
||||||
|
|
||||||
public class ParserTokenManager implements ParserConstants
|
public class ParserTokenManager implements ParserConstants
|
||||||
|
|
|
@ -67,6 +67,20 @@
|
||||||
sooner. The default is 30 seconds.
|
sooner. The default is 30 seconds.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.master.info.port</name>
|
||||||
|
<value>-1</value>
|
||||||
|
<description>The port for the hbase master web UI
|
||||||
|
Set to -1 if you do not want the info server to run.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hbase.regionserver.info.port</name>
|
||||||
|
<value>-1</value>
|
||||||
|
<description>The port for the hbase regionserver web UI
|
||||||
|
Set to -1 if you do not want the info server to run.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.master.lease.thread.wakefrequency</name>
|
<name>hbase.master.lease.thread.wakefrequency</name>
|
||||||
<value>3000</value>
|
<value>3000</value>
|
||||||
|
|
|
@ -71,7 +71,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void setUp() throws Exception {
|
protected void setUp() throws Exception {
|
||||||
super.setUp();
|
super.setUp();
|
||||||
this.cluster =
|
this.cluster =
|
||||||
new MiniHBaseCluster(this.conf, this.regionServers, this.miniHdfs);
|
new MiniHBaseCluster(this.conf, this.regionServers, this.miniHdfs);
|
||||||
|
@ -79,7 +79,7 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void tearDown() throws Exception {
|
protected void tearDown() throws Exception {
|
||||||
super.tearDown();
|
super.tearDown();
|
||||||
if (this.cluster != null) {
|
if (this.cluster != null) {
|
||||||
this.cluster.shutdown();
|
this.cluster.shutdown();
|
||||||
|
|
|
@ -66,7 +66,7 @@ public abstract class HBaseTestCase extends TestCase {
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void setUp() throws Exception {
|
protected void setUp() throws Exception {
|
||||||
super.setUp();
|
super.setUp();
|
||||||
this.testDir = getUnitTestdir(getName());
|
this.testDir = getUnitTestdir(getName());
|
||||||
this.localFs = FileSystem.getLocal(this.conf);
|
this.localFs = FileSystem.getLocal(this.conf);
|
||||||
|
@ -77,7 +77,7 @@ public abstract class HBaseTestCase extends TestCase {
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public void tearDown() throws Exception {
|
protected void tearDown() throws Exception {
|
||||||
if (this.localFs != null && this.testDir != null &&
|
if (this.localFs != null && this.testDir != null &&
|
||||||
this.localFs.exists(testDir)) {
|
this.localFs.exists(testDir)) {
|
||||||
this.localFs.delete(testDir);
|
this.localFs.delete(testDir);
|
||||||
|
|
|
@ -84,7 +84,6 @@ public class MiniHBaseCluster implements HConstants {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Starts a MiniHBaseCluster on top of an existing HDFSCluster
|
* Starts a MiniHBaseCluster on top of an existing HDFSCluster
|
||||||
*
|
|
||||||
*<pre>
|
*<pre>
|
||||||
****************************************************************************
|
****************************************************************************
|
||||||
* * * * * * N O T E * * * * *
|
* * * * * * N O T E * * * * *
|
||||||
|
@ -181,7 +180,6 @@ public class MiniHBaseCluster implements HConstants {
|
||||||
this.regionServer = r;
|
this.regionServer = r;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
LOG.info("Starting " + getName());
|
LOG.info("Starting " + getName());
|
||||||
|
@ -461,4 +459,8 @@ public class MiniHBaseCluster implements HConstants {
|
||||||
r.flushcache(false);
|
r.flushcache(false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ArrayList<RegionServerThread> getRegionThreads() {
|
||||||
|
return this.regionThreads;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -34,7 +34,7 @@ public class TestToString extends TestCase {
|
||||||
final String hostport = "127.0.0.1:9999";
|
final String hostport = "127.0.0.1:9999";
|
||||||
HServerAddress address = new HServerAddress(hostport);
|
HServerAddress address = new HServerAddress(hostport);
|
||||||
assertEquals("HServerAddress toString", address.toString(), hostport);
|
assertEquals("HServerAddress toString", address.toString(), hostport);
|
||||||
HServerInfo info = new HServerInfo(address, -1);
|
HServerInfo info = new HServerInfo(address, -1, 60030);
|
||||||
assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
|
assertEquals("HServerInfo", "address: " + hostport + ", startcode: -1" +
|
||||||
", load: (requests: 0 regions: 0)", info.toString());
|
", load: (requests: 0 regions: 0)", info.toString());
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,8 +21,10 @@ package org.apache.hadoop.hbase.shell;
|
||||||
|
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
import java.io.PrintStream;
|
import java.io.PrintStream;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
|
import java.io.Writer;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -157,7 +159,9 @@ public class TestHBaseShell extends HBaseClusterTestCase {
|
||||||
private ReturnMsg runCommand(final String cmdStr)
|
private ReturnMsg runCommand(final String cmdStr)
|
||||||
throws ParseException, UnsupportedEncodingException {
|
throws ParseException, UnsupportedEncodingException {
|
||||||
LOG.info("Running command: " + cmdStr);
|
LOG.info("Running command: " + cmdStr);
|
||||||
Parser parser = new Parser(cmdStr);
|
Writer out = new OutputStreamWriter(System.out, "UTF-8");
|
||||||
|
TableFormatterFactory tff = new TableFormatterFactory(out, this.conf);
|
||||||
|
Parser parser = new Parser(cmdStr, out, tff.get());
|
||||||
Command cmd = parser.terminatedCommand();
|
Command cmd = parser.terminatedCommand();
|
||||||
ReturnMsg rm = cmd.execute(this.conf);
|
ReturnMsg rm = cmd.execute(this.conf);
|
||||||
dumpStdout();
|
dumpStdout();
|
||||||
|
|
Loading…
Reference in New Issue