You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jv...@apache.org on 2011/12/10 01:12:13 UTC

svn commit: r1212697 - in /hive/trunk/testutils/ptest: ./ Buffer.py Process.py README Report.py Ssh.py config.py hivetest.py templates/ templates/Properties.html templates/TestCase.html templates/TestRun.html templates/TestSuite.html templates/common.css

Author: jvs
Date: Sat Dec 10 00:12:12 2011
New Revision: 1212697

URL: http://svn.apache.org/viewvc?rev=1212697&view=rev
Log:
HIVE-1487 [jira] Add ptest script
(Marek Sapota via John Sichi)

Summary: HIVE-1464 speeded up serial runs somewhat - but looks like it's still
too slow. we should use parallel junit or some similar setup to run test queries
in parallel. this should be really easy as well need to just use a separate
warehouse/metadb and potentiall mapred system dir location.

Test Plan: EMPTY

Reviewers: JIRA, jsichi

Reviewed By: jsichi

CC: mareksapotafb, jsichi

Differential Revision: 603

Added:
    hive/trunk/testutils/ptest/
    hive/trunk/testutils/ptest/Buffer.py
    hive/trunk/testutils/ptest/Process.py
    hive/trunk/testutils/ptest/README
    hive/trunk/testutils/ptest/Report.py
    hive/trunk/testutils/ptest/Ssh.py
    hive/trunk/testutils/ptest/config.py
    hive/trunk/testutils/ptest/hivetest.py
    hive/trunk/testutils/ptest/templates/
    hive/trunk/testutils/ptest/templates/Properties.html
    hive/trunk/testutils/ptest/templates/TestCase.html
    hive/trunk/testutils/ptest/templates/TestRun.html
    hive/trunk/testutils/ptest/templates/TestSuite.html
    hive/trunk/testutils/ptest/templates/common.css

Added: hive/trunk/testutils/ptest/Buffer.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/Buffer.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/Buffer.py (added)
+++ hive/trunk/testutils/ptest/Buffer.py Sat Dec 10 00:12:12 2011
@@ -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.
+
+import collections
+
+class Buffer():
+    def __init__(self, size = 100, abandon_output = True):
+        self.buf_short = collections.deque(maxlen = size)
+        self.buf = []
+        self.abandon_output = abandon_output
+
+    def put(self, line):
+        if not self.abandon_output:
+            self.buf.append(line)
+        self.buf_short.append(line)
+
+    def get_short(self):
+        return ''.join(self.buf_short)
+
+    def get_long(self):
+        if self.abandon_output:
+            return None
+        else:
+            return ''.join(self.buf)

Added: hive/trunk/testutils/ptest/Process.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/Process.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/Process.py (added)
+++ hive/trunk/testutils/ptest/Process.py Sat Dec 10 00:12:12 2011
@@ -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.
+
+from __future__ import print_function
+
+from subprocess import Popen, PIPE, STDOUT
+from shlex import split
+
+from Buffer import Buffer
+
+def run(cmd, quiet = False, abandon_output = True):
+    proc = Popen(split(cmd), stdout = PIPE, stderr = STDOUT)
+    buf = Buffer(abandon_output = abandon_output)
+    line = proc.stdout.readline()
+    while len(line):
+        buf.put(line)
+        if not quiet:
+            print(line, end = '')
+        line = proc.stdout.readline()
+    # Process could probably close the descriptor before exiting.
+    proc.wait()
+    if proc.returncode != 0:
+        raise Exception('Process exited with a non-zero return code.  ' +
+                'Last output of the program:\n\n' +
+                '---------- Start of exception log --\n' +
+                buf.get_short().strip() +
+                '\n---------- End of exception log --\n')
+    return buf.get_long()

Added: hive/trunk/testutils/ptest/README
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/README?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/README (added)
+++ hive/trunk/testutils/ptest/README Sat Dec 10 00:12:12 2011
@@ -0,0 +1,49 @@
+= Configuration file =
+
+Configuration file is JSON formated, example:
+
+{
+    "qfile_hosts": [
+        ["hostname1", 2],
+        ["hostname2", 4],
+        ["hostname3", 4],
+    ],
+    "other_hosts": [
+        ["hostname1", 2],
+        ["hostname4", 5]
+    ],
+    "master_base_path": "${{HOME}}/hivetests",
+    "host_base_path": "/mnt/drive/hivetests"
+    "java_home": "/opt/jdk"
+}
+
+== qfile_hosts ==
+List of hosts that should run TestCliDriver and TestNegativeCliDriver test
+cases.  Number following the host name is number of simultaneous tests that
+should be run on this host, you should probably set it near number of cores that
+host has.
+
+== other_hosts ==
+List of hosts that should run all other test cases.  Number has the same meaning
+as in `qfile_hosts`.
+
+== master_base_path ==
+Path on localhost (master) where this script can build Hive, store reports, etc.
+This path should be available from every slave node and should point to the same
+data (home on NFS would be a good choice).
+
+== host_base_path ==
+Path on slaves where Hive repo will be cloned and tests will be run.
+'-your_user_name' will be actually appended to this path to allow parallel runs
+by different users.
+
+== java_home ==
+Should point to Java environment that should be used.
+
+== About paths ==
+You can use environmental variables with `${{my_env}}`, as home is used in the
+example.
+
+You shouldn't point this paths to your work repository or any directory that
+stores data you don't want to lose.  This script might wipe everything under
+`master_base_path` and `host_base_path` as needed.

