Return-Path: Delivered-To: apmail-hadoop-zookeeper-commits-archive@minotaur.apache.org Received: (qmail 35835 invoked from network); 9 Jun 2010 15:27:24 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 9 Jun 2010 15:27:24 -0000 Received: (qmail 28636 invoked by uid 500); 9 Jun 2010 15:27:24 -0000 Delivered-To: apmail-hadoop-zookeeper-commits-archive@hadoop.apache.org Received: (qmail 28612 invoked by uid 500); 9 Jun 2010 15:27:24 -0000 Mailing-List: contact zookeeper-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: zookeeper-dev@ Delivered-To: mailing list zookeeper-commits@hadoop.apache.org Received: (qmail 28604 invoked by uid 99); 9 Jun 2010 15:27:24 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 09 Jun 2010 15:27:24 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 09 Jun 2010 15:27:15 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id D6D332388A74; Wed, 9 Jun 2010 15:26:26 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit Subject: svn commit: r953041 [2/6] - in /hadoop/zookeeper/trunk: ./ src/contrib/loggraph/ src/contrib/loggraph/bin/ src/contrib/loggraph/src/ src/contrib/loggraph/src/java/ src/contrib/loggraph/src/java/org/ src/contrib/loggraph/src/java/org/apache/ src/contrib... Date: Wed, 09 Jun 2010 15:26:25 -0000 To: zookeeper-commits@hadoop.apache.org From: phunt@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100609152628.D6D332388A74@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/TxnLogSource.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/TxnLogSource.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/TxnLogSource.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/TxnLogSource.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,377 @@ +/** + * 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.zookeeper.graph; + +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.FileInputStream; +import java.io.IOException; +import java.text.DateFormat; +import java.util.Date; +import java.util.zip.Adler32; +import java.util.zip.Checksum; +import java.util.HashMap; + +import org.apache.jute.BinaryInputArchive; +import org.apache.jute.InputArchive; +import org.apache.jute.Record; +import org.apache.zookeeper.server.TraceFormatter; +import org.apache.zookeeper.server.persistence.FileHeader; +import org.apache.zookeeper.server.persistence.FileTxnLog; +import org.apache.zookeeper.server.util.SerializeUtils; +import org.apache.zookeeper.txn.TxnHeader; + +import org.apache.zookeeper.ZooDefs.OpCode; + +import org.apache.zookeeper.txn.CreateSessionTxn; +import org.apache.zookeeper.txn.CreateTxn; +import org.apache.zookeeper.txn.DeleteTxn; +import org.apache.zookeeper.txn.ErrorTxn; +import org.apache.zookeeper.txn.SetACLTxn; +import org.apache.zookeeper.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnHeader; + +import java.io.File; +import java.io.Closeable; +import java.io.FileNotFoundException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import org.apache.log4j.Logger; + +public class TxnLogSource implements LogSource { + private static final Logger LOG = Logger.getLogger(TxnLogSource.class); + + private LogSkipList skiplist = null; + private static final int skipN = 10000; + + private String file = null; + private long starttime = 0; + private long endtime = 0; + private long size = 0; + + public boolean overlapsRange(long starttime, long endtime) { + return (starttime <= this.endtime && endtime >= this.starttime); + } + + public long size() { return size; } + public long getStartTime() { return starttime; } + public long getEndTime() { return endtime; } + public LogSkipList getSkipList() { return skiplist; } + + public static boolean isTransactionFile(String file) throws IOException { + RandomAccessFileReader reader = new RandomAccessFileReader(new File(file)); + BinaryInputArchive logStream = new BinaryInputArchive(reader); + FileHeader fhdr = new FileHeader(); + fhdr.deserialize(logStream, "fileheader"); + reader.close(); + + return fhdr.getMagic() == FileTxnLog.TXNLOG_MAGIC; + } + + private class TxnLogSourceIterator implements LogIterator { + private LogEntry next = null; + private long starttime = 0; + private long endtime = 0; + private TxnLogSource src = null; + private RandomAccessFileReader reader = null; + private BinaryInputArchive logStream = null; + private long skippedAtStart = 0; + private FilterOp filter = null; + + public TxnLogSourceIterator(TxnLogSource src, long starttime, long endtime) throws IllegalArgumentException, FilterException { + this(src,starttime,endtime,null); + } + + public TxnLogSourceIterator(TxnLogSource src, long starttime, long endtime, FilterOp filter) throws IllegalArgumentException, FilterException { + try { + this.src = src; + this.starttime = starttime; + this.endtime = endtime; + reader = new RandomAccessFileReader(new File(src.file)); + logStream = new BinaryInputArchive(reader); + FileHeader fhdr = new FileHeader(); + fhdr.deserialize(logStream, "fileheader"); + } catch (Exception e) { + throw new IllegalArgumentException("Cannot open transaction log ("+src.file+") :" + e); + } + + LogSkipList.Mark start = src.getSkipList().findMarkBefore(starttime); + try { + reader.seek(start.getBytes()); + skippedAtStart = start.getEntriesSkipped(); + } catch (IOException ioe) { + // if we can't skip, we should just read from the start + } + + this.filter = filter; + + LogEntry e; + while ((e = readNextEntry()) != null && e.getTimestamp() < endtime) { + if (e.getTimestamp() >= starttime && (filter == null || filter.matches(e)) ) { + next = e; + return; + } + skippedAtStart++; + } + + + } + + public long size() throws IOException { + if (this.endtime >= src.getEndTime()) { + return src.size() - skippedAtStart; + } + + long pos = reader.getPosition(); + LogEntry e; + + LogSkipList.Mark lastseg = src.getSkipList().findMarkBefore(this.endtime); + reader.seek(lastseg.getBytes()); + // number of entries skipped to get to the end of the iterator, less the number skipped to get to the start + long count = lastseg.getEntriesSkipped() - skippedAtStart; + + while ((e = readNextEntry()) != null) { + if (e.getTimestamp() > this.endtime) { + break; + } + count++; + } + reader.seek(pos);; + + return count; + } + + private LogEntry readNextEntry() { + LogEntry e = null; + try { + long crcValue; + byte[] bytes; + try { + crcValue = logStream.readLong("crcvalue"); + + bytes = logStream.readBuffer("txnEntry"); + } catch (EOFException ex) { + return null; + } + + if (bytes.length == 0) { + return null; + } + Checksum crc = new Adler32(); + crc.update(bytes, 0, bytes.length); + if (crcValue != crc.getValue()) { + throw new IOException("CRC doesn't match " + crcValue + + " vs " + crc.getValue()); + } + InputArchive iab = BinaryInputArchive.getArchive(new ByteArrayInputStream(bytes)); + TxnHeader hdr = new TxnHeader(); + Record r = SerializeUtils.deserializeTxn(iab, hdr); + + switch (hdr.getType()) { + case OpCode.createSession: { + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "createSession"); + } + break; + case OpCode.closeSession: { + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "closeSession"); + } + break; + case OpCode.create: + if (r != null) { + CreateTxn create = (CreateTxn)r; + String path = create.getPath(); + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "create", path); + } + break; + case OpCode.setData: + if (r != null) { + SetDataTxn set = (SetDataTxn)r; + String path = set.getPath(); + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "setData", path); + } + break; + case OpCode.setACL: + if (r != null) { + SetACLTxn setacl = (SetACLTxn)r; + String path = setacl.getPath(); + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "setACL", path); + } + break; + case OpCode.error: + if (r != null) { + ErrorTxn error = (ErrorTxn)r; + + e = new TransactionEntry(hdr.getTime(), hdr.getClientId(), hdr.getCxid(), hdr.getZxid(), "error", "Error: " + error.getErr()); + } + break; + default: + LOG.info("Unknown op: " + hdr.getType()); + break; + } + + if (logStream.readByte("EOR") != 'B') { + throw new EOFException("Last transaction was partial."); + } + } catch (Exception ex) { + LOG.error("Error reading transaction from (" + src.file + ") :" + e); + return null; + } + return e; + } + + public boolean hasNext() { + return next != null; + } + + public LogEntry next() throws NoSuchElementException { + LogEntry ret = next; + LogEntry e = readNextEntry(); + + if (filter != null) { + try { + while (e != null && !filter.matches(e)) { + e = readNextEntry(); + } + } catch (FilterException fe) { + throw new NoSuchElementException(fe.toString()); + } + } + if (e != null && e.getTimestamp() < endtime) { + next = e; + } else { + next = null; + } + return ret; + } + + public void remove() throws UnsupportedOperationException { + throw new UnsupportedOperationException("remove not supported for Txn logs"); + } + + public void close() throws IOException { + reader.close(); + } + } + + public LogIterator iterator(long starttime, long endtime) throws IllegalArgumentException { + try { + return iterator(starttime, endtime, null); + } catch (FilterException fe) { + assert(false); // should never ever happen + return null; + } + } + + public LogIterator iterator(long starttime, long endtime, FilterOp filter) throws IllegalArgumentException, FilterException { + // sanitise start and end times + if (endtime < starttime) { + throw new IllegalArgumentException("End time (" + endtime + ") must be greater or equal to starttime (" + starttime + ")"); + } + + return new TxnLogSourceIterator(this, starttime, endtime, filter); + } + + public LogIterator iterator() throws IllegalArgumentException { + return iterator(starttime, endtime+1); + } + + public TxnLogSource(String file) throws IOException { + this.file = file; + + skiplist = new LogSkipList(); + + RandomAccessFileReader reader = new RandomAccessFileReader(new File(file)); + try { + BinaryInputArchive logStream = new BinaryInputArchive(reader); + FileHeader fhdr = new FileHeader(); + fhdr.deserialize(logStream, "fileheader"); + + byte[] bytes = null; + while (true) { + long lastFp = reader.getPosition(); + + long crcValue; + + try { + crcValue = logStream.readLong("crcvalue"); + bytes = logStream.readBuffer("txnEntry"); + } catch (EOFException e) { + break; + } + + if (bytes.length == 0) { + break; + } + Checksum crc = new Adler32(); + crc.update(bytes, 0, bytes.length); + if (crcValue != crc.getValue()) { + throw new IOException("CRC doesn't match " + crcValue + + " vs " + crc.getValue()); + } + if (logStream.readByte("EOR") != 'B') { + throw new EOFException("Last transaction was partial."); + } + InputArchive iab = BinaryInputArchive.getArchive(new ByteArrayInputStream(bytes)); + TxnHeader hdr = new TxnHeader(); + Record r = SerializeUtils.deserializeTxn(iab, hdr); + + if (starttime == 0) { + starttime = hdr.getTime(); + } + endtime = hdr.getTime(); + + if (size % skipN == 0) { + skiplist.addMark(hdr.getTime(), lastFp, size); + } + size++; + } + if (bytes == null) { + throw new IOException("Nothing read from ("+file+")"); + } + } finally { + reader.close(); + } + } + + public String toString() { + return "TxnLogSource(file=" + file + ", size=" + size + ", start=" + starttime + ", end=" + endtime +")"; + } + + public static void main(String[] args) throws IOException, FilterException { + TxnLogSource s = new TxnLogSource(args[0]); + System.out.println(s); + + LogIterator iter; + + if (args.length == 3) { + long starttime = Long.valueOf(args[1]); + long endtime = Long.valueOf(args[2]); + FilterOp fo = new FilterParser("(or (and (> zxid 0x2f0bd6f5e0) (< zxid 0x2f0bd6f5e9)) (= operation \"error\"))").parse(); + System.out.println("fo: " + fo); + iter = s.iterator(starttime, endtime, fo); + } else { + iter = s.iterator(); + } + System.out.println(iter); + while (iter.hasNext()) { + System.out.println(iter.next()); + } + iter.close(); + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/AndOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/AndOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/AndOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/AndOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,33 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class AndOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + for (FilterOp f : subOps) { + if (!f.matches(entry)) { + return false; + } + } + return true; + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/Arg.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/Arg.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/Arg.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/Arg.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,36 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.FilterOp.*; + +public class Arg { + private ArgType type; + protected T value; + + protected Arg(ArgType type) { + this.type = type; + } + + public ArgType getType() { return type; } + public T getValue() { return value; } + + public String toString() { + return "[" + type + ":" + value + "]"; + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/EqualsOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/EqualsOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/EqualsOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/EqualsOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,44 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class EqualsOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + + Object last = null; + for (Arg a : args) { + Object v = a.getValue(); + if (a.getType() == FilterOp.ArgType.SYMBOL) { + String key = (String)a.getValue(); + v = entry.getAttribute(key); + } + + if (last != null + && !last.equals(v)) { + return false; + } + last = v; + } + + return true; + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/GreaterThanOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/GreaterThanOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/GreaterThanOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/GreaterThanOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,70 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class GreaterThanOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + Arg first = args.get(0); + + if (first != null) { + FilterOp.ArgType type = first.getType(); + if (type == FilterOp.ArgType.SYMBOL) { + String key = (String)first.getValue(); + Object v = entry.getAttribute(key); + if (v instanceof String) { + type = FilterOp.ArgType.STRING; + } else if (v instanceof Double || v instanceof Long || v instanceof Integer || v instanceof Short) { + type = FilterOp.ArgType.NUMBER; + } else { + throw new FilterException("LessThanOp: Invalid argument, first argument resolves to neither a String nor a Number"); + } + } + + Object last = null; + for (Arg a : args) { + Object v = a.getValue(); + if (a.getType() == FilterOp.ArgType.SYMBOL) { + String key = (String)a.getValue(); + v = entry.getAttribute(key); + } + + if (last != null) { + if (type == FilterOp.ArgType.STRING) { + if (((String)last).compareTo((String)v) <= 0) { + return false; + } + } else if (type == FilterOp.ArgType.NUMBER) { + // System.out.println("last[" + ((Number)last).longValue() + "] v["+ ((Number)v).longValue() + "]"); + if (((Number)last).longValue() <= ((Number)v).longValue()) { + return false; + } + } + } + last = v; + } + return true; + } else { + return true; + } + } + +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/LessThanOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/LessThanOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/LessThanOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/LessThanOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,69 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class LessThanOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + Arg first = args.get(0); + + if (first != null) { + FilterOp.ArgType type = first.getType(); + if (type == FilterOp.ArgType.SYMBOL) { + String key = (String)first.getValue(); + Object v = entry.getAttribute(key); + if (v instanceof String) { + type = FilterOp.ArgType.STRING; + } else if (v instanceof Double || v instanceof Long || v instanceof Integer || v instanceof Short) { + type = FilterOp.ArgType.NUMBER; + } else { + throw new FilterException("LessThanOp: Invalid argument, first argument resolves to neither a String nor a Number"); + } + } + + Object last = null; + for (Arg a : args) { + Object v = a.getValue(); + if (a.getType() == FilterOp.ArgType.SYMBOL) { + String key = (String)a.getValue(); + v = entry.getAttribute(key); + } + + if (last != null) { + if (type == FilterOp.ArgType.STRING) { + if (((String)last).compareTo((String)v) >= 0) { + return false; + } + } else if (type == FilterOp.ArgType.NUMBER) { + if (((Number)last).doubleValue() >= ((Number)v).doubleValue()) { + return false; + } + } + } + last = v; + } + return true; + } else { + return true; + } + } + +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NotOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NotOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NotOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NotOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class NotOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + if (subOps.size() != 1) { + throw new FilterException("Not operation can only take one argument"); + } + return !subOps.get(0).matches(entry); + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NumberArg.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NumberArg.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NumberArg.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/NumberArg.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,28 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.FilterOp.*; + +public class NumberArg extends Arg { + public NumberArg(Long value) { + super(ArgType.NUMBER); + this.value = value; + } +}; + Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/OrOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/OrOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/OrOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/OrOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,33 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class OrOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + for (FilterOp f : subOps) { + if (f.matches(entry)) { + return true; + } + } + return false; + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/StringArg.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/StringArg.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/StringArg.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/StringArg.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,28 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.FilterOp.*; + +public class StringArg extends Arg { + public StringArg(String value) { + super(ArgType.STRING); + this.value = value; + } +}; + Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/SymbolArg.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/SymbolArg.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/SymbolArg.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/SymbolArg.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,27 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.FilterOp.*; + +public class SymbolArg extends Arg { + public SymbolArg(String value) { + super(ArgType.SYMBOL); + this.value = value; + } +}; Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/XorOp.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/XorOp.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/XorOp.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/filterops/XorOp.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,40 @@ +/** + * 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.zookeeper.graph.filterops; + +import org.apache.zookeeper.graph.LogEntry; +import org.apache.zookeeper.graph.FilterOp; +import org.apache.zookeeper.graph.FilterException; + +public class XorOp extends FilterOp { + public boolean matches(LogEntry entry) throws FilterException { + int count = 0; + for (FilterOp f : subOps) { + if (f.matches(entry)) { + count++; + if (count > 1) { + return false; + } + } + } + if (count == 1) { + return true; + } + return false; + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/FileLoader.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/FileLoader.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/FileLoader.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/FileLoader.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,60 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.File; +import java.io.IOException; +import java.io.FileNotFoundException; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import org.apache.zookeeper.graph.*; + +public class FileLoader extends JsonServlet +{ + private MergedLogSource source = null; + + public FileLoader(MergedLogSource src) throws Exception { + source = src; + } + + String handleRequest(JsonRequest request) throws Exception + { + String output = ""; + + String file = request.getString("path", "/"); + JSONObject o = new JSONObject(); + try { + this.source.addSource(file); + o.put("status", "OK"); + + } catch (Exception e) { + o.put("status", "ERR"); + o.put("error", e.toString()); + } + + return JSONValue.toJSONString(o); + } +} \ No newline at end of file Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Fs.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Fs.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Fs.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Fs.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,69 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.File; +import java.io.IOException; +import java.io.FileNotFoundException; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; +import java.util.Arrays; +import java.util.Comparator; + +public class Fs extends JsonServlet +{ + String handleRequest(JsonRequest request) throws Exception + { + String output = ""; + JSONArray filelist = new JSONArray(); + + File base = new File(request.getString("path", "/")); + if (!base.exists() || !base.isDirectory()) { + throw new FileNotFoundException("Couldn't find [" + request + "]"); + } + File[] files = base.listFiles(); + Arrays.sort(files, new Comparator() { + public int compare(File o1, File o2) { + if (o1.isDirectory() != o2.isDirectory()) { + if (o1.isDirectory()) { + return -1; + } else { + return 1; + } + } + return o1.getName().compareToIgnoreCase(o2.getName()); + } + }); + + for (File f : files) { + JSONObject o = new JSONObject(); + o.put("file", f.getName()); + o.put("type", f.isDirectory() ? "D" : "F"); + o.put("path", f.getCanonicalPath()); + filelist.add(o); + } + return JSONValue.toJSONString(filelist); + } +} \ No newline at end of file Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/GraphData.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/GraphData.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/GraphData.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/GraphData.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,84 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.File; +import java.io.IOException; +import java.io.FileNotFoundException; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import java.util.regex.Pattern; +import java.util.regex.Matcher; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import org.apache.zookeeper.graph.*; +import org.apache.log4j.Logger; + +public class GraphData extends JsonServlet +{ + private static final Logger LOG = Logger.getLogger(GraphData.class); + private static final int DEFAULT_PERIOD = 1000; + + private LogSource source = null; + + public GraphData(LogSource src) throws Exception { + this.source = src; + } + + String handleRequest(JsonRequest request) throws Exception { + + + long starttime = 0; + long endtime = 0; + long period = 0; + FilterOp fo = null; + + starttime = request.getNumber("start", 0); + endtime = request.getNumber("end", 0); + period = request.getNumber("period", 0); + String filterstr = request.getString("filter", ""); + + if (filterstr.length() > 0) { + fo = new FilterParser(filterstr).parse(); + } + + if (starttime == 0) { starttime = source.getStartTime(); } + if (endtime == 0) { + if (period > 0) { + endtime = starttime + period; + } else { + endtime = starttime + DEFAULT_PERIOD; + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("handle(start= " + starttime + ", end=" + endtime + ", period=" + period + ")"); + } + + LogIterator iterator = (fo != null) ? + source.iterator(starttime, endtime, fo) : source.iterator(starttime, endtime); + return new JsonGenerator(iterator).toString(); + } +} \ No newline at end of file Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/JsonServlet.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/JsonServlet.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/JsonServlet.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/JsonServlet.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,85 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.IOException; + +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import java.util.Map; + +abstract public class JsonServlet extends HttpServlet { + abstract String handleRequest(JsonRequest request) throws Exception; + + protected class JsonRequest { + private Map map; + + public JsonRequest(ServletRequest request) { + map = request.getParameterMap(); + } + + public long getNumber(String name, long defaultnum) { + String[] vals = (String[])map.get(name); + if (vals == null || vals.length == 0) { + return defaultnum; + } + + try { + return Long.valueOf(vals[0]); + } catch (NumberFormatException e) { + return defaultnum; + } + } + + public String getString(String name, String defaultstr) { + String[] vals = (String[])map.get(name); + if (vals == null || vals.length == 0) { + return defaultstr; + } else { + return vals[0]; + } + } + } + + protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException + { + response.setContentType("text/plain;charset=utf-8"); + response.setStatus(HttpServletResponse.SC_OK); + + try { + String req = request.getRequestURI().substring(request.getServletPath().length()); + + response.getWriter().println(handleRequest(new JsonRequest(request))); + } catch (Exception e) { + JSONObject o = new JSONObject(); + o.put("error", e.toString()); + response.getWriter().println(JSONValue.toJSONString(o)); + } catch (java.lang.OutOfMemoryError oom) { + JSONObject o = new JSONObject(); + o.put("error", "Out of memory. Perhaps you've requested too many logs. Try narrowing you're filter criteria."); + response.getWriter().println(JSONValue.toJSONString(o)); + } + } +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/NumEvents.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,86 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.File; +import java.io.IOException; +import java.io.FileNotFoundException; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import java.util.regex.Pattern; +import java.util.regex.Matcher; + +import org.apache.zookeeper.graph.*; +import org.apache.log4j.Logger; + + +public class NumEvents extends JsonServlet +{ + private static final Logger LOG = Logger.getLogger(NumEvents.class); + private static final int DEFAULT_PERIOD = 1000; + + private LogSource source = null; + + public NumEvents(LogSource src) throws Exception { + this.source = src; + } + + String handleRequest(JsonRequest request) throws Exception { + String output = ""; + + long starttime = 0; + long endtime = 0; + long period = 0; + + starttime = request.getNumber("start", 0); + endtime = request.getNumber("end", 0); + period = request.getNumber("period", 0); + + if (starttime == 0) { starttime = source.getStartTime(); } + if (endtime == 0) { + if (period > 0) { + endtime = starttime + period; + } else { + endtime = source.getEndTime(); + } + } + + LogIterator iter = source.iterator(starttime, endtime); + JSONObject data = new JSONObject(); + data.put("startTime", starttime); + data.put("endTime", endtime); + long size = 0; + + size = iter.size(); + + data.put("numEntries", size); + if (LOG.isDebugEnabled()) { + LOG.debug("handle(start= " + starttime + ", end=" + endtime + ", numEntries=" + size +")"); + } + return JSONValue.toJSONString(data); + } +} + Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/StaticContent.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,50 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.BufferedReader; + +import java.io.IOException; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +public class StaticContent extends HttpServlet { + protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException + { + String path = request.getRequestURI().substring(request.getServletPath().length()); + + InputStream resource = ClassLoader.getSystemResourceAsStream("org/apache/zookeeper/graph/resources" + path); + if (resource == null) { + response.getWriter().println(path + " not found!"); + response.setStatus(HttpServletResponse.SC_NOT_FOUND); + return; + } + + while (resource.available() > 0) { + response.getWriter().write(resource.read()); + } + // response.setContentType("text/plain;charset=utf-8"); + response.setStatus(HttpServletResponse.SC_OK); + } + +} Added: hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Throughput.java URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Throughput.java?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Throughput.java (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/src/java/org/apache/zookeeper/graph/servlets/Throughput.java Wed Jun 9 15:26:22 2010 @@ -0,0 +1,124 @@ +/** + * 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.zookeeper.graph.servlets; + +import java.io.IOException; +import java.io.BufferedOutputStream; +import java.io.FileOutputStream; +import java.io.DataOutputStream; +import java.io.PrintStream; + +import java.util.HashSet; +import java.util.LinkedHashMap; + +import org.apache.zookeeper.graph.*; +import org.apache.log4j.Logger; + +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + + +public class Throughput extends JsonServlet +{ + private static final int MS_PER_SEC = 1000; + private static final int MS_PER_MIN = MS_PER_SEC*60; + private static final int MS_PER_HOUR = MS_PER_MIN*60; + + private LogSource source = null; + + public Throughput(LogSource src) throws Exception { + this.source = src; + } + + public String handleRequest(JsonRequest request) throws Exception { + long starttime = 0; + long endtime = 0; + long period = 0; + long scale = 0; + + starttime = request.getNumber("start", 0); + endtime = request.getNumber("end", 0); + period = request.getNumber("period", 0); + + + if (starttime == 0) { starttime = source.getStartTime(); } + if (endtime == 0) { + if (period > 0) { + endtime = starttime + period; + } else { + endtime = source.getEndTime(); + } + } + + String scalestr = request.getString("scale", "minutes"); + if (scalestr.equals("seconds")) { + scale = MS_PER_SEC; + } else if (scalestr.equals("hours")) { + scale = MS_PER_HOUR; + } else { + scale = MS_PER_MIN; + } + + LogIterator iter = source.iterator(starttime, endtime); + + long current = 0; + long currentms = 0; + HashSet zxids_ms = new HashSet(); + long zxidcount = 0; + + JSONArray events = new JSONArray(); + while (iter.hasNext()) { + LogEntry e = iter.next(); + if (e.getType() != LogEntry.Type.TXN) { + continue; + } + + TransactionEntry cxn = (TransactionEntry)e; + + long ms = cxn.getTimestamp(); + long inscale = ms/scale; + + if (currentms != ms && currentms != 0) { + zxidcount += zxids_ms.size(); + zxids_ms.clear(); + } + + if (inscale != current && current != 0) { + JSONObject o = new JSONObject(); + o.put("time", current*scale); + o.put("count", zxidcount); + events.add(o); + zxidcount = 0; + } + current = inscale; + currentms = ms; + + zxids_ms.add(cxn.getZxid()); + } + JSONObject o = new JSONObject(); + o.put("time", current*scale); + o.put("count", zxidcount); + events.add(o); + + iter.close(); + + return JSONValue.toJSONString(events); + } + +}; Added: hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/log4j.properties URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/log4j.properties?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/log4j.properties (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/log4j.properties Wed Jun 9 15:26:22 2010 @@ -0,0 +1,11 @@ +log4j.rootLogger=TRACE, CONSOLE + +# Print the date in ISO 8601 format +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.Threshold=TRACE +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} - %-5p [%t:%C{1}@%L] - %m%n + +log4j.logger.org.apache.zookeeper.graph.LogSkipList=off +log4j.logger.org.apache.zookeeper.graph.RandomAccessFileReader=off +#log4j.logger.org.apache.zookeeper.graph.Log4JSource=off \ No newline at end of file Added: hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/date.format.js URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/date.format.js?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/date.format.js (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/date.format.js Wed Jun 9 15:26:22 2010 @@ -0,0 +1,126 @@ +/* + * Date Format 1.2.3 + * (c) 2007-2009 Steven Levithan + * MIT license + * + * Includes enhancements by Scott Trenda + * and Kris Kowal + * + * Accepts a date, a mask, or a date and a mask. + * Returns a formatted version of the given date. + * The date defaults to the current date/time. + * The mask defaults to dateFormat.masks.default. + */ + +var dateFormat = function () { + var token = /d{1,4}|m{1,4}|yy(?:yy)?|([HhMsTt])\1?|[LloSZ]|"[^"]*"|'[^']*'/g, + timezone = /\b(?:[PMCEA][SDP]T|(?:Pacific|Mountain|Central|Eastern|Atlantic) (?:Standard|Daylight|Prevailing) Time|(?:GMT|UTC)(?:[-+]\d{4})?)\b/g, + timezoneClip = /[^-+\dA-Z]/g, + pad = function (val, len) { + val = String(val); + len = len || 2; + while (val.length < len) val = "0" + val; + return val; + }; + + // Regexes and supporting functions are cached through closure + return function (date, mask, utc) { + var dF = dateFormat; + + // You can't provide utc if you skip other args (use the "UTC:" mask prefix) + if (arguments.length == 1 && Object.prototype.toString.call(date) == "[object String]" && !/\d/.test(date)) { + mask = date; + date = undefined; + } + + // Passing date through Date applies Date.parse, if necessary + date = date ? new Date(date) : new Date; + if (isNaN(date)) throw SyntaxError("invalid date"); + + mask = String(dF.masks[mask] || mask || dF.masks["default"]); + + // Allow setting the utc argument via the mask + if (mask.slice(0, 4) == "UTC:") { + mask = mask.slice(4); + utc = true; + } + + var _ = utc ? "getUTC" : "get", + d = date[_ + "Date"](), + D = date[_ + "Day"](), + m = date[_ + "Month"](), + y = date[_ + "FullYear"](), + H = date[_ + "Hours"](), + M = date[_ + "Minutes"](), + s = date[_ + "Seconds"](), + L = date[_ + "Milliseconds"](), + o = utc ? 0 : date.getTimezoneOffset(), + flags = { + d: d, + dd: pad(d), + ddd: dF.i18n.dayNames[D], + dddd: dF.i18n.dayNames[D + 7], + m: m + 1, + mm: pad(m + 1), + mmm: dF.i18n.monthNames[m], + mmmm: dF.i18n.monthNames[m + 12], + yy: String(y).slice(2), + yyyy: y, + h: H % 12 || 12, + hh: pad(H % 12 || 12), + H: H, + HH: pad(H), + M: M, + MM: pad(M), + s: s, + ss: pad(s), + l: pad(L, 3), + L: pad(L > 99 ? Math.round(L / 10) : L), + t: H < 12 ? "a" : "p", + tt: H < 12 ? "am" : "pm", + T: H < 12 ? "A" : "P", + TT: H < 12 ? "AM" : "PM", + Z: utc ? "UTC" : (String(date).match(timezone) || [""]).pop().replace(timezoneClip, ""), + o: (o > 0 ? "-" : "+") + pad(Math.floor(Math.abs(o) / 60) * 100 + Math.abs(o) % 60, 4), + S: ["th", "st", "nd", "rd"][d % 10 > 3 ? 0 : (d % 100 - d % 10 != 10) * d % 10] + }; + + return mask.replace(token, function ($0) { + return $0 in flags ? flags[$0] : $0.slice(1, $0.length - 1); + }); + }; +}(); + +// Some common format strings +dateFormat.masks = { + "default": "ddd mmm dd yyyy HH:MM:ss", + shortDate: "m/d/yy", + mediumDate: "mmm d, yyyy", + longDate: "mmmm d, yyyy", + fullDate: "dddd, mmmm d, yyyy", + shortTime: "h:MM TT", + mediumTime: "h:MM:ss TT", + longTime: "h:MM:ss TT Z", + isoDate: "yyyy-mm-dd", + isoTime: "HH:MM:ss", + isoDateTime: "yyyy-mm-dd'T'HH:MM:ss", + isoUtcDateTime: "UTC:yyyy-mm-dd'T'HH:MM:ss'Z'" +}; + +// Internationalization strings +dateFormat.i18n = { + dayNames: [ + "Sun", "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", + "Sunday", "Monday", "Tuesday", "Wednesday", "Thursday", "Friday", "Saturday" + ], + monthNames: [ + "Jan", "Feb", "Mar", "Apr", "May", "Jun", "Jul", "Aug", "Sep", "Oct", "Nov", "Dec", + "January", "February", "March", "April", "May", "June", "July", "August", "September", "October", "November", "December" + ] +}; + +// For convenience... +Date.prototype.format = function (mask, utc) { + return dateFormat(this, mask, utc); +}; + Added: hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.bar.js URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.bar.js?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.bar.js (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.bar.js Wed Jun 9 15:26:22 2010 @@ -0,0 +1,385 @@ +/* + * g.Raphael 0.4 - Charting library, based on Raphaël + * + * Copyright (c) 2009 Dmitry Baranovskiy (http://g.raphaeljs.com) + * Licensed under the MIT (http://www.opensource.org/licenses/mit-license.php) license. + */ +Raphael.fn.g.barchart = function (x, y, width, height, values, opts) { + opts = opts || {}; + var type = {round: "round", sharp: "sharp", soft: "soft"}[opts.type] || "square", + gutter = parseFloat(opts.gutter || "20%"), + chart = this.set(), + bars = this.set(), + covers = this.set(), + covers2 = this.set(), + total = Math.max.apply(Math, values), + stacktotal = [], + paper = this, + multi = 0, + colors = opts.colors || this.g.colors, + len = values.length; + if (this.raphael.is(values[0], "array")) { + total = []; + multi = len; + len = 0; + for (var i = values.length; i--;) { + bars.push(this.set()); + total.push(Math.max.apply(Math, values[i])); + len = Math.max(len, values[i].length); + } + if (opts.stacked) { + for (var i = len; i--;) { + var tot = 0; + for (var j = values.length; j--;) { + tot +=+ values[j][i] || 0; + } + stacktotal.push(tot); + } + } + for (var i = values.length; i--;) { + if (values[i].length < len) { + for (var j = len; j--;) { + values[i].push(0); + } + } + } + total = Math.max.apply(Math, opts.stacked ? stacktotal : total); + } + + total = (opts.to) || total; + var barwidth = width / (len * (100 + gutter) + gutter) * 100, + barhgutter = barwidth * gutter / 100, + barvgutter = opts.vgutter == null ? 20 : opts.vgutter, + stack = [], + X = x + barhgutter, + Y = (height - 2 * barvgutter) / total; + if (!opts.stretch) { + barhgutter = Math.round(barhgutter); + barwidth = Math.floor(barwidth); + } + !opts.stacked && (barwidth /= multi || 1); + for (var i = 0; i < len; i++) { + stack = []; + for (var j = 0; j < (multi || 1); j++) { + var h = Math.round((multi ? values[j][i] : values[i]) * Y), + top = y + height - barvgutter - h, + bar = this.g.finger(Math.round(X + barwidth / 2), top + h, barwidth, h, true, type).attr({stroke: colors[multi ? j : i], fill: colors[multi ? j : i]}); + if (multi) { + bars[j].push(bar); + } else { + bars.push(bar); + } + bar.y = top; + bar.x = Math.round(X + barwidth / 2); + bar.w = barwidth; + bar.h = h; + bar.value = multi ? values[j][i] : values[i]; + if (!opts.stacked) { + X += barwidth; + } else { + stack.push(bar); + } + } + if (opts.stacked) { + var cvr; + covers2.push(cvr = this.rect(stack[0].x - stack[0].w / 2, y, barwidth, height).attr(this.g.shim)); + cvr.bars = this.set(); + var size = 0; + for (var s = stack.length; s--;) { + stack[s].toFront(); + } + for (var s = 0, ss = stack.length; s < ss; s++) { + var bar = stack[s], + cover, + h = (size + bar.value) * Y, + path = this.g.finger(bar.x, y + height - barvgutter - !!size * .5, barwidth, h, true, type, 1); + cvr.bars.push(bar); + size && bar.attr({path: path}); + bar.h = h; + bar.y = y + height - barvgutter - !!size * .5 - h; + covers.push(cover = this.rect(bar.x - bar.w / 2, bar.y, barwidth, bar.value * Y).attr(this.g.shim)); + cover.bar = bar; + cover.value = bar.value; + size += bar.value; + } + X += barwidth; + } + X += barhgutter; + } + covers2.toFront(); + X = x + barhgutter; + if (!opts.stacked) { + for (var i = 0; i < len; i++) { + for (var j = 0; j < (multi || 1); j++) { + var cover; + covers.push(cover = this.rect(Math.round(X), y + barvgutter, barwidth, height - barvgutter).attr(this.g.shim)); + cover.bar = multi ? bars[j][i] : bars[i]; + cover.value = cover.bar.value; + X += barwidth; + } + X += barhgutter; + } + } + chart.label = function (labels, isBottom) { + labels = labels || []; + this.labels = paper.set(); + var L, l = -Infinity; + if (opts.stacked) { + for (var i = 0; i < len; i++) { + var tot = 0; + for (var j = 0; j < (multi || 1); j++) { + tot += multi ? values[j][i] : values[i]; + if (j == multi - 1) { + var label = paper.g.labelise(labels[i], tot, total); + L = paper.g.text(bars[i * (multi || 1) + j].x, y + height - barvgutter / 2, label).insertBefore(covers[i * (multi || 1) + j]); + var bb = L.getBBox(); + if (bb.x - 7 < l) { + L.remove(); + } else { + this.labels.push(L); + l = bb.x + bb.width; + } + } + } + } + } else { + for (var i = 0; i < len; i++) { + for (var j = 0; j < (multi || 1); j++) { + var label = paper.g.labelise(multi ? labels[j] && labels[j][i] : labels[i], multi ? values[j][i] : values[i], total); + L = paper.g.text(bars[i * (multi || 1) + j].x, isBottom ? y + height - barvgutter / 2 : bars[i * (multi || 1) + j].y - 10, label).insertBefore(covers[i * (multi || 1) + j]); + var bb = L.getBBox(); + if (bb.x - 7 < l) { + L.remove(); + } else { + this.labels.push(L); + l = bb.x + bb.width; + } + } + } + } + return this; + }; + chart.hover = function (fin, fout) { + covers2.hide(); + covers.show(); + covers.mouseover(fin).mouseout(fout); + return this; + }; + chart.hoverColumn = function (fin, fout) { + covers.hide(); + covers2.show(); + fout = fout || function () {}; + covers2.mouseover(fin).mouseout(fout); + return this; + }; + chart.click = function (f) { + covers2.hide(); + covers.show(); + covers.click(f); + return this; + }; + chart.each = function (f) { + if (!Raphael.is(f, "function")) { + return this; + } + for (var i = covers.length; i--;) { + f.call(covers[i]); + } + return this; + }; + chart.eachColumn = function (f) { + if (!Raphael.is(f, "function")) { + return this; + } + for (var i = covers2.length; i--;) { + f.call(covers2[i]); + } + return this; + }; + chart.clickColumn = function (f) { + covers.hide(); + covers2.show(); + covers2.click(f); + return this; + }; + chart.push(bars, covers, covers2); + chart.bars = bars; + chart.covers = covers; + return chart; +}; +Raphael.fn.g.hbarchart = function (x, y, width, height, values, opts) { + opts = opts || {}; + var type = {round: "round", sharp: "sharp", soft: "soft"}[opts.type] || "square", + gutter = parseFloat(opts.gutter || "20%"), + chart = this.set(), + bars = this.set(), + covers = this.set(), + covers2 = this.set(), + total = Math.max.apply(Math, values), + stacktotal = [], + paper = this, + multi = 0, + colors = opts.colors || this.g.colors, + len = values.length; + if (this.raphael.is(values[0], "array")) { + total = []; + multi = len; + len = 0; + for (var i = values.length; i--;) { + bars.push(this.set()); + total.push(Math.max.apply(Math, values[i])); + len = Math.max(len, values[i].length); + } + if (opts.stacked) { + for (var i = len; i--;) { + var tot = 0; + for (var j = values.length; j--;) { + tot +=+ values[j][i] || 0; + } + stacktotal.push(tot); + } + } + for (var i = values.length; i--;) { + if (values[i].length < len) { + for (var j = len; j--;) { + values[i].push(0); + } + } + } + total = Math.max.apply(Math, opts.stacked ? stacktotal : total); + } + + total = (opts.to) || total; + var barheight = Math.floor(height / (len * (100 + gutter) + gutter) * 100), + bargutter = Math.floor(barheight * gutter / 100), + stack = [], + Y = y + bargutter, + X = (width - 1) / total; + !opts.stacked && (barheight /= multi || 1); + for (var i = 0; i < len; i++) { + stack = []; + for (var j = 0; j < (multi || 1); j++) { + var val = multi ? values[j][i] : values[i], + bar = this.g.finger(x, Y + barheight / 2, Math.round(val * X), barheight - 1, false, type).attr({stroke: colors[multi ? j : i], fill: colors[multi ? j : i]}); + if (multi) { + bars[j].push(bar); + } else { + bars.push(bar); + } + bar.x = x + Math.round(val * X); + bar.y = Y + barheight / 2; + bar.w = Math.round(val * X); + bar.h = barheight; + bar.value = +val; + if (!opts.stacked) { + Y += barheight; + } else { + stack.push(bar); + } + } + if (opts.stacked) { + var cvr = this.rect(x, stack[0].y - stack[0].h / 2, width, barheight).attr(this.g.shim); + covers2.push(cvr); + cvr.bars = this.set(); + var size = 0; + for (var s = stack.length; s--;) { + stack[s].toFront(); + } + for (var s = 0, ss = stack.length; s < ss; s++) { + var bar = stack[s], + cover, + val = Math.round((size + bar.value) * X), + path = this.g.finger(x, bar.y, val, barheight - 1, false, type, 1); + cvr.bars.push(bar); + size && bar.attr({path: path}); + bar.w = val; + bar.x = x + val; + covers.push(cover = this.rect(x + size * X, bar.y - bar.h / 2, bar.value * X, barheight).attr(this.g.shim)); + cover.bar = bar; + size += bar.value; + } + Y += barheight; + } + Y += bargutter; + } + covers2.toFront(); + Y = y + bargutter; + if (!opts.stacked) { + for (var i = 0; i < len; i++) { + for (var j = 0; j < multi; j++) { + var cover = this.rect(x, Y, width, barheight).attr(this.g.shim); + covers.push(cover); + cover.bar = bars[j][i]; + Y += barheight; + } + Y += bargutter; + } + } + chart.label = function (labels, isRight) { + labels = labels || []; + this.labels = paper.set(); + for (var i = 0; i < len; i++) { + for (var j = 0; j < multi; j++) { + var label = paper.g.labelise(multi ? labels[j] && labels[j][i] : labels[i], multi ? values[j][i] : values[i], total); + var X = isRight ? bars[i * (multi || 1) + j].x - barheight / 2 + 3 : x + 5, + A = isRight ? "end" : "start", + L; + this.labels.push(L = paper.g.text(X, bars[i * (multi || 1) + j].y, label).attr({"text-anchor": A}).insertBefore(covers[0])); + if (L.getBBox().x < x + 5) { + L.attr({x: x + 5, "text-anchor": "start"}); + } else { + bars[i * (multi || 1) + j].label = L; + } + } + } + return this; + }; + chart.hover = function (fin, fout) { + covers2.hide(); + covers.show(); + fout = fout || function () {}; + covers.mouseover(fin).mouseout(fout); + return this; + }; + chart.hoverColumn = function (fin, fout) { + covers.hide(); + covers2.show(); + fout = fout || function () {}; + covers2.mouseover(fin).mouseout(fout); + return this; + }; + chart.each = function (f) { + if (!Raphael.is(f, "function")) { + return this; + } + for (var i = covers.length; i--;) { + f.call(covers[i]); + } + return this; + }; + chart.eachColumn = function (f) { + if (!Raphael.is(f, "function")) { + return this; + } + for (var i = covers2.length; i--;) { + f.call(covers2[i]); + } + return this; + }; + chart.click = function (f) { + covers2.hide(); + covers.show(); + covers.click(f); + return this; + }; + chart.clickColumn = function (f) { + covers.hide(); + covers2.show(); + covers2.click(f); + return this; + }; + chart.push(bars, covers, covers2); + chart.bars = bars; + chart.covers = covers; + return chart; +}; Added: hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.dot.js URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.dot.js?rev=953041&view=auto ============================================================================== --- hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.dot.js (added) +++ hadoop/zookeeper/trunk/src/contrib/loggraph/web/org/apache/zookeeper/graph/resources/g.dot.js Wed Jun 9 15:26:22 2010 @@ -0,0 +1,110 @@ +/* + * g.Raphael 0.4 - Charting library, based on Raphaël + * + * Copyright (c) 2009 Dmitry Baranovskiy (http://g.raphaeljs.com) + * Licensed under the MIT (http://www.opensource.org/licenses/mit-license.php) license. + */ +Raphael.fn.g.dotchart = function (x, y, width, height, valuesx, valuesy, size, opts) { + function drawAxis(ax) { + +ax[0] && (ax[0] = paper.g.axis(x + gutter, y + gutter, width - 2 * gutter, minx, maxx, opts.axisxstep || Math.floor((width - 2 * gutter) / 20), 2, opts.axisxlabels || null, opts.axisxtype || "t")); + +ax[1] && (ax[1] = paper.g.axis(x + width - gutter, y + height - gutter, height - 2 * gutter, miny, maxy, opts.axisystep || Math.floor((height - 2 * gutter) / 20), 3, opts.axisylabels || null, opts.axisytype || "t")); + +ax[2] && (ax[2] = paper.g.axis(x + gutter, y + height - gutter + maxR, width - 2 * gutter, minx, maxx, opts.axisxstep || Math.floor((width - 2 * gutter) / 20), 0, opts.axisxlabels || null, opts.axisxtype || "t")); + +ax[3] && (ax[3] = paper.g.axis(x + gutter - maxR, y + height - gutter, height - 2 * gutter, miny, maxy, opts.axisystep || Math.floor((height - 2 * gutter) / 20), 1, opts.axisylabels || null, opts.axisytype || "t")); + } + opts = opts || {}; + var xdim = this.g.snapEnds(Math.min.apply(Math, valuesx), Math.max.apply(Math, valuesx), valuesx.length - 1), + minx = xdim.from, + maxx = xdim.to, + gutter = opts.gutter || 10, + ydim = this.g.snapEnds(Math.min.apply(Math, valuesy), Math.max.apply(Math, valuesy), valuesy.length - 1), + miny = ydim.from, + maxy = ydim.to, + len = Math.max(valuesx.length, valuesy.length, size.length), + symbol = this.g.markers[opts.symbol] || "disc", + res = this.set(), + series = this.set(), + max = opts.max || 100, + top = Math.max.apply(Math, size), + R = [], + paper = this, + k = Math.sqrt(top / Math.PI) * 2 / max; + + for (var i = 0; i < len; i++) { + R[i] = Math.min(Math.sqrt(size[i] / Math.PI) * 2 / k, max); + } + gutter = Math.max.apply(Math, R.concat(gutter)); + var axis = this.set(), + maxR = Math.max.apply(Math, R); + if (opts.axis) { + var ax = (opts.axis + "").split(/[,\s]+/); + drawAxis(ax); + var g = [], b = []; + for (var i = 0, ii = ax.length; i < ii; i++) { + var bb = ax[i].all ? ax[i].all.getBBox()[["height", "width"][i % 2]] : 0; + g[i] = bb + gutter; + b[i] = bb; + } + gutter = Math.max.apply(Math, g.concat(gutter)); + for (var i = 0, ii = ax.length; i < ii; i++) if (ax[i].all) { + ax[i].remove(); + ax[i] = 1; + } + drawAxis(ax); + for (var i = 0, ii = ax.length; i < ii; i++) if (ax[i].all) { + axis.push(ax[i].all); + } + res.axis = axis; + } + var kx = (width - gutter * 2) / ((maxx - minx) || 1), + ky = (height - gutter * 2) / ((maxy - miny) || 1); + for (var i = 0, ii = valuesy.length; i < ii; i++) { + var sym = this.raphael.is(symbol, "array") ? symbol[i] : symbol, + X = x + gutter + (valuesx[i] - minx) * kx, + Y = y + height - gutter - (valuesy[i] - miny) * ky; + sym && R[i] && series.push(this.g[sym](X, Y, R[i]).attr({fill: opts.heat ? this.g.colorValue(R[i], maxR) : Raphael.fn.g.colors[0], "fill-opacity": opts.opacity ? R[i] / max : 1, stroke: "none"})); + } + var covers = this.set(); + for (var i = 0, ii = valuesy.length; i < ii; i++) { + var X = x + gutter + (valuesx[i] - minx) * kx, + Y = y + height - gutter - (valuesy[i] - miny) * ky; + covers.push(this.circle(X, Y, maxR).attr(this.g.shim)); + opts.href && opts.href[i] && covers[i].attr({href: opts.href[i]}); + covers[i].r = +R[i].toFixed(3); + covers[i].x = +X.toFixed(3); + covers[i].y = +Y.toFixed(3); + covers[i].X = valuesx[i]; + covers[i].Y = valuesy[i]; + covers[i].value = size[i] || 0; + covers[i].dot = series[i]; + } + res.covers = covers; + res.series = series; + res.push(series, axis, covers); + res.hover = function (fin, fout) { + covers.mouseover(fin).mouseout(fout); + return this; + }; + res.click = function (f) { + covers.click(f); + return this; + }; + res.each = function (f) { + if (!Raphael.is(f, "function")) { + return this; + } + for (var i = covers.length; i--;) { + f.call(covers[i]); + } + return this; + }; + res.href = function (map) { + var cover; + for (var i = covers.length; i--;) { + cover = covers[i]; + if (cover.X == map.x && cover.Y == map.y && cover.value == map.value) { + cover.attr({href: map.href}); + } + } + }; + return res; +};