You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2009/09/22 23:27:00 UTC

svn commit: r817845 - in /hadoop/hive/trunk: ./ hwi/src/java/org/apache/hadoop/hive/hwi/ hwi/src/test/org/apache/hadoop/hive/hwi/ hwi/web/

Author: namit
Date: Tue Sep 22 21:26:49 2009
New Revision: 817845

URL: http://svn.apache.org/viewvc?rev=817845&view=rev
Log:
HIVE-716. Web interface wait/notify, interface changes
(Edward Capriolo via namit)

Summary:

Trac Bug: #

Blame Rev:

Reviewed By: namit

Test Plan:

Revert Plan:

Database Impact:

Memcache Impact:

Other Notes:

EImportant:

- begin *PUBLIC* platform impact section -
Bugzilla: #
- end platform impact -


Added:
    hadoop/hive/trunk/hwi/web/session_result.jsp
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
    hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionManager.java
    hadoop/hive/trunk/hwi/src/test/org/apache/hadoop/hive/hwi/TestHWISessionManager.java
    hadoop/hive/trunk/hwi/web/session_manage.jsp
    hadoop/hive/trunk/hwi/web/session_remove.jsp

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Tue Sep 22 21:26:49 2009
@@ -134,6 +134,9 @@
 
     HIVE-795. Better error messages from Hive Server (Bill Graham via namit)
 
+    HIVE-716. Web interface wait/notify, interface changes
+    (Edward Capriolo via namit)
+
 Release 0.4.0 -  Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java (original)