Added: hive/trunk/testutils/ptest/Report.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/Report.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/Report.py (added)
+++ hive/trunk/testutils/ptest/Report.py Sat Dec 10 00:12:12 2011
@@ -0,0 +1,230 @@
+#!/usr/bin/env python
+#
+# 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.
+
+from __future__ import print_function
+
+import os
+import os.path
+import re
+import base64
+import argparse
+from xml.dom import Node
+from xml.dom.minidom import parseString
+
+from mako.template import Template
+
+report_dir = os.path.dirname(os.path.realpath(__file__))
+one_file = False
+
+class TemplateRenderer():
+    def __init__(self, template):
+        self.__template = template
+
+    def render(self, file_name = None):
+        if file_name is None:
+            file_name = self.__template
+        if not file_name.startswith('/'):
+            current_dir = os.path.dirname(os.path.realpath(__file__))
+            file_name = os.path.join(current_dir, file_name)
+        return Template(filename = file_name).render(this = self)
+
+    def render_link(self, link_name, file_name):
+        if one_file:
+            return 'data:text/html;charset=utf-8;base64,' + \
+                    base64.b64encode(self.render(file_name))
+        else:
+            part = self.render(file_name)
+            with open(report_dir + '/' + link_name, 'w') as f:
+                f.write(part)
+            return link_name
+
+    def render_files(self):
+        report = self.render()
+        with open(report_dir + '/report.html', 'w') as f:
+            f.write(report)
+
+class TestCase(TemplateRenderer):
+    def __init__(self, element):
+        TemplateRenderer.__init__(self, 'templates/TestCase.html')
+
+        self.__class_name = element.getAttribute('classname')
+        self.__name = element.getAttribute('name')
+        self.__time = float(element.getAttribute('time'))
+        self.__failure = False
+        self.__error = False
+        self.__log = None
+
+        for child in element.childNodes:
+            if child.nodeType == Node.ELEMENT_NODE and child.tagName == 'failure':
+                self.__failure = True
+                self.__log = child.firstChild.nodeValue
+            elif child.nodeType == Node.ELEMENT_NODE and child.tagName == 'error':
+                self.__error = True
+                self.__log = child.firstChild.nodeValue
+
+    def success(self):
+        return not (self.failure() or self.error())
+
+    def failure(self):
+        return self.__failure
+
+    def error(self):
+        return self.__error
+
+    def get_log(self):
+        return self.__log
+
+    def get_name(self):
+        return self.__name
+
+    def get_time(self):
+        return self.__time
+
+class TestSuite(TemplateRenderer):
+    def __init__(self, text):
+        TemplateRenderer.__init__(self, 'templates/TestSuite.html')
+
+        self.properties = {}
+        self.test_cases = []
+
+        xml = parseString(text)
+        self.__populate_properties(xml)
+        self.__populate_test_cases(xml)
+
+        top = xml.getElementsByTagName('testsuite')[0]
+        self.__errors = int(top.getAttribute('errors'))
+        self.__failures = int(top.getAttribute('failures'))
+        self.__tests = int(top.getAttribute('tests'))
+        self.__host_name = top.getAttribute('hostname')
+        dist_dir = self.properties['dist.dir']
+        build_number = re.findall(self.__host_name + '-([0-9]+)$', dist_dir)
+        if build_number:
+            # Multiple builds per host.
+            self.__host_name += '-' + build_number[0]
+        self.__name = top.getAttribute('name').split('.')[-1]
+        self.__time = float(top.getAttribute('time'))
+
+    def __populate_properties(self, xml):
+        properties = xml.getElementsByTagName('property')
+        for prop in properties:
+            self.properties[prop.getAttribute('name')] = prop.getAttribute('value')
+
+    def __populate_test_cases(self, xml):
+        test_cases = xml.getElementsByTagName('testcase')
+        for test in test_cases:
+            self.test_cases.append(TestCase(test))
+
+    def tests(self):
+        return self.__tests
+
+    def failures(self):
+        return self.__failures
+
+    def errors(self):
+        return self.__errors
+
+    def passes(self):
+        return self.tests() - self.failures() - self.errors()
+
+    def time(self):
+        return self.__time
+
+    def host_name(self):
+        return self.__host_name
+
+    def name(self):
+        return self.__name
+
+    def label(self):
+        return self.host_name() + '-' + self.name()
+
+class TestRun(TemplateRenderer):
+    def __init__(self, pwd):
+        TemplateRenderer.__init__(self, 'templates/TestRun.html')
+
+        self.test_suites = []
+
+        files = os.listdir(pwd)
+        pattern = re.compile('^TEST-.*\.xml$')
+        for f in files:
+            if pattern.search(f) is not None:
+                with open(os.path.join(pwd, f)) as handle:
+                    self.test_suites.append(TestSuite(handle.read()))
+
+    def passes(self):
+        return reduce(lambda acc, x: acc + x.passes(), self.test_suites, 0)
+
+    def failures(self):
+        return reduce(lambda acc, x: acc + x.failures(), self.test_suites, 0)
+
+    def errors(self):
+        return reduce(lambda acc, x: acc + x.errors(), self.test_suites, 0)
+
+    def tests(self):
+        return reduce(lambda acc, x: acc + x.tests(), self.test_suites, 0)
+
+    def time(self):
+        return reduce(lambda acc, x: acc + x.time(), self.test_suites, 0)
+
+    def success_rate(self):
+        if self.tests():
+          return 100.0 * self.passes() / self.tests()
+        else:
+          return 100.0
+
+def make_report(args):
+    global report_dir, one_file
+    report_dir = args.report_dir
+    one_file = args.one_file
+
+    test_run = TestRun(args.log_dir)
+    test_run.render_files()
+    print('Summary:')
+    print('  tests run:', test_run.tests())
+    print('  failures:', test_run.failures())
+    print('  errors:', test_run.errors())
+
+    failed_results = {}
+    files = os.listdir(args.log_dir)
+    pattern = re.compile('^([^-]+-[^-]+)-(.*)\.fail$')
+    for f in files:
+        match = pattern.findall(f)
+        if match:
+            (host, test, ) = match[0]
+            if host not in failed_results:
+                failed_results[host] = []
+            failed_results[host].append(test)
+    if failed_results:
+        print()
+        print('Some tests faled to produce a log and are not included in the report:')
+        for host in failed_results:
+            print('  {0}:'.format(host))
+            for test in failed_results[host]:
+                print('    {0}'.format(test))
+
+if __name__ == '__main__':
+    parser = argparse.ArgumentParser(
+            description = 'Create HTML report from JUnit logs.')
+    parser.add_argument(dest = 'log_dir',
+            help = 'Path to directory containing JUnit logs')
+    parser.add_argument(dest = 'report_dir',
+            help = 'Where should the report be generated')
+    parser.add_argument('--one-file', action = 'store_true', dest = 'one_file',
+            help = 'Inline everything and generate only one file')
+    args = parser.parse_args()
+
+    make_report(args)

Added: hive/trunk/testutils/ptest/Ssh.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/Ssh.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/Ssh.py (added)
+++ hive/trunk/testutils/ptest/Ssh.py Sat Dec 10 00:12:12 2011
@@ -0,0 +1,126 @@
+# 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.
+
+from threading import Thread
+from Queue import Queue
+
+import Process
+
+class SSHConnection():
+    def __init__(self, host, num = None):
+        self.host = host
+        if num is None:
+            self.hostname = host
+        else:
+            self.hostname = host + '-' + str(num);
+        self.pwd = '/'
+        self.env = {}
+        self.path = []
+
+    def cd(self, path):
+        self.pwd = path.format(host = self.hostname)
+
+    def export(self, env, value):
+        self.env[env] = value.format(host = self.hostname)
+
+    def add_path(self, path):
+        self.path.append(path.format(host = self.hostname))
+
+    def prefix(self, cmd):
+        pre = []
+        pre.append('cd "{0}"'.format(self.pwd))
+        for (e, v) in self.env.iteritems():
+            pre.append('export {0}="{1}"'.format(e, v))
+        for p in self.path:
+            pre.append('export PATH="{0}:${{PATH}}"'.format(p))
+        pre.append(cmd)
+        return ' && '.join(pre)
+
+    def run(self, cmd, warn_only = False, quiet = False, vewy_quiet = False,
+            abandon_output = True):
+        # Don't use single quotes in `cmd`, this will break and end badly.
+        cmd = cmd.format(host = self.hostname)
+        cmd = self.prefix(cmd)
+        print(self.hostname + ' =>')
+        if vewy_quiet:
+            # Be vewy, vewy quiet, I'm hunting wabbits.
+            print('[command hidden]\n')
+            quiet = True
+        else:
+            print(cmd + '\n')
+        cmd = "ssh -nT '{0}' '{1}'".format(self.host, cmd)
+        try:
+            return Process.run(cmd, quiet, abandon_output)
+        except Exception as e:
+            if warn_only:
+                print(str(e) + '---------- This was only a warning, ' +
+                        'it won\'t stop the execution --\n')
+                return None
+            else:
+                raise e
+
+class SSHSet():
+    def __init__(self, conn = []):
+        self.conn = conn
+
+    def __len__(self):
+        return len(self.conn)
+
+    def add(self, conn):
+        if isinstance(conn, list):
+            self.conn.extend(conn)
+        else:
+            self.conn.append(conn)
+
+    def cd(self, path):
+        for conn in self.conn:
+            conn.cd(path)
+
+    def export(self, env, value):
+        for conn in self.conn:
+            conn.export(env, value)
+
+    def add_path(self, path):
+        for conn in self.conn:
+            conn.add_path(path)
+
+    def run(self, cmd, parallel = True, quiet = False, vewy_quiet = False,
+            abandon_output = True, warn_only = False):
+        if not parallel:
+            for conn in self.conn:
+                conn.run(cmd, quiet = quiet, vewy_quiet = vewy_quiet,
+                        abandon_output = abandon_output, warn_only = warn_only)
+        else:
+            threads = []
+            queue = Queue()
+            def wrapper(conn, cmd, queue):
+                try:
+                    conn.run(cmd, quiet = quiet, vewy_quiet = vewy_quiet,
+                            abandon_output = abandon_output,
+                            warn_only = warn_only)
+                except Exception as e:
+                    queue.put(Exception(conn.hostname + ' => ' + str(e)))
+            for conn in self.conn:
+                thread = Thread(target = wrapper, args = (conn, cmd, queue, ))
+                thread.start()
+                threads.append(thread)
+            for thread in threads:
+                thread.join()
+            if not queue.empty():
+                l = []
+                while not queue.empty():
+                    e = queue.get()
+                    l.append(str(e));
+                raise Exception('\n'.join(l))