+++ hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java Tue Sep 22 21:26:49 2009
@@ -1,3 +1,21 @@
+/**
+ * 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.hive.hwi;
 
 import java.io.File;
@@ -6,10 +24,13 @@
 import java.io.PrintStream;
 import java.util.Collection;
 import java.util.Vector;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.cli.OptionsProcessor;
-import org.apache.hadoop.hive.ql.processors.SetProcessor;
+import org.apache.hadoop.hive.ql.processors.CommandProcessor;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.ExecDriver;
@@ -27,152 +48,150 @@
  */
 public class HWISessionItem implements Runnable, Comparable<HWISessionItem> {
 
-	protected static final Log l4j = LogFactory.getLog(HWISessionItem.class
-			.getName());
-
-	/**
-	 * Represents the state a session item can be in.
-	 * 
-	 */
-	public enum WebSessionItemStatus {
-		NEW, QUERY_SET, QUERY_RUNNING, QUERY_COMPLETE, DESTROY, KILL_QUERY
-	};
-
-	private String sessionName;
-	private HWISessionItem.WebSessionItemStatus status;
-
-	private CliSessionState ss;
-	private SetProcessor sp;
-	private Driver qp;
-
-	private String resultFile;
-	private String errorFile;
-
-	private String query;
-	private int queryRet;
-
-	HiveConf conf;
-	Thread runnable;
-	HWIAuth auth;
-	private String historyFile;
-
-	/**
-	 * Creates an instance of WebSessionItem, sets status to NEW.
-	 */
-	protected HWISessionItem() {
-		l4j.debug("HWISessionItem created");
-		status = WebSessionItemStatus.NEW;
-		queryRet = -40;
-		runnable = new Thread(this);
-		runnable.start();
-	}
-
-	/**
-	 * This is the initialization process that is carried out for each
-	 * SessionItem. The goal is to emulate the startup of CLIDriver.
-	 */
-	protected void itemInit() {
-		l4j.debug("HWISessionItem itemInit start " + this.getSessionName());
-		OptionsProcessor oproc = new OptionsProcessor();
+  protected static final Log l4j = LogFactory.getLog(HWISessionItem.class.getName());
 
-		if (System.getProperty("hwi-args") != null) {
-			String[] parts = System.getProperty("hwi-args").split("\\s+");
+  /**  Represents the state a session item can be in. */
+  public enum WebSessionItemStatus {
+    NEW, READY, QUERY_SET, QUERY_RUNNING, DESTROY, KILL_QUERY
+  };
 
-			if (!oproc.process_stage1(parts)) {
-			}
-		}
-
-		SessionState.initHiveLog4j();
-		conf = new HiveConf(SessionState.class);
-		ss = new CliSessionState(conf);
-		SessionState.start(ss);
-		sp = new SetProcessor();
-		qp = new Driver();
-
-		runSetProcessorQuery("hadoop.job.ugi=" + auth.getUser() + ","
-				+ auth.getGroups()[0]);
-		runSetProcessorQuery("user.name=" + auth.getUser());
-		/*
-		 * HiveHistoryFileName will not be accessible outside this thread. We must
-		 * capture this now.
-		 */
-		this.historyFile = this.ss.get().getHiveHistory().getHistFileName();
-		l4j.debug("HWISessionItem itemInit Complete " + this.getSessionName());
-	}
-
-	/**
-	 * Set processor queries block for only a short amount of time. The client can
-	 * issue these directly.
-	 * 
-	 * @param query
-	 *          This is a query in the form of SET THIS=THAT
-	 * @return chained call to setProcessor.run(String)
-	 */
-	public int runSetProcessorQuery(String query) {
-		return sp.run(query);
-	}
-
-	/**
-	 * HWISessionItem is a Runnable instance. Calling this method will change the
-	 * status to QUERY_SET and notify(). The run method detects this and then
-	 * continues processing.
-	 */
-	public void clientStart() throws HWIException {
-
-		if (this.status == WebSessionItemStatus.QUERY_RUNNING) {
-			throw new HWIException("Query already running");
-		}
-		this.status = WebSessionItemStatus.QUERY_SET;
-		synchronized (this.runnable) {
-			this.runnable.notifyAll();
-		}
-		l4j.debug(this.getSessionName() + " Query is set to start");
-	}
+  /** The Web Interface sessionName this is used to identify the session */
+  private String sessionName;
+	
+  /** Respresents the current status of the session. Used by components to determine state. 
+  * Operations will throw exceptions if the item is not in the correct state. */
+  private HWISessionItem.WebSessionItemStatus status;
+
+  private CliSessionState ss;
+  
+  /** Standard out from the session will be written to this local file */
+  private String resultFile;
+  
+  /** Standard error from the session will be written to this local file */
+  private String errorFile;
+
+  /** The results from the Driver. This is used for storing the most result 
+  results from the driver in memory */ 
+  private Vector<Vector<String>>resultBucket;
+
+  /** Limits the resultBucket to be no greater then this size */
+  private int resultBucketMaxSize;
+
+  /** List of queries that this item should/has operated on */
+  private List<String>  queries;
+
+  /** status code results of queries */
+  private List<Integer> queryRet;
+
+  /** Reference to the configuration */
+  private HiveConf conf;
+  
+  /** User privileges */
+  private HWIAuth auth;
+
+  public Thread runnable;
+
+  /** Threading SessionState issues require us to capture a reference to 
+  * the hive history file and store it*/
+  private String historyFile;
+
+  /**
+  * Creates an instance of WebSessionItem, sets status to NEW.
+  */
+  public HWISessionItem(HWIAuth auth, String sessionName) {
+    this.auth=auth;
+    this.sessionName=sessionName;
+    l4j.debug("HWISessionItem created");
+    status = WebSessionItemStatus.NEW;
+    queries = new ArrayList<String>();
+    queryRet = new ArrayList<Integer>();
+    resultBucket = new Vector<Vector<String>>();
+    resultBucketMaxSize=1000;
+    runnable = new Thread(this);
+    runnable.start();
+  
+    l4j.debug("Wait for NEW->READY transition");
+    synchronized (this.runnable){
+      if (this.status != WebSessionItemStatus.READY) {
+	try {
+	  this.runnable.wait();
+        } catch (Exception ex) {}
+      }
+    }
+    l4j.debug("NEW->READY transition complete");
+  }
 
-	public void clientKill() throws HWIException {
-		if (this.status != WebSessionItemStatus.QUERY_RUNNING) {
-			throw new HWIException("Can not kill that which is not running.");
-		}
-		this.status = WebSessionItemStatus.KILL_QUERY;
-		l4j.debug(this.getSessionName() + " Query is set to KILL_QUERY");
-	}
+  /**
+  * This is the initialization process that is carried out for each
+  * SessionItem. The goal is to emulate the startup of CLIDriver.
+  */
+  private void itemInit() {
+    l4j.debug("HWISessionItem itemInit start " + this.getSessionName());
+    OptionsProcessor oproc = new OptionsProcessor();
+
+    if (System.getProperty("hwi-args") != null) {
+      String[] parts = System.getProperty("hwi-args").split("\\s+");
+      if (!oproc.process_stage1(parts)) {
+      }
+    }
 
-	/** This method clears the private member variables. */
-	public void clientRenew() throws HWIException {
-		if (this.status == WebSessionItemStatus.QUERY_RUNNING) {
-			throw new HWIException("Query already running");
-		}
+    SessionState.initHiveLog4j();
+    conf = new HiveConf(SessionState.class);
+    ss = new CliSessionState(conf);
+    SessionState.start(ss);
+    queries.add("set hadoop.job.ugi=" + auth.getUser() + ","
+                                + auth.getGroups()[0]);
+    queries.add("set user.name="+auth.getUser() );
+    /*
+    * HiveHistoryFileName will not be accessible outside this thread. We must
+    * capture this now.
+    */
+    this.historyFile = this.ss.get().getHiveHistory().getHistFileName();
+    l4j.debug("HWISessionItem itemInit Complete " + this.getSessionName());
+    this.status= WebSessionItemStatus.READY;
 
-		this.query = null;
-		this.resultFile = null;
-		this.errorFile = null;
-		this.status = WebSessionItemStatus.NEW;
-		this.resultFile = null;
-		this.conf = null;
-		this.ss = null;
-		this.qp = null;
-		this.sp = null;
-		l4j.debug(this.getSessionName() + " Query is renewed to start");
-	}
+    synchronized (this.runnable){
+      this.runnable.notifyAll();
+    }
+  }
 
-	/**
-	 * This is a chained call to SessionState.setIsSilent(). Use this if you do
-	 * not want the result file to have information status
-	 */
-	public void setSSIsSilent(boolean silent) throws HWIException {
-		if (ss == null)
-			throw new HWIException("Session State is null");
-		this.ss.setIsSilent(silent);
-	}
+  /**
+  * HWISessionItem is a Runnable instance. Calling this method will change the
+  * status to QUERY_SET and notify(). The run method detects this and then
+  * continues processing.
+  */
+  public void clientStart() throws HWIException {
+    if (this.status == WebSessionItemStatus.QUERY_RUNNING) {
+      throw new HWIException("Query already running");
+    }
+    this.status = WebSessionItemStatus.QUERY_SET;
+    synchronized (this.runnable) {
+      this.runnable.notifyAll();
+    }
+    l4j.debug(this.getSessionName() + " Query is set to start");
+  }
 
-	/**
-	 * This is a chained call to SessionState.getIsSilent()
-	 */
-	public boolean getSSIsSilent() throws HWIException {
-		if (ss == null)
-			throw new HWIException("Session State is null");
-		return ss.getIsSilent();
-	}
+  public void clientKill() throws HWIException {
+    if (this.status != WebSessionItemStatus.QUERY_RUNNING) {
+      throw new HWIException("Can not kill that which is not running.");
+    }
+    this.status = WebSessionItemStatus.KILL_QUERY;
+    l4j.debug(this.getSessionName() + " Query is set to KILL_QUERY");
+  }
+
+  /** This method clears the private member variables. */
+  public void clientRenew() throws HWIException {
+    throwIfRunning();
+    this.queries = new ArrayList<String>();
+    this.queryRet = new ArrayList<Integer>();
+    this.resultBucket = new Vector<Vector<String>>();
+    this.resultFile = null;
+    this.errorFile = null;
+    //this.conf = null;
+    //this.ss = null;
+    this.status = WebSessionItemStatus.NEW;
+    l4j.debug(this.getSessionName() + " Query is renewed to start");
+  }
 
 	/**
 	 * This is a callback style function used by the HiveSessionManager. The
@@ -245,216 +264,211 @@
 		}
 		return jtparts[0]+":"+jthttpParts[1]+"/jobdetails.jsp?jobid="+jobid+"&refresh=30";
 	}
-	@Override
-	/*
-	 * HWISessionItem uses a wait() notify() system. If the thread detects conf to
-	 * be null, control is transfered to initItem().A status of QUERY_SET causes
-	 * control to transfer to the runQuery() method. DESTROY will cause the run
-	 * loop to end permanently.
-	 */
-	public void run() {
-		synchronized (this.runnable) {
-			while (this.status != HWISessionItem.WebSessionItemStatus.DESTROY) {
-				if (conf == null) {
-					this.itemInit();
-				}
-				if (this.status == WebSessionItemStatus.QUERY_SET) {
-					this.runQuery();
-				}
-				try {
-					this.runnable.wait();
-				} catch (InterruptedException e) {
-					l4j.error("in wait() state ", e);
-				}
-			}
-		}
-	}
-
-	/**
-	 * This method calls the qp.run() method, writes the output to the result
-	 * file, when finished the status will be QUERY_COMPLETE.
-	 */
-	public void runQuery() {
-
-		FileOutputStream fos = null;
-
-		if (this.getResultFile() != null) {
-			try {
-				fos = new FileOutputStream(new File(this.resultFile));
-				ss.out = new PrintStream(fos, true, "UTF-8");
-			} catch (java.io.FileNotFoundException fex) {
-				l4j.error(this.getSessionName() + " opening resultfile "
+  @Override
+  /*
+  * HWISessionItem uses a wait() notify() system. If the thread detects conf to
+  * be null, control is transfered to initItem(). A status of QUERY_SET causes
+  * control to transfer to the runQuery() method. DESTROY will cause the run
+  * loop to end permanently.
+  */
+  public void run() {
+    synchronized (this.runnable) {
+      while (this.status != HWISessionItem.WebSessionItemStatus.DESTROY) {
+        if (this.status == WebSessionItemStatus.NEW) {
+	  this.itemInit();
+	}
+ 					
+	if (this.status == WebSessionItemStatus.QUERY_SET) {
+	  this.runQuery();
+	}
+				
+	try {
+	  this.runnable.wait();
+	} catch (InterruptedException e) {
+	  l4j.error("in wait() state ", e);
+	}
+      } //end while
+    } //end sync
+  } //end run
+
+  /**
+  runQuery iterates the list of queries executing each query.
+  */
+  public void runQuery() {
+    FileOutputStream fos = null;
+    if (this.getResultFile() != null) {
+      try {
+	fos = new FileOutputStream(new File(this.resultFile));
+	ss.out = new PrintStream(fos, true, "UTF-8");
+      } catch (java.io.FileNotFoundException fex) {
+        l4j.error(this.getSessionName() + " opening resultfile "
 						+ this.resultFile, fex);
-			} catch (java.io.UnsupportedEncodingException uex) {
-				l4j.error(this.getSessionName() + " opening resultfile "
+      } catch (java.io.UnsupportedEncodingException uex) {
+        l4j.error(this.getSessionName() + " opening resultfile "
 						+ this.resultFile, uex);
-			}
+      }
+    } else {
+      l4j.debug(this.getSessionName() + " Output file was not specified");
+    }
+    l4j.debug(this.getSessionName() + " state is now QUERY_RUNNING.");
+    this.status = WebSessionItemStatus.QUERY_RUNNING;
+
+    //expect one return per query
+    queryRet = new ArrayList<Integer> ( this.queries.size() );
+    for (int i=0;i<this.queries.size();i++){
+      String cmd = queries.get(i);
+      String cmd_trimmed = cmd.trim();
+      String[] tokens = cmd_trimmed.split("\\s+");
+      String cmd_1 = cmd_trimmed.substring(tokens[0].length()).trim();
+
+      CommandProcessor proc = CommandProcessorFactory.get(tokens[0]);
+      if (proc !=null){
+        if (proc instanceof Driver) {
+          Driver qp = (Driver) proc;
+          queryRet.add ( new Integer(qp.run(cmd)));
+          Vector<String> res = new Vector<String>();
+          try {
+            while (qp.getResults(res)) {
+	      resultBucket.add(res);
+              if (resultBucket.size() > resultBucketMaxSize) 
+    		resultBucket.remove(0);
+              for (String row : res) {
+		if (ss != null) { 
+  	          if (ss.out != null) 
+  	            ss.out.println(row);
 		} else {
-			l4j.debug(this.getSessionName() + " Output file was not specified");
+		  throw new RuntimeException ("ss was null" );
 		}
+  	      }
+//  	      res.clear();
+            }
+          } catch (IOException ex) {
+            l4j.error(this.getSessionName() + " getting results "
+            + this.getResultFile() + " caused exception.", ex);
+          }
+          qp.close();
+        } else {
+          queryRet.add( new Integer(proc.run(cmd_1) ) );
+        }
+      } else {
+        //processor was null
+	l4j.error(this.getSessionName() + 
+	" query processor was not found for query "+ cmd );
+      } 
+    } // end for
 
-		l4j.debug(this.getSessionName() + " state is now QUERY_RUNNING.");
-		this.status = WebSessionItemStatus.QUERY_RUNNING;
-
-		queryRet = qp.run(this.query);
-		Vector<String> res = new Vector<String>();
+    //cleanup
     try {
-  		while (qp.getResults(res)) {
-  			for (String row : res) {
-  				if (ss.out != null) {
-  					ss.out.println(row);
-  				}
-  			}
-  			res.clear();
-  		}
+      if (fos != null) {
+        fos.close();
+      }
     } catch (IOException ex) {
-      l4j.error(this.getSessionName() + " getting results "
-          + this.getResultFile() + " caused exception.", ex);
+      l4j.error(this.getSessionName() + " closing result file "
+	+ this.getResultFile() + " caused exception.", ex);
     }
-		try {
-			if (fos != null) {
-				fos.close();
-			}
-		} catch (IOException ex) {
-			l4j.error(this.getSessionName() + " closing result file "
-					+ this.getResultFile() + " caused exception.", ex);
-		}
-		this.status = WebSessionItemStatus.QUERY_COMPLETE;
-		l4j.debug(this.getSessionName() + " state is now QUERY_COMPLETE.");
-	}
-
-	public int compareTo(HWISessionItem other) {
-		if (other == null)
-			return -1;
-		return this.getSessionName().compareTo(other.getSessionName());
-	}
-
-	/**
-	 * 
-	 * @return the HiveHistoryViewer for the session
-	 * @throws HWIException
-	 */
-	public HiveHistoryViewer getHistoryViewer() throws HWIException {
-		if (ss == null)
-			throw new HWIException("Session state was null");
-		/*
-		 * we can not call this.ss.get().getHiveHistory().getHistFileName() directly
-		 * as this call is made from a a Jetty thread and will return null
-		 */
-		HiveHistoryViewer hv = new HiveHistoryViewer(this.historyFile);
-		return hv;
-	}
-
-	/**
-	 * Uses the sessionName property to compare to sessions
-	 * 
-	 * @return true if sessionNames are equal false otherwise
-	 */
-	public boolean equals(Object other) {
-		if (other == null)
-			return false;
-		if (!(other instanceof HWISessionItem))
-			return false;
-		HWISessionItem o = (HWISessionItem) other;
-		if (this.getSessionName().equals(o.getSessionName())) {
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	protected void setQp(Driver qp) {
-		this.qp = qp;
-	}
-
-	/**
-	 * The query executed by Hive
-	 * 
-	 * @return The query that this is executing or will be executed
-	 */
-	public String getQuery() {
-		return query;
-	}
-
-	/**
-	 * Use this function to set the query that Hive will run.
-	 * 
-	 * @param query
-	 *          A query in Hive Query Language
-	 */
-	public void setQuery(String query) {
-		this.query = query;
-	}
-
-	/**
-	 * Used to determine the status of a query, possibly why it failed
-	 * 
-	 * @return The result from Hive queryProcessor
-	 */
-	public int getQueryRet() {
-		return queryRet;
-	}
-
-	protected void setQueryRet(int queryRet) {
-		this.queryRet = queryRet;
-	}
-
-	public String getResultFile() {
-		return resultFile;
-	}
-
-	public void setResultFile(String resultFile) {
-		this.resultFile = resultFile;
-	}
-
-	/**
-	 * The session name is an identifier to recognize the session
-	 * 
-	 * @return the session's name
-	 */
-	public String getSessionName() {
-		return sessionName;
-	}
-
-	protected void setSessionName(String sessionName) {
-		this.sessionName = sessionName;
-	}
-
-	protected SetProcessor getSp() {
-		return sp;
-	}
-
-	protected void setSp(SetProcessor sp) {
-		this.sp = sp;
-	}
-
-	protected CliSessionState getSs() {
-		return ss;
-	}
+    this.status = WebSessionItemStatus.READY;
+    l4j.debug(this.getSessionName() + " state is now READY");
+    synchronized (this.runnable){
+      this.runnable.notifyAll();
+    }
+  }
 
-	protected void setSs(CliSessionState ss) {
-		this.ss = ss;
-	}
+ /**
+ * This is a chained call to SessionState.setIsSilent(). Use this if you do
+ * not want the result file to have information status
+ */
+  public void setSSIsSilent(boolean silent) throws HWIException {
+    if (ss == null)
+      throw new HWIException("Session State is null");
+    this.ss.setIsSilent(silent);
+ }
 
-	/**
-	 * Used to represent to the user and other components what state the
-	 * HWISessionItem is in. Certain commands can only be run when the application
-	 * is in certain states.
-	 * 
-	 * @return the current status of the session
-	 */
-	public WebSessionItemStatus getStatus() {
-		return status;
-	}
+ /**
+ * This is a chained call to SessionState.getIsSilent()
+ */
+ public boolean getSSIsSilent() throws HWIException {
+   if (ss == null)
+     throw new HWIException("Session State is null");
+   return ss.getIsSilent();
+ }
+
+  /** to support sorting/Set*/
+  public int compareTo(HWISessionItem other) {
+    if (other == null)
+      return -1;
+    return this.getSessionName().compareTo(other.getSessionName());
+  }
+
+  /**
+  * 
+  * @return the HiveHistoryViewer for the session
+  * @throws HWIException
+  */
+  public HiveHistoryViewer getHistoryViewer() throws HWIException {
+    if (ss == null)
+      throw new HWIException("Session state was null");
+    /*
+    * we can not call this.ss.get().getHiveHistory().getHistFileName() directly
+    * as this call is made from a a Jetty thread and will return null
+    */
+    HiveHistoryViewer hv = new HiveHistoryViewer(this.historyFile);
+    return hv;
+  }
+
+  /**
+  * Uses the sessionName property to compare to sessions
+  * 
+  * @return true if sessionNames are equal false otherwise
+  */
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (!(other instanceof HWISessionItem))
+      return false;
+    HWISessionItem o = (HWISessionItem) other;
+    if (this.getSessionName().equals(o.getSessionName())) {
+      return true;
+    } else {
+      return false;
+    }
+  }
 
-	/**
-	 * Currently unused
-	 * 
-	 * @return a String with the full path to the error file.
-	 */
-	public String getErrorFile() {
-		return errorFile;
-	}
+  public String getResultFile() {
+    return resultFile;
+  }
+
+  public void setResultFile(String resultFile) {
+    this.resultFile = resultFile;
+  }
+
+  /**
+  * The session name is an identifier to recognize the session
+  * 
+  * @return the session's name
+  */
+  public String getSessionName() {
+    return sessionName;
+  }
+
+  /**
+  * Used to represent to the user and other components what state the
+  * HWISessionItem is in. Certain commands can only be run when the application
+  * is in certain states.
+  * 
+  * @return the current status of the session
+  */
+  public WebSessionItemStatus getStatus() {
+    return status;
+  }
+
+  /**
+  * Currently unused
+  * 
+  * @return a String with the full path to the error file.
+  */
+  public String getErrorFile() {
+    return errorFile;
+  }
 
 	/**
 	 * Currently unused
@@ -462,22 +476,82 @@
 	 * @param errorFile
 	 *          the full path to the file for results.
 	 */
-	public void setErrorFile(String errorFile) {
-		this.errorFile = errorFile;
-	}
-
-	/**
-	 * @return the auth
-	 */
-	public HWIAuth getAuth() {
-		return auth;
-	}
-
-	/**
-	 * @param auth the auth to set
-	 */
-	protected void setAuth(HWIAuth auth) {
-		this.auth = auth;
-	}
+  public void setErrorFile(String errorFile) {
+    this.errorFile = errorFile;
+  }
+
+  /**
+  * @return the auth
+  */
+  public HWIAuth getAuth() {
+    return auth;
+  }
+
+  /**
+  * @param auth the auth to set
+  */
+  protected void setAuth(HWIAuth auth) {
+    this.auth = auth;
+  }
+
+  /** returns an unmodifiable list of queries */
+  public List<String> getQueries(){
+    return java.util.Collections.unmodifiableList(this.queries);
+  }
+  
+  /** adds a new query to the execution list 
+   @param query query to be added to the list*/
+  public void addQuery(String query) throws HWIException {
+    throwIfRunning();
+    this.queries.add(query);
+  }
+
+  /** removes a query from the execution list 
+  * @param item the 0 based index of the item to be removed
+  */
+  public void removeQuery(int item) throws HWIException {
+    throwIfRunning();
+    queries.remove(item);  	
+  }
+ 
+  public void clearQueries() throws HWIException {
+    throwIfRunning();
+    queries.clear();
+  }
+
+  /** returns the value for resultBucketMaxSize */
+  public int getResultBucketMaxSize(){
+    return resultBucketMaxSize;
+  }
+
+  /** sets the value for resultBucketMaxSize 
+    @param size the new size
+  */
+  public void setResultBucketMaxSize(int size){
+    resultBucketMaxSize=size;
+  }
+
+  /** gets the value for resultBucket */
+  public Vector<Vector<String>> getResultBucket(){
+    return resultBucket;
+  }
+
+  /**
+  * The HWISessionItem stores the result of each query in an array
+  * @return unmodifiable list of return codes 
+  */
+  public List<Integer> getQueryRet(){
+    return java.util.Collections.unmodifiableList(queryRet);
+  }
+
+  /**
+  * If the ItemStatus is QueryRunning most of the configuration
+  * is in a read only state.  
+  */
+  private void throwIfRunning() throws HWIException {
+    if (this.status == WebSessionItemStatus.QUERY_RUNNING) {
+      throw new HWIException("Query already running");
+    }
+  }
 
 }

Modified: hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionManager.java?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionManager.java (original)
+++ hadoop/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionManager.java Tue Sep 22 21:26:49 2009
@@ -1,3 +1,20 @@
+/**
+ * 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.hive.hwi;
 
 import java.util.*;
@@ -136,9 +153,7 @@
 			if (findSessionItemByName(a, sessionName) == null) {
 				l4j.debug("Initializing session: " + sessionName + " a for "
 						+ a.getUser());
-				si = new HWISessionItem();
-				si.setSessionName(sessionName);
-				si.setAuth(a);
+				si = new HWISessionItem(a,sessionName);
 
 				if (!items.containsKey(a)) {
 					l4j.debug("SessionList is empty " + a.getUser());

Modified: hadoop/hive/trunk/hwi/src/test/org/apache/hadoop/hive/hwi/TestHWISessionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/src/test/org/apache/hadoop/hive/hwi/TestHWISessionManager.java?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/hwi/src/test/org/apache/hadoop/hive/hwi/TestHWISessionManager.java (original)
+++ hadoop/hive/trunk/hwi/src/test/org/apache/hadoop/hive/hwi/TestHWISessionManager.java Tue Sep 22 21:26:49 2009
@@ -1,3 +1,21 @@
+/**
+ * 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.hive.hwi;
 
 import java.io.BufferedReader;
@@ -11,7 +29,8 @@
 import org.apache.hadoop.hive.hwi.HWISessionItem;
 import org.apache.hadoop.hive.hwi.HWISessionManager;
 import org.apache.hadoop.hive.ql.history.HiveHistoryViewer;
-
+import java.util.ArrayList;
+import java.util.Vector;
 public class TestHWISessionManager extends TestCase {
 
 	private static String tableName = "test_hwi_table";
@@ -66,30 +85,45 @@
 		HWISessionItem searchItem = hsm.findSessionItemByName(user1, "session1");
 		assertEquals(searchItem, user1_item1);
 
-		searchItem.setQuery("create table " + tableName
+		searchItem.addQuery("create table " + tableName
 				+ " (key int, value string)");
+		searchItem.addQuery("describe "+tableName);
 		searchItem.clientStart();
 
 		// wait for the session manager to make the table. It is non blocking API.
-		while (searchItem.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
-			Thread.sleep(1);
-		}
-		assertEquals(searchItem.getQueryRet(), 0);
-
+ 		synchronized (searchItem.runnable ) {
+		  while (searchItem.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
+			searchItem.runnable.wait();
+                  }
+		}
+		ArrayList<Integer> zero = new ArrayList<Integer>(); zero.add(0); zero.add(0); zero.add(0); zero.add(0);
+		ArrayList<Integer> zero3 = new ArrayList<Integer>(); zero3.add(0); zero3.add(0); zero3.add(0);
+		ArrayList<Integer> zero1 = new ArrayList<Integer>(); zero1.add(0);
+		assertEquals( zero , searchItem.getQueryRet());
+		
+		Vector<Vector<String>> searchBlockRes = searchItem.getResultBucket();
+		
+		String resLine =  searchBlockRes.get(0).get(0) ;
+		assertEquals(true, resLine.contains("key") );
+                assertEquals(true, resLine.contains("int") );
+                String resLine2= searchBlockRes.get(0).get(1) ;
+                assertEquals(true, resLine2.contains("value") );
+                assertEquals(true, resLine2.contains("string") );
+	
 		// load data into table
 		searchItem.clientRenew();
-		searchItem.setQuery(("load data local inpath '" + dataFilePath.toString()
-				+ "' into table " + tableName));
+		searchItem.addQuery("load data local inpath '" + dataFilePath.toString()
+				+ "' into table " + tableName);
 		searchItem.clientStart();
-		while (searchItem.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
+		while (searchItem.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
 			Thread.sleep(1);
 		}
-		assertEquals(searchItem.getQueryRet(), 0);
+		assertEquals(zero1 , searchItem.getQueryRet());
 
 		// start two queries simultaniously
-		user1_item2.setQuery("select distinct(test_hwi_table.key) from "
+		user1_item2.addQuery("select distinct(test_hwi_table.key) from "
 				+ tableName);
-		user2_item1.setQuery("select distinct(test_hwi_table.key) from "
+		user2_item1.addQuery("select distinct(test_hwi_table.key) from "
 				+ tableName);
 
 		// set result files to compare results
@@ -114,16 +148,21 @@
 		user1_item2.clientStart();
 		user2_item1.clientStart();
 
-		while (user1_item2.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
-			Thread.sleep(1);
-		}
-		while (user2_item1.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
-			Thread.sleep(1);
+		synchronized (user1_item2.runnable) {
+		  while (user1_item2.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
+		    user1_item2.runnable.wait();		
+		  }
 		}
 
-		assertEquals(user1_item2.getQueryRet(), 0);
-		assertEquals(user2_item1.getQueryRet(), 0);
-		assertEquals(isFileContentEqual(result1, result2), true);
+ 		synchronized (user2_item1.runnable) {
+		  while (user2_item1.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
+		    user2_item1.runnable.wait();
+		  }
+                }
+
+		assertEquals(zero3 , user1_item2.getQueryRet() );
+		assertEquals(zero3 , user2_item1.getQueryRet() );
+		assertEquals(true , isFileContentEqual(result1, result2));
 
 		// clean up the files
 		result1.delete();
@@ -131,26 +170,31 @@
 
 		// test a session renew/refresh
 		user2_item1.clientRenew();
-		user2_item1.setQuery("select distinct(test_hwi_table.key) from "
+		user2_item1.addQuery("select distinct(test_hwi_table.key) from "
 				+ tableName);
 		user2_item1.clientStart();
-		while (user2_item1.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
-			Thread.sleep(1);
+
+		synchronized (user2_item1.runnable ) {
+		  while (user2_item1.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
+		    user2_item1.runnable.wait();
+		  }
 		}
 
 		// cleanup
 		HWISessionItem cleanup = hsm.createSession(user1, "cleanup");
-		cleanup.setQuery("drop table " + tableName);
+		cleanup.addQuery("drop table " + tableName);
 		cleanup.clientStart();
 
-		while (cleanup.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE) {
-			Thread.sleep(1);
+		synchronized (cleanup.runnable) {
+		  while (cleanup.getStatus() != HWISessionItem.WebSessionItemStatus.READY) {
+		    cleanup.runnable.wait();
+ 		  }
 		}
 
 		// test the history is non null object.
 		HiveHistoryViewer hhv = cleanup.getHistoryViewer();
 		assertNotNull(hhv);
-		assertEquals(cleanup.getQueryRet(), 0);
+		assertEquals( zero3 , cleanup.getQueryRet() );
 	}
 
 	public boolean isFileContentEqual(File one, File two) throws Exception {

Modified: hadoop/hive/trunk/hwi/web/session_manage.jsp
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/web/session_manage.jsp?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/hwi/web/session_manage.jsp (original)
+++ hadoop/hive/trunk/hwi/web/session_manage.jsp Tue Sep 22 21:26:49 2009
@@ -1,3 +1,22 @@
+<%--
+/**
+ * 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.
+ */
+--%>
 <%@page import="org.apache.hadoop.hive.hwi.*" %>
 <%@page errorPage="error_page.jsp" %>
 <% HWISessionManager hs = (HWISessionManager) application.getAttribute("hs");; %>
@@ -18,27 +37,28 @@
 %>
 <% 
   if (request.getParameter("start")!=null ){ 
-    if ( sess.getStatus()!=HWISessionItem.WebSessionItemStatus.QUERY_RUNNING){
+    if ( sess.getStatus()==HWISessionItem.WebSessionItemStatus.READY){
       sess.setErrorFile(errorFile);
-	  sess.setResultFile(resultFile);
-	  sess.setQuery(query);
-	  
-	  if (query.length()==0){
-		  message="You did not specify a query";
-		  start="NO";
-	  }
-	  
-	  if (silent.equalsIgnoreCase("YES") )
-		sess.setSSIsSilent(true);
-	  else
-		sess.setSSIsSilent(false);
+      sess.setResultFile(resultFile);
+      sess.clearQueries();
+      for (String q : query.split(";") ){
+        sess.addQuery(q);
+      }
+      if (query.length()==0){
+        message="You did not specify a query";
+        start="NO";
+      }
+      if (silent.equalsIgnoreCase("YES") )
+	sess.setSSIsSilent(true);
+      else
+	sess.setSSIsSilent(false);
 		   
-	  message="Changes accepted.";
-	  if (start.equalsIgnoreCase("YES") ){
-	    sess.clientStart();
-		message="Session is set to start.";
-	  }
+	message="Changes accepted.";
+	if (start.equalsIgnoreCase("YES") ){
+	  sess.clientStart();
+          message="Session is set to start.";
 	}
+      }
   } 
 %>
 
@@ -72,9 +92,9 @@
           
           Session History:  <a href="/hwi/session_history.jsp?sessionName=<%=sessionName%>"><%=sessionName%></a><br> 
           Session Diagnostics: <a href="/hwi/session_diagnostics.jsp?sessionName=<%=sessionName%>"><%=sessionName%></a><br>
-		  Set Processor: <a href="/hwi/set_processor.jsp?sessionName=<%=sessionName%>"><%=sessionName%></a><br> 
           Session Remove: <a href="/hwi/session_remove.jsp?sessionName=<%=sessionName%>"><%=sessionName%></a><br> 
-          <br>
+          Session Result Bucket: <a href="/hwi/session_result.jsp?sessionName=<%=sessionName%>"><%=sessionName%></a><br>
+	<br>
           
 			<form action="session_manage.jsp">
 				<input type="hidden" name="sessionName" value="<%=sessionName %>">
@@ -102,7 +122,13 @@
 				<tr>
 					<td>Query</td>
 					<td><textarea name="query" rows="8" cols="70"><% 
-				if (sess.getQuery()==null) { out.print(""); } else { out.print(sess.getQuery()); }
+				if (sess.getQueries()==null) { 
+				  out.print(""); 
+				} else { 
+			          for (String qu: sess.getQueries() ) {
+				    out.print(qu); out.print(" ; ");
+				  }	
+				}
 				%></textarea></td>
 				</tr>
 				
@@ -129,8 +155,11 @@
 				</tr>
 					
 				<tr>
-					<td>Query Return Code</td>
-					<td><%=sess.getQueryRet() %></td>
+					<td>Query Return Codes</td>
+					<td> <% for (int i=0; i< sess.getQueryRet().size();++i ){ %>
+						<%=i%> : <%=sess.getQueryRet().get(i)%><br>
+					     <% } %>
+					</td>
 				</tr>
 				<tr>
 					<td colSpan="2">

Modified: hadoop/hive/trunk/hwi/web/session_remove.jsp
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/web/session_remove.jsp?rev=817845&r1=817844&r2=817845&view=diff
==============================================================================
--- hadoop/hive/trunk/hwi/web/session_remove.jsp (original)
+++ hadoop/hive/trunk/hwi/web/session_remove.jsp Tue Sep 22 21:26:49 2009
@@ -1,3 +1,22 @@
+<%--
+/**
+ * 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.
+ */
+--%>
 <%@ page import="org.apache.hadoop.hive.hwi.*" %>
 <%@ page errorPage="error_page.jsp" %>
 <% HWIAuth auth = (HWIAuth) session.getAttribute("auth"); %>
@@ -10,8 +29,7 @@
 <% 
   if (request.getParameter("confirm")!=null){ 
     HWISessionItem i = hs.findSessionItemByName(auth,sessionName);	
-    if (i.getStatus() == HWISessionItem.WebSessionItemStatus.QUERY_COMPLETE || 
-    		i.getStatus() == HWISessionItem.WebSessionItemStatus.NEW){
+    if (i.getStatus() != HWISessionItem.WebSessionItemStatus.QUERY_RUNNING ){ 
 		hs.findAllSessionsForUser(auth).remove(i);
 		message="Session removed";
     } else {

Added: hadoop/hive/trunk/hwi/web/session_result.jsp
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/hwi/web/session_result.jsp?rev=817845&view=auto
==============================================================================
--- hadoop/hive/trunk/hwi/web/session_result.jsp (added)
+++ hadoop/hive/trunk/hwi/web/session_result.jsp Tue Sep 22 21:26:49 2009
@@ -0,0 +1,67 @@
+<%-- 
+ /**
+ * 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.
+ */
+--%>
+<%@page contentType="text/html" pageEncoding="UTF-8" %>
+<%@page errorPage="error_page.jsp" %>
+<%@page import="org.apache.hadoop.hive.hwi.*" %>
+<%@page import="org.apache.hadoop.hive.conf.*" %>
+<%@page import="java.util.*" %>
+
+<% HWISessionManager hs = (HWISessionManager) application.getAttribute("hs"); %>
+<% if (hs == null) { %>
+  <jsp:forward page="error.jsp">
+    <jsp:param name="message" value="Hive Session Manager Not Found" />
+  </jsp:forward>
+<% } %>
+
+<% HWIAuth auth = (HWIAuth) session.getAttribute("auth"); %>
+<% if (auth==null) { %>
+	<jsp:forward page="/authorize.jsp" />
+<% } %>
+<% String sessionName = request.getParameter("sessionName"); %>
+<% HWISessionItem si = hs.findSessionItemByName(auth,sessionName); %>
+<html>
+  <head>
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <title>Session Result Bucket</title>
+  </head>
+  <body>
+    <table>
+      <tr>
+        <td valign="top"><jsp:include page="left_navigation.jsp"/></td>
+        <td valign="top">
+          <h2>Session Result Bucket</h2>
+          
+          <% if (si!=null) { %>
+            <table border=1">
+            <% for (Vector<String> row : si.getResultBucket() ) { %>
+              <tr>
+	      <% for (String col : row ) { %>  
+		<td><%=col%></td>
+	      <% } %>
+              </tr>
+            <% } %>
+	    </table>	
+          <% } %>
+          
+        </td>
+      </tr>
+    </table>
+  </body>
+</html>