Added: hive/trunk/testutils/ptest/config.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/config.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/config.py (added)
+++ hive/trunk/testutils/ptest/config.py Sat Dec 10 00:12:12 2011
@@ -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.
+
+from __future__ import print_function
+
+import json
+import copy
+import getpass
+import os.path
+
+from Ssh import SSHConnection, SSHSet
+
+local = None
+qfile_set = None
+other_set = None
+remote_set = None
+all_set = None
+
+master_base_path = None
+host_base_path = None
+java_home = None
+
+def load(config_file = '~/.hive_ptest.conf'):
+    global local, qfile_set, other_set, remote_set, all_set
+    global master_base_path, host_base_path, java_home
+
+    config_file = os.path.expanduser(config_file)
+
+    cfg = None
+    try:
+        with open(config_file) as f:
+            cfg = json.loads(f.read())
+
+        host_nodes = {}
+        def get_node(host):
+            if not host in host_nodes:
+                host_nodes[host] = -1
+            host_nodes[host] += 1
+            return SSHConnection(host, host_nodes[host])
+
+        qfile = []
+        for (host, num, ) in cfg['qfile_hosts']:
+            for i in range(num):
+                qfile.append(get_node(host))
+
+        other = []
+        for (host, num, ) in cfg['other_hosts']:
+            for i in range(num):
+                other.append(get_node(host))
+
+        local = SSHConnection('localhost')
+
+        qfile_set = SSHSet(qfile)
+        other_set = SSHSet(other)
+
+        # Make copies, otherwise they they will be passed by reference and
+        # reused.  Reuse is bad - you don't want `cd` on remote_set to affect
+        # anything in the all_set.
+
+        remote_set = SSHSet(copy.copy(qfile))
+        remote_set.add(copy.copy(other))
+
+        all_set = SSHSet(copy.copy(qfile))
+        all_set.add(copy.copy(other))
+        all_set.add(local)
+
+        master_base_path = cfg['master_base_path']
+        host_base_path = cfg['host_base_path'] + '-' + getpass.getuser()
+        java_home = cfg['java_home']
+    except Exception as e:
+        raise Exception('Failed to parse your configuration file (' +
+                config_file + ').  Maybe you forgot the `--config` switch?', e)

Added: hive/trunk/testutils/ptest/hivetest.py
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/hivetest.py?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/hivetest.py (added)
+++ hive/trunk/testutils/ptest/hivetest.py Sat Dec 10 00:12:12 2011
@@ -0,0 +1,466 @@
+#!/usr/bin/env python
+#
+# 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.
+
+from __future__ import print_function
+
+import argparse
+import time
+from threading import Thread
+import os.path
+import collections
+import re
+
+import Report
+import config
+
+# WARNING
+#
+# If you are editing this code, please be aware that commands passed to `run`
+# should not use single quotes, this will break and end badly as the final
+# command looks like `ssh 'host' 'some command - single quote will break it'`.
+# Also please be aware that `run` uses `.format` to change `{host}` in commands
+# into actual host name it is running on, running `.format` on strings using
+# `{host}`, for example including `host_code_path` will not work.
+#
+# Also this code assumes master_base_path is available to all testing machines
+# and is mounted in the same place on all of them.
+#
+# Getting rid of this restrictions without making the code much more complicated
+# is very welcome.
+
+# This is configured in user configuration file.
+
+local = None
+qfile_set = None
+other_set = None
+remote_set = None
+all_set = None
+
+master_base_path = None
+host_base_path = None
+
+# End of user configurated things.
+
+ant_path = None
+arc_path = None
+phutil_path = None
+code_path = None
+report_path = None
+host_code_path = None
+
+def read_conf(config_file):
+    global local, qfile_set, other_set, remote_set, all_set
+    global master_base_path, host_base_path
+    global ant_path, arc_path, phutil_path, code_path, report_path, host_code_path
+
+    if config_file is not None:
+        config.load(config_file)
+    else:
+        config.load()
+
+    local = config.local
+    qfile_set = config.qfile_set
+    other_set = config.other_set
+    remote_set = config.remote_set
+    all_set = config.all_set
+
+    master_base_path = config.master_base_path
+    host_base_path = config.host_base_path
+
+    ant_path = master_base_path + '/apache-ant-1.8.2'
+    arc_path = master_base_path + '/arcanist'
+    phutil_path = master_base_path + '/libphutil'
+    code_path = master_base_path + '/trunk'
+    report_path = master_base_path + '/report/' + time.strftime('%m.%d.%Y_%H:%M:%S')
+    host_code_path = host_base_path + '/trunk-{host}'
+
+    # Setup of needed environmental variables and paths
+
+    # Ant
+    all_set.add_path(ant_path + '/bin')
+
+    # Arcanist
+    all_set.add_path(arc_path + '/bin')
+
+    # Java
+    all_set.export('JAVA_HOME', config.java_home)
+    all_set.add_path(config.java_home + '/bin')
+
+    # Hive
+    remote_set.export('HIVE_HOME', host_code_path + '/build/dist')
+    remote_set.add_path(host_code_path + '/build/dist/bin')
+
+    # Hadoop
+    remote_set.export('HADOOP_HOME', host_code_path +
+            '/build/hadoopcore/hadoop-0.20.1')
+
+def get_ant():
+    # Gets Ant 1.8.2 from one of Apache mirrors.
+    print('\n-- Installing Ant 1.8.2\n')
+
+    if local.run('test -d "{0}"'.format(ant_path), warn_only = True,
+            abandon_output = False) is None:
+        local.cd(master_base_path)
+        local.run('curl "http://apache.osuosl.org//ant/binaries/apache-ant-1.8.2-bin.tar.gz" | tar xz')
+    else:
+        print('\n  Ant 1.8.2 already installed\n')
+
+def get_arc():
+    # Gets latest Arcanist and libphtuil from their Git repositories.
+    print('\n-- Updating Arcanist installation\n')
+
+    if local.run('test -d "{0}"'.format(arc_path), warn_only = True,
+            abandon_output = False) is None:
+        local.run('mkdir -p "{0}"'.format(os.path.dirname(arc_path)))
+        local.run('git clone git://github.com/facebook/arcanist.git "{0}"'
+                .format(arc_path))
+
+    if local.run('test -d "{0}"'.format(phutil_path), warn_only = True,
+            abandon_output = False) is None:
+        local.run('mkdir -p "{0}"'.format(os.path.dirname(phutil_path)))
+        local.run('git clone git://github.com/facebook/libphutil.git "{0}"'
+                .format(phutil_path))
+
+    local.cd(arc_path)
+    local.run('git pull')
+    local.cd(phutil_path)
+    local.run('git pull')
+
+def get_clean_hive():
+    # Gets latest Hive from Apache Git repository and cleans the repository
+    # (undo of any changes and removal of all generated files).  Also runs
+    # `arc-setup` so the repo is ready to be used.
+    print('\n-- Updating Hive repo\n')
+
+    if local.run('test -d "{0}"'.format(code_path), warn_only = True,
+            abandon_output = False) is None:
+        local.run('mkdir -p "{0}"'.format(os.path.dirname(code_path)))
+        local.run('git clone git://git.apache.org/hive.git "{0}"'.format(code_path))
+
+    local.cd(code_path)
+    local.run('git reset --hard HEAD')
+    local.run('git clean -dffx')
+    local.run('git pull')
+    local.run('ant arc-setup')
+
+def prepare_for_reports():
+    # Generates directories for test reports.  All test nodes will copy results
+    # to this directories.
+    print('\n-- Creating a directory for JUnit reports\n')
+    # Remove previous reports that might be there.
+    local.run('rm -rf "{0}"'.format(report_path), warn_only = True)
+    local.run('mkdir -p "{0}/logs"'.format(report_path))
+    local.run('mkdir -p "{0}/out/clientpositive"'.format(report_path))
+    local.run('mkdir -p "{0}/out/clientnegative"'.format(report_path))
+
+def patch_hive(patches = [], revision = None):
+    # Applies given patches to the Hive repo.  Revision means a Differential
+    # revision, patches list is a list of paths to patches on local file system.
+    #
+    # Allowing multiple revisions and patches would complicate things a little
+    # (order of applied patches should be preserved, but argparse will split
+    # them into two lists) so only multiple local patches are allowed.
+    # Shouldn't be a big problem as you can use `arc export` to get the patches
+    # locally.
+    local.cd(code_path)
+    if revision is not None:
+        print('\n-- Patching Hive repo using a Differential revision\n')
+        revision = revision.upper()
+        if not revision.startswith('D'):
+            revision = 'D' + revision
+        local.run('arc patch "{0}"'.format(revision))
+    if patches:
+        print('\n-- Patching Hive repo using a patch from local file system\n')
+        for patch in patches:
+            local.run('patch -f -p0 < "{0}"'.format(patch))
+
+def build_hive():
+    print('\n-- Building Hive\n')
+    local.cd(code_path)
+    local.run('ant package')
+
+def propagate_hive():
+    # Expects master_base_path to be available on all test nodes in the same
+    # place (for example using NFS).
+    print('\n-- Propagating Hive repo to all hosts\n')
+    remote_set.run('mkdir -p "{0}"'.format(host_code_path))
+    remote_set.run('rsync -qa --delete "{0}/" "{1}"'.format(
+        code_path, host_code_path))
+
+def segment_tests(path):
+    # Removes `.q` files that should not be run on this host.  The huge shell
+    # command is slow (not really suprising considering amount of forking it has
+    # to do), you are welcome to make it better=).
+    local.cd(code_path + path)
+    tests = local.run('ls -1', quiet = True, abandon_output = False).strip().split('\n')
+
+    qfile_set.cd(host_code_path + path)
+    cmd = []
+    i = 0
+    for test in tests:
+        host = qfile_set.conn[i].hostname
+        cmd.append('if [[ "{host}" != "' + host + '" ]]; then rm -f "' + test + '"; fi')
+        i = (i + 1) % len(qfile_set)
+    cmd = ' && '.join(cmd)
+    # The command is huge and printing it out is not very useful, using wabbit
+    # hunting mode.
+    qfile_set.run(cmd, vewy_quiet = True)
+
+def prepare_tests():
+    print('\n-- Preparing test sets on all hosts\n')
+    segment_tests('/ql/src/test/queries/clientpositive')
+    segment_tests('/ql/src/test/queries/clientnegative')
+
+def collect_log(name):
+    # Moves JUnit log to the global logs directory.
+    #
+    # This has the same restriction on master_base_path as propagate_hive.
+    new_name = name.split('.')
+    new_name[-2] += '-{host}'
+    new_name = '.'.join(new_name)
+    qfile_set.cd(host_code_path + '/build/ql/test')
+    # If tests failed there may be no file, so warn only if `cp` is having
+    # problems.
+    qfile_set.run(
+        'cp "' + name + '" "' + report_path + '/logs/' + new_name + '" || ' +
+        'touch "' + report_path + '/logs/{host}-' + name + '.fail"'
+    )
+    # Get the hive.log too.
+    qfile_set.cd(host_code_path + '/build/ql/tmp')
+    qfile_set.run('cp "hive.log" "' + report_path + '/logs/hive-{host}-' + name + '.log"',
+            warn_only = True)
+
+def collect_out(name):
+    # Moves `.out` file (test output) to the global logs directory.
+    #
+    # This has the same restriction on master_base_path as propagate_hive.
+    qfile_set.cd(host_code_path + '/build/ql/test/logs/' + name)
+    # Warn only if no files are found.
+    qfile_set.run('cp * "' + report_path + '/out/' + name + '"', warn_only = True)
+
+def run_tests():
+    # Runs TestCliDriver and TestNegativeCliDriver testcases.
+    print('\n-- Running .q file tests on all hosts\n')
+
+    # Using `quiet` because output of `ant test` is not very useful when we are
+    # running on many hosts and it all gets mixed up.  In case of an error
+    # you'll get last lines generated by `ant test` anyway (might be totally
+    # irrelevant if one of the first tests fails and Ant reports a failure after
+    # running all the other test, fortunately JUnit report saves the Ant output
+    # if you need it for some reason).
+
+    qfile_set.cd(host_code_path)
+    qfile_set.run('ant -Dtestcase=TestCliDriver -Doffline=true test',
+            quiet = True, warn_only = True)
+    collect_log('TEST-org.apache.hadoop.hive.cli.TestCliDriver.xml')
+    collect_out('clientpositive')
+
+    qfile_set.cd(host_code_path)
+    qfile_set.run('ant -Dtestcase=TestNegativeCliDriver -Doffline=true test',
+            quiet = True, warn_only = True)
+    collect_log('TEST-org.apache.hadoop.hive.cli.TestNegativeCliDriver.xml')
+    collect_out('clientnegative')
+
+def run_other_tests():
+    # Runs all other tests that run_test doesn't run.
+
+    def get_other_list():
+        local.cd(code_path)
+        # Generate test classes in build.
+        local.run('ant -Dtestcase=nothing test')
+        tests = local.run(' | '.join([
+            'find build/*/test/classes -name "Test*.class"',
+            'sed -e "s:[^/]*/::g"',
+            'grep -v TestSerDe.class',
+            'grep -v TestHiveMetaStore.class',
+            'grep -v TestCliDriver.class',
+            'grep -v TestNegativeCliDriver.class',
+            'grep -v ".*\$.*\.class"',
+            'sed -e "s:\.class::"'
+        ]), abandon_output = False)
+        return tests.split()
+
+    def segment_other():
+        # Split all test cases between hosts.
+        def get_command(test):
+            return '; '.join([
+                'ant -Dtestcase=' + test + ' -Doffline=true test',
+
+                'cp "`find . -name "TEST-*.xml"`" "' + report_path + '/logs/" || ' +
+                'touch "' + report_path + '/logs/{host}-' + test + '.fail"',
+
+                'cp "build/ql/tmp/hive.log" "' + report_path + '/logs/hive-{host}-' + test + '.log"'
+            ])
+        cmd = []
+        i = 0
+        for test in get_other_list():
+            # Special case, don't run minimr tests in parallel.  They will run
+            # on the first host, and no other tests will run there (unless we
+            # have a single host).
+            #
+            # TODO: Real fix would be to allow parallel runs of minimr tests.
+            if len(other_set) > 1:
+                if re.match('.*minimr.*', test.lower()):
+                    host = other_set.conn[0].hostname
+                else:
+                    i = (i + 1) % len(other_set)
+                    if i == 0:
+                        i = 1
+                    host = other_set.conn[i].hostname
+            else:
+                # We are running on single host.
+                host = other_set.conn[0].hostname
+            cmd.append(
+                'if [[ "{host}" == "' + host + '" ]]; then ' +
+                get_command(test) +
+                '; fi'
+            )
+        return ' ; '.join(cmd)
+
+    command = segment_other()
+    other_set.cd(host_code_path)
+    # See comment about quiet option in run_tests.
+    other_set.run(command, quiet = True, warn_only = True)
+
+def generate_report(one_file_report = False):
+    # Uses `Report.py` to create a HTML report.
+    print('\n-- Generating a test report\n')
+
+    # Call format to remove '{{' and '}}'.
+    path = os.path.expandvars(report_path.format())
+    CmdArgs = collections.namedtuple('CmdArgs', ['one_file', 'log_dir', 'report_dir'])
+    args = CmdArgs(
+        one_file = one_file_report,
+        log_dir = '{0}/logs'.format(path),
+        report_dir = path
+    )
+    Report.make_report(args)
+
+    print('\n-- Test report has been generated and is available here:')
+    print('-- "{0}/report.html"'.format(path))
+    print()
+
+def stop_tests():
+    # Brutally stops tests on all hosts, something more subtle would be nice and
+    # would allow the same user to run this script multiple times
+    # simultaneously.
+    print('\n-- Stopping tests on all hosts\n')
+    remote_set.run('killall -9 java', warn_only = True)
+
+def remove_code():
+    # Running this only on one connection per host so there are no conflicts
+    # between several `rm` calls.  This removes all repositories, it would have
+    # to be changed if we were to allow multiple simultaneous runs of this
+    # script.
+
+    print('\n-- Removing Hive code from all hosts\n')
+    # We could remove only `host_code_path`, but then we would have abandoned
+    # directories after lowering number of processes running on one host.
+    cmd = 'rm -rf "' + host_base_path + '"'
+    cmd = 'if [[ `echo "{host}" | grep -q -- "-0$"; echo "$?"` -eq "0" ]]; then ' + \
+            cmd + '; fi'
+    remote_set.run(cmd)
+
+def overwrite_results():
+    # Copy generated `.q.out` files to master repo.
+
+    local.cd(code_path)
+    expanded_path = local.run('pwd', abandon_output = False)
+    print('\n-- Copying generated `.q.out` files to master repository: ' +
+            expanded_path)
+
+    for name in ['clientpositive', 'clientnegative']:
+        local.cd(report_path + '/out/' + name)
+        # Don't panic if no files are found.
+        local.run('cp * "' + code_path + '/ql/src/test/results/' + name + '"',
+                warn_only = True)
+
+# -- Tasks that can be called from command line start here.
+
+def cmd_prepare(patches = [], revision = None):
+    get_ant()
+    get_arc()
+    get_clean_hive()
+    patch_hive(patches, revision)
+    build_hive()
+    propagate_hive()
+    prepare_tests()
+
+def cmd_run_tests(one_file_report = False):
+    t = Thread(target = run_other_tests)
+    t.start()
+    prepare_for_reports()
+    run_tests()
+    t.join()
+
+    if args.overwrite:
+        overwrite_results()
+
+    generate_report(one_file_report)
+
+def cmd_test(patches = [], revision = None, one_file_report = False):
+    cmd_prepare(patches, revision)
+    cmd_run_tests(one_file_report)
+
+def cmd_stop():
+    stop_tests()
+
+def cmd_remove():
+    remove_code()
+
+parser = argparse.ArgumentParser(description =
+        'Hive test farm controller.')
+parser.add_argument('--config', dest = 'config',
+        help = 'Path to configuration file')
+parser.add_argument('--prepare', action = 'store_true', dest = 'prepare',
+        help = 'Builds Hive and propagates it to all test machines')
+parser.add_argument('--run-tests', action = 'store_true', dest = 'run_tests',
+        help = 'Runs tests on all test machines')
+parser.add_argument('--test', action = 'store_true', dest = 'test',
+        help = 'Same as running `prepare` and then `run-tests`')
+parser.add_argument('--report-name', dest = 'report_name',
+        help = 'Store report and logs directory called `REPORT_NAME`')
+parser.add_argument('--stop', action = 'store_true', dest = 'stop',
+        help = 'Kill misbehaving tests on all machines')
+parser.add_argument('--remove', action = 'store_true', dest = 'remove',
+        help = 'Remove Hive trunk copies from test machines')
+parser.add_argument('--revision', dest = 'revision',
+        help = 'Differential revision to test')
+parser.add_argument('--patch', dest = 'patch', nargs = '*',
+        help = 'Patches from local file system to test')
+parser.add_argument('--one-file-report', dest = 'one_file_report',
+        action = 'store_true',
+        help = 'Generate one (huge) report file instead of multiple small ones')
+parser.add_argument('--overwrite', dest = 'overwrite', action = 'store_true',
+        help = 'Overwrite result files in master repo')
+args = parser.parse_args()
+
+read_conf(args.config)
+
+if args.report_name:
+    report_path = '/'.join(report_path.split('/')[:-1] + [args.report_name])
+
+if args.prepare:
+    cmd_prepare(args.patch, args.revision)
+elif args.run_tests:
+    cmd_run_tests(args.one_file_report)
+elif args.test:
+    cmd_test(args.patch, args.revision, args.one_file_report)
+elif args.stop:
+    cmd_stop()
+elif args.remove:
+    cmd_remove()

Added: hive/trunk/testutils/ptest/templates/Properties.html
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/templates/Properties.html?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/templates/Properties.html (added)
+++ hive/trunk/testutils/ptest/templates/Properties.html Sat Dec 10 00:12:12 2011
@@ -0,0 +1,41 @@
+<!doctype html>
+<!--
+  - 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.
+-->
+<html>
+  <head>
+    <title>${this.host_name()} - Properties</title>
+    <style type="text/css">
+      ${this.render('templates/common.css')}
+    </style>
+  </head>
+  <body>
+    % if this.properties:
+      <table class="properties">
+        <tr>
+          <th>Name</th>
+          <th>Value</th>
+        </tr>
+        % for (name, value, ) in this.properties.iteritems():
+          <tr>
+            <td>${name}</td>
+            <td class="long-lines">${value}</td>
+          </tr>
+        % endfor
+      </table>
+    % endif
+  </body>
+</html>

Added: hive/trunk/testutils/ptest/templates/TestCase.html
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/templates/TestCase.html?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/templates/TestCase.html (added)
+++ hive/trunk/testutils/ptest/templates/TestCase.html Sat Dec 10 00:12:12 2011
@@ -0,0 +1,43 @@
+<!--
+  - 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.
+-->
+
+<tr
+  % if this.failure() or this.error():
+    class="failure"
+  % endif
+>
+  <td>${this.get_name()}</td>
+  <td>
+    % if this.failure():
+      Failed
+    % elif this.error():
+      Error
+    % else:
+      Success
+    % endif
+  </td>
+  <td>${this.get_time()}s</td>
+</tr>
+% if this.failure() or this.error():
+  <tr class="failure">
+    <td colspan="3">
+      <pre>
+${this.get_log()}
+      </pre>
+    </td>
+  </tr>
+% endif

Added: hive/trunk/testutils/ptest/templates/TestRun.html
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/templates/TestRun.html?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/templates/TestRun.html (added)
+++ hive/trunk/testutils/ptest/templates/TestRun.html Sat Dec 10 00:12:12 2011
@@ -0,0 +1,90 @@
+<!doctype html>
+<!--
+  - 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.
+-->
+<html>
+  <head>
+    <title>Unit Test Results.</title>
+    <style type="text/css">
+      ${this.render('templates/common.css')}
+    </style>
+  </head>
+  <body>
+    <h1 id="top">Unit Test Results.</h1>
+
+    <h2>Summary</h2>
+    <table>
+      <tr>
+        <th>Tests</th>
+        <th>Passes</th>
+        <th>Failures</th>
+        <th>Errors</th>
+        <th>Success rate</th>
+        <th>Time</th>
+      </tr>
+      <tr
+        % if this.failures() + this.errors():
+          class="failure"
+        % endif
+      >
+        <td>${this.tests()}</td>
+        <td>${this.passes()}</td>
+        <td>${this.failures()}</td>
+        <td>${this.errors()}</td>
+        <td>${round(this.success_rate(), 2)}%</td>
+        <td>${round(this.time(), 2)}s</td>
+      </tr>
+    </table>
+
+    % if this.test_suites:
+      <h2>Test results</h2>
+      <table>
+        <tr>
+          <th>Host</th>
+          <th>Name</th>
+          <th>Tests</th>
+          <th>Passes</th>
+          <th>Failures</th>
+          <th>Errors</th>
+          <th>Time</th>
+        </tr>
+        % for test_suite in this.test_suites:
+          <tr
+            % if test_suite.failures() + test_suite.errors():
+              class="failure"
+            % endif
+          >
+            <td>
+              <a href="#${test_suite.label()}">
+                ${test_suite.host_name()}
+              </a>
+            </td>
+            <td>${test_suite.name()}</td>
+            <td>${test_suite.tests()}</td>
+            <td>${test_suite.passes()}</td>
+            <td>${test_suite.failures()}</td>
+            <td>${test_suite.errors()}</td>
+            <td>${round(test_suite.time(), 2)}s</td>
+          </tr>
+        % endfor
+      </table>
+    % endif
+
+    % for test_suite in this.test_suites:
+      ${test_suite.render()}
+    % endfor
+  </body>
+</html>

Added: hive/trunk/testutils/ptest/templates/TestSuite.html
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/templates/TestSuite.html?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/templates/TestSuite.html (added)
+++ hive/trunk/testutils/ptest/templates/TestSuite.html Sat Dec 10 00:12:12 2011
@@ -0,0 +1,37 @@
+<!--
+  - 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.
+-->
+
+<h3 id="${this.label()}">${this.label()}</h3>
+% if this.test_cases:
+  <table>
+    <tr>
+      <th class="wide">Name</th>
+      <th>Status</th>
+      <th>Time</th>
+    </tr>
+    % for test_case in this.test_cases:
+      ${test_case.render()}
+    % endfor
+  </table>
+% endif
+<a
+  class="float-right"
+  href="${this.render_link(this.label() + '-properties.html', 'templates/Properties.html')}"
+>
+  Properties &gt;
+</a>
+<a href="#top">Back to top</a>

Added: hive/trunk/testutils/ptest/templates/common.css
URL: http://svn.apache.org/viewvc/hive/trunk/testutils/ptest/templates/common.css?rev=1212697&view=auto
==============================================================================
--- hive/trunk/testutils/ptest/templates/common.css (added)
+++ hive/trunk/testutils/ptest/templates/common.css Sat Dec 10 00:12:12 2011
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+body {
+  font-size: 0.85em;
+}
+
+a {
+  color: blue;
+}
+
+table {
+  width: 100%;
+}
+
+table.properties {
+  table-layout: fixed;
+}
+
+table.properties th:first-child {
+  width: 30%;
+}
+
+table.properties th:first-child + th {
+  width: 70%;
+}
+
+table th {
+  background-color: #a6caf0;
+  padding: 0.3em;
+  text-align: left;
+}
+
+table td {
+  background-color: #eeeee0;
+  padding: 0.3em;
+}
+
+table tr.failure {
+  color: red;
+  font-weight: bold;
+}
+
+table tr.failure a {
+  color: red;
+}
+
+.long-lines {
+  word-wrap: break-word;
+}
+
+.float-right {
+  float: right;
+}
+
+.wide {
+  width: 100%;
+}