From 101bcf75910ace3c4aac60b9c267da489ce3cf42 Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Mon, 19 Sep 2016 10:26:56 -0700 Subject: [PATCH 1/6] STORM-2090: Add integration test for storm windowing --- .travis.yml | 9 +- integration-test/README.md | 49 +++ integration-test/config/Vagrantfile | 183 +++++++++ integration-test/config/cluster.xml | 96 +++++ integration-test/config/common.sh | 5 + integration-test/config/config-supervisord.sh | 5 + integration-test/config/etc-hosts | 5 + integration-test/config/install-storm.sh | 23 ++ integration-test/config/install-zookeeper.sh | 4 + integration-test/config/start-supervisord.sh | 1 + integration-test/config/storm.yaml | 17 + integration-test/config/user-script.sh | 1 + integration-test/pom.xml | 250 ++++++++++++ integration-test/run-it.sh | 81 ++++ .../storm/ExclamationTopology.java | 95 +++++ .../hortonworks/storm/debug/DebugHelper.java | 42 ++ .../storm/st/topology/TestableTopology.java | 33 ++ .../window/SlidingTimeCorrectness.java | 167 ++++++++ .../window/SlidingWindowCorrectness.java | 156 ++++++++ .../window/TumblingTimeCorrectness.java | 164 ++++++++ .../window/TumblingWindowCorrectness.java | 153 ++++++++ .../st/topology/window/data/FromJson.java | 25 ++ .../st/topology/window/data/TimeData.java | 113 ++++++ .../topology/window/data/TimeDataWindow.java | 94 +++++ .../hortonworks/storm/st/utils/HdfsUtils.java | 58 +++ .../storm/st/utils/StringDecorator.java | 40 ++ .../hortonworks/storm/st/utils/TimeUtil.java | 57 +++ .../com/hortonworks/storm/st/DemoTest.java | 87 +++++ .../storm/st/helper/AbstractTest.java | 30 ++ .../storm/st/meta/TestngListener.java | 103 +++++ .../st/tests/window/SlidingWindowTest.java | 189 +++++++++ .../st/tests/window/TumblingWindowTest.java | 100 +++++ .../storm/st/utils/AssertUtil.java | 74 ++++ .../storm/st/utils/DataGenerator.java | 37 ++ .../hortonworks/storm/st/utils/UiAssert.java | 53 +++ .../hortonworks/storm/st/utils/UiUtil.java | 107 ++++++ .../storm/st/wrapper/HdfsWrap.java | 107 ++++++ .../hortonworks/storm/st/wrapper/LogData.java | 70 ++++ .../storm/st/wrapper/StormCluster.java | 114 ++++++ .../storm/st/wrapper/TopoWrap.java | 362 ++++++++++++++++++ .../src/test/resources/storm-conf/storm.yaml | 95 +++++ 41 files changed, 3452 insertions(+), 2 deletions(-) create mode 100644 integration-test/README.md create mode 100644 integration-test/config/Vagrantfile create mode 100644 integration-test/config/cluster.xml create mode 100644 integration-test/config/common.sh create mode 100644 integration-test/config/config-supervisord.sh create mode 100644 integration-test/config/etc-hosts create mode 100644 integration-test/config/install-storm.sh create mode 100644 integration-test/config/install-zookeeper.sh create mode 100644 integration-test/config/start-supervisord.sh create mode 100644 integration-test/config/storm.yaml create mode 100644 integration-test/config/user-script.sh create mode 100755 integration-test/pom.xml create mode 100755 integration-test/run-it.sh create mode 100644 integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java create mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java create mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java create mode 100644 integration-test/src/test/resources/storm-conf/storm.yaml diff --git a/.travis.yml b/.travis.yml index 941e09cc777..286f1d4b6b5 100644 --- a/.travis.yml +++ b/.travis.yml @@ -9,6 +9,9 @@ # 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. +addons: + hosts: + - node1 env: - MODULES=storm-core @@ -22,8 +25,10 @@ before_install: - nvm install 0.12.2 - nvm use 0.12.2 install: /bin/bash ./dev-tools/travis/travis-install.sh `pwd` -script: /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES -sudo: false +script: + - /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES + - /bin/bash ./integration-test/run-it.sh +sudo: true cache: directories: - "$HOME/.m2/repository" diff --git a/integration-test/README.md b/integration-test/README.md new file mode 100644 index 00000000000..42a086c0b56 --- /dev/null +++ b/integration-test/README.md @@ -0,0 +1,49 @@ +End to end storm-integration-tests +================================== + +Bring up a cluster +------------------ +Vagrant setup can be used for bringing up unsecure Storm 1.0 +https://github.com/raghavgautam/storm-vagrant + +Configs for running +------------------- +Change following storm.yaml: +storm-integration-test/src/test/resources/storm.yaml + +Running tests end to end from Commandline +----------------------------------------- +To run all tests: +```sh +mvn clean package -DskipTests && mvn test +``` + +To run a single test: +```sh +mvn clean package -DskipTests && mvn test -Dtest=SlidingWindowCountTest +``` + +Running tests from IDE +---------------------- +Make sure that the following is run before tests are launched. +```sh +mvn package -DskipTests +``` + +Running tests with custom storm version +--------------------------------------- +You can supply custom storm version using `-Dstorm.version=` property to all the maven commands. +```sh +mvn clean package -DskipTests -Dstorm.version= +mvn test -Dtest=DemoTest -Dstorm.version= +``` + +To find version of the storm that you are running run `storm version` command. + +Submiting topologies +-------------------- +All the topologies can be submitted using `storm jar` command. + +Code +---- +Start off by looking at file [DemoTest.java](https://github.com/raghavgautam/storm-integration-test/blob/master/src/test/java/com/hortonworks/storm/st/DemoTest.java). diff --git a/integration-test/config/Vagrantfile b/integration-test/config/Vagrantfile new file mode 100644 index 00000000000..7898d9b065f --- /dev/null +++ b/integration-test/config/Vagrantfile @@ -0,0 +1,183 @@ +# -*- mode: ruby; compile-command: "vagrant destroy -f; vagrant up" -*- +# vi: set ft=ruby : +# +# 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. +# +require 'uri' +# Vagrantfile API/syntax version. Don't touch unless you know what you're doing! +VAGRANTFILE_API_VERSION = "2" +STORM_BOX_TYPE = "hashicorp/precise64" +STORM_ZIP = Dir.glob("../../storm-dist/binary/target/**/*.zip") +if(STORM_ZIP.length != 1) + raise "expected one storm-binary found: " + STORM_ZIP.join(",") +end +STORM_ARCHIVE = STORM_ZIP[0] +STORM_VERSION = File.basename(STORM_ARCHIVE, '.*') +STORM_SUPERVISOR_COUNT = 2 + +Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| + + config.vm.box = STORM_BOX_TYPE + #config.hostmanager.manage_host = true + #config.hostmanager.enabled = true + + if Vagrant.has_plugin?("vagrant-cachier") + # Configure cached packages to be shared between instances of the same base box. + # More info on the "Usage" link above + config.cache.scope = :box + + # OPTIONAL: If you are using VirtualBox, you might want to use that to enable + # NFS for shared folders. This is also very useful for vagrant-libvirt if you + # want bi-directional sync + config.cache.synced_folder_opts = { + type: :nfs, + # The nolock option can be useful for an NFSv3 client that wants to avoid the + # NLM sideband protocol. Without this option, apt-get might hang if it tries + # to lock files needed for /var/cache/* operations. All of this can be avoided + # by using NFSv4 everywhere. Please note that the tcp option is not the default. + mount_options: ['rw', 'vers=3', 'tcp', 'nolock'] + } + end + + if(!File.exist?(STORM_ARCHIVE)) + `wget -N #{STORM_DIST_URL}` + end + + config.vm.synced_folder "../../", "/home/vagrant/build/vagrant/storm" + config.vm.synced_folder "~/.m2", "/home/vagrant/.m2" + + config.vm.define "node1" do |node1| + node1.vm.provider "virtualbox" do |v| + v.customize ["modifyvm", :id, "--natdnshostresolver1", "on"] + end + node1.vm.network "private_network", ip: "192.168.50.3" + node1.vm.hostname = "node1" + node1.vm.provision :shell, :inline => "echo run integration test; whoami; env; cd /home/vagrant/build/vagrant/storm/; pwd; bash integration-test/run-it.sh", privileged: false + #node1.vm.provision :shell, :inline => "sudo ln -fs /vagrant/etc-hosts /etc/hosts" + end + + + # All Vagrant configuration is done here. The most common configuration + # options are documented and commented below. For a complete reference, + # please see the online documentation at vagrantup.com. + + # Every Vagrant virtual environment requires a box to build off of. + #config.vm.box = "precise32" + + # The url from where the 'config.vm.box' box will be fetched if it + # doesn't already exist on the user's system. + #config.vm.box_url = "http://files.vagrantup.com/precise32.box" + + # Create a forwarded port mapping which allows access to a specific port + # within the machine from a port on the host machine. In the example below, + # accessing "localhost:8080" will access port 80 on the guest machine. + # config.vm.network :forwarded_port, guest: 8080, host: 8080 + + # Create a private network, which allows host-only access to the machine + # using a specific IP. + config.vm.network :private_network, ip: "192.168.100.100" + + # Create a public network, which generally matched to bridged network. + # Bridged networks make the machine appear as another physical device on + # your network. + # config.vm.network :public_network + + # If true, then any SSH connections made will enable agent forwarding. + # Default value: false + # config.ssh.forward_agent = true + + # Share an additional folder to the guest VM. The first argument is + # the path on the host to the actual folder. The second argument is + # the path on the guest to mount the folder. And the optional third + # argument is a set of non-required options. + # config.vm.synced_folder "../data", "/vagrant_data" + + # Provider-specific configuration so you can fine-tune various + # backing providers for Vagrant. These expose provider-specific options. + # Example for VirtualBox: + # + config.vm.provider :virtualbox do |vb| + # # Don't boot with headless mode + vb.gui = false + # + # # Use VBoxManage to customize the VM. For example to change memory: + vb.customize ["modifyvm", :id, "--memory", "3072"] + end + # + # View the documentation for the provider you're using for more + # information on available options. + + # Enable provisioning with Puppet stand alone. Puppet manifests + # are contained in a directory path relative to this Vagrantfile. + # You will need to create the manifests directory and a manifest in + # the file precise32.pp in the manifests_path directory. + # + # An example Puppet manifest to provision the message of the day: + # + # # group { "puppet": + # # ensure => "present", + # # } + # # + # # File { owner => 0, group => 0, mode => 0644 } + # # + # # file { '/etc/motd': + # # content => "Welcome to your Vagrant-built virtual machine! + # # Managed by Puppet.\n" + # # } + # + # config.vm.provision :puppet do |puppet| + # puppet.manifests_path = "manifests" + # puppet.manifest_file = "site.pp" + # end + + # Enable provisioning with chef solo, specifying a cookbooks path, roles + # path, and data_bags path (all relative to this Vagrantfile), and adding + # some recipes and/or roles. + # + # config.vm.provision :chef_solo do |chef| + # chef.cookbooks_path = "../my-recipes/cookbooks" + # chef.roles_path = "../my-recipes/roles" + # chef.data_bags_path = "../my-recipes/data_bags" + # chef.add_recipe "mysql" + # chef.add_role "web" + # + # # You may also specify custom JSON attributes: + # chef.json = { :mysql_password => "foo" } + # end + + # Enable provisioning with chef server, specifying the chef server URL, + # and the path to the validation key (relative to this Vagrantfile). + # + # The Opscode Platform uses HTTPS. Substitute your organization for + # ORGNAME in the URL and validation key. + # + # If you have your own Chef Server, use the appropriate URL, which may be + # HTTP instead of HTTPS depending on your configuration. Also change the + # validation key to validation.pem. + # + # config.vm.provision :chef_client do |chef| + # chef.chef_server_url = "https://api.opscode.com/organizations/ORGNAME" + # chef.validation_key_path = "ORGNAME-validator.pem" + # end + # + # If you're using the Opscode platform, your validator client is + # ORGNAME-validator, replacing ORGNAME with your organization name. + # + # If you have your own Chef Server, the default validation client name is + # chef-validator, unless you changed the configuration. + # + # chef.validation_client_name = "ORGNAME-validator" +end diff --git a/integration-test/config/cluster.xml b/integration-test/config/cluster.xml new file mode 100644 index 00000000000..d37e6e6c078 --- /dev/null +++ b/integration-test/config/cluster.xml @@ -0,0 +1,96 @@ + + + + + + + /var/log/storm/${logfile.name} + + /var/log/storm/${logfile.name}.%i + 1 + 9 + + + + 100MB + + + + %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n + + + + + /var/log/storm/access.log + + /var/log/storm/access.log.%i + 1 + 9 + + + + 100MB + + + + %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n + + + + + /var/log/storm/metrics.log + + metrics.log.%i + 1 + 9 + + + + 2MB + + + + %d %-8r %m%n + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/integration-test/config/common.sh b/integration-test/config/common.sh new file mode 100644 index 00000000000..372c1a2bbc5 --- /dev/null +++ b/integration-test/config/common.sh @@ -0,0 +1,5 @@ +USER_SCRIPT="user-script.sh" +[[ -f $USER_SCRIPT ]] && echo "Running ${USER_SCRIPT}" && bash ${USER_SCRIPT} || echo "${USER_SCRIPT} not found/executed, continuing." +#apt-get update +#apt-get --yes remove openjdk-6-jre-headless +#apt-get --yes install openjdk-7-jdk diff --git a/integration-test/config/config-supervisord.sh b/integration-test/config/config-supervisord.sh new file mode 100644 index 00000000000..1a286b1912c --- /dev/null +++ b/integration-test/config/config-supervisord.sh @@ -0,0 +1,5 @@ +echo [program:storm-$1] | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf +echo command=storm $1 | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf +echo directory=/home/storm | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf +echo autorestart=true | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf +echo user=storm | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf \ No newline at end of file diff --git a/integration-test/config/etc-hosts b/integration-test/config/etc-hosts new file mode 100644 index 00000000000..597d9813d16 --- /dev/null +++ b/integration-test/config/etc-hosts @@ -0,0 +1,5 @@ +127.0.0.1 localhost localhost +192.168.50.3 zookeeper zookeeper +192.168.50.4 nimbus nimbus +192.168.50.5 supervisor1 supervisor1 +192.168.50.6 supervisor2 supervisor2 diff --git a/integration-test/config/install-storm.sh b/integration-test/config/install-storm.sh new file mode 100644 index 00000000000..20391e1b718 --- /dev/null +++ b/integration-test/config/install-storm.sh @@ -0,0 +1,23 @@ +# $1 is the storm binary zip file +SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" + +groupadd storm +useradd --gid storm --home-dir /home/storm --create-home --shell /bin/bash storm + +unzip -o "$1" -d /usr/share/ +chown -R storm:storm /usr/share/apache-storm* +ln -s /usr/share/apache-storm* /usr/share/storm +ln -s /usr/share/storm/bin/storm /usr/bin/storm + +mkdir /etc/storm +chown storm:storm /etc/storm + +rm /usr/share/storm/conf/storm.yaml +cp "${SCRIPT_DIR}/storm.yaml" /usr/share/storm/conf/ +cp "${SCRIPT_DIR}/cluster.xml" /usr/share/storm/logback/ +ln -s /usr/share/storm/conf/storm.yaml /etc/storm/storm.yaml + +mkdir /var/log/storm +chown storm:storm /var/log/storm + +#sed -i 's/${storm.home}\/logs/\/var\/log\/storm/g' /usr/share/storm/logback/cluster.xml diff --git a/integration-test/config/install-zookeeper.sh b/integration-test/config/install-zookeeper.sh new file mode 100644 index 00000000000..45d186bee56 --- /dev/null +++ b/integration-test/config/install-zookeeper.sh @@ -0,0 +1,4 @@ +apt-get --yes install zookeeper=3.3.5* zookeeperd=3.3.5* +service zookeeper stop +echo maxClientCnxns=200 >> /etc/zookeeper/conf/zoo.cfg +service zookeeper start diff --git a/integration-test/config/start-supervisord.sh b/integration-test/config/start-supervisord.sh new file mode 100644 index 00000000000..37b4cf8c5ef --- /dev/null +++ b/integration-test/config/start-supervisord.sh @@ -0,0 +1 @@ +/etc/init.d/supervisor start \ No newline at end of file diff --git a/integration-test/config/storm.yaml b/integration-test/config/storm.yaml new file mode 100644 index 00000000000..3a9df98f01d --- /dev/null +++ b/integration-test/config/storm.yaml @@ -0,0 +1,17 @@ + +storm.zookeeper.servers: + - "node1" + +nimbus.seeds: ["node1"] + +# netty transport +storm.messaging.transport: "org.apache.storm.messaging.netty.Context" +storm.messaging.netty.buffer_size: 16384 +storm.messaging.netty.max_retries: 10 +storm.messaging.netty.min_wait_ms: 1000 +storm.messaging.netty.max_wait_ms: 5000 + +drpc.servers: + - "node1" + +supervisor.slots.ports: [6700, 6701, 6702, 6703, 6704, 6705, 6706, 6707, 6708, 6709] diff --git a/integration-test/config/user-script.sh b/integration-test/config/user-script.sh new file mode 100644 index 00000000000..0dcf94dfad0 --- /dev/null +++ b/integration-test/config/user-script.sh @@ -0,0 +1 @@ +apt-get --yes install emacs tmux diff --git a/integration-test/pom.xml b/integration-test/pom.xml new file mode 100755 index 00000000000..50c90359d8b --- /dev/null +++ b/integration-test/pom.xml @@ -0,0 +1,250 @@ + + + + 4.0.0 + + storm-integration-test + org.apache.storm + storm-integration-test + 0.3 + jar + + + UTF-8 + + provided + 1.0.1.2.5.0.0-781 + true + false + /etc/storm/conf + /etc/hadoop/conf + + + + + intellij + + compile + true + src/test/resources/storm-conf/ + src/main/resources/hadoop-conf/ + + + + + + + + true + + + false + + central + http://repo1.maven.org/maven2/ + + + + true + + + true + + clojars + https://clojars.org/repo/ + + + hortonworks + http://nexus-private.hortonworks.com/nexus/content/groups/public/ + + + + + + org.testng + testng + 6.8.5 + test + + + joda-time + joda-time + 2.3 + + + org.seleniumhq.selenium + selenium-firefox-driver + 2.45.0 + test + + + org.seleniumhq.selenium + selenium-support + 2.45.0 + test + + + org.apache.storm + storm-core + ${storm.version} + provided + + + org.apache.storm + storm-solr + ${storm.version} + + + org.apache.storm + storm-starter + ${storm.version} + + + org.apache.storm + storm-hdfs + ${storm.version} + + + + + src/main/java + src/test/java + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + + org.apache.maven.plugins + maven-resources-plugin + 2.6 + + + org.apache.maven.plugins + maven-surefire-plugin + 2.18.1 + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.1 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-resources-plugin + 2.6 + + UTF-8 + + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${redirectTestOutputToFile} + 1C + -Xmx1024m + + + listener + com.hortonworks.storm.st.meta.TestngListener + + + + ${regression.downloadWorkerLogs} + + + + ${storm.conf.dir} + ${hadoop.conf.dir} + ${extra.classpath.1} + ${extra.classpath.2} + ${extra.classpath.3} + ${extra.classpath.4} + ${extra.classpath.5} + ${extra.classpath.6} + + + + + + org.apache.maven.plugins + maven-shade-plugin + + true + + + *:* + + META-INF/*.SF + META-INF/*.sf + META-INF/*.DSA + META-INF/*.dsa + META-INF/*.RSA + META-INF/*.rsa + META-INF/*.EC + META-INF/*.ec + META-INF/MSFTSIG.SF + META-INF/MSFTSIG.RSA + + + + + + + package + + shade + + + + + + + + + + + + + + + ${hadoop.conf.dir} + + + + diff --git a/integration-test/run-it.sh b/integration-test/run-it.sh new file mode 100755 index 00000000000..eed6025743d --- /dev/null +++ b/integration-test/run-it.sh @@ -0,0 +1,81 @@ +#!/usr/bin/env bash +# -*- compile-command: "cd config/ && vagrant destroy -f; vagrant up" -*- +# +# 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. +# +set -x +set -e +SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +echo SCRIPT_DIR="${SCRIPT_DIR}" +STORM_SRC_DIR=$(dirname ${SCRIPT_DIR}) +echo SCRIPT_SRC_DIR="${SCRIPT_SRC_DIR}" +function die() { + echo $* + exit 1 +} +function list_storm_processes() { + (ps -ef | grep -i -e zookeeper | grep -v grep) && (ps -ef | grep -i -e storm.home | grep -v grep) +} + +list_storm_processes || true +# increasing swap space so we can run lots of workers +sudo dd if=/dev/zero of=/swapfile.img bs=8192 count=1M +sudo mkswap /swapfile.img +sudo swapon /swapfile.img + +if [[ "${USER}" == "vagrant" ]]; then # install oracle jdk8 + sudo apt-get update + sudo apt-get -y install python-software-properties + sudo apt-add-repository -y ppa:webupd8team/java + sudo apt-get update + echo "oracle-java8-installer shared/accepted-oracle-license-v1-1 select true" | sudo debconf-set-selections + sudo apt-get install -y oracle-java8-installer + sudo apt-get -y install maven + java -version + mvn --version + export MAVEN_OPTS="-Xmx3000m" +else + ( while true; do echo "heartbeat"; sleep 300; done ) & #heartbeat needed by travis ci + (cd ${STORM_SRC_DIR} && mvn clean install -DskipTests=true) || die "maven install command failed" + (cd ${STORM_SRC_DIR}/storm-dist/binary && mvn package -Dgpg.skip=true) || die "maven package command failed" +fi +storm_binary_zip=$(find ${STORM_SRC_DIR}/storm-dist -iname '*.zip') +storm_binary_name=$(basename ${storm_binary_zip}) +export STORM_VERSION=$(grep -oPe '\d.*(?=.zip)' <<<${storm_binary_name}) +echo "Using storm version:" ${STORM_VERSION} + +# setup storm cluster +list_storm_processes || true +sudo bash ${SCRIPT_DIR}/config/common.sh +sudo bash ${SCRIPT_DIR}/config/install-zookeeper.sh +sudo bash ${SCRIPT_DIR}/config/install-storm.sh $storm_binary_zip +function start_storm_process() { + echo starting: storm $1 + sudo su storm -c "cd /home/storm && storm $1" & +} +start_storm_process nimbus +start_storm_process ui +start_storm_process supervisor +start_storm_process logviewer +#start_storm_process drpc +pushd "${SCRIPT_DIR}" +mvn clean package -DskipTests -Dstorm.version=${STORM_VERSION} +for i in {1..20} ; do + list_storm_processes && break + sleep 6 +done +list_storm_processes +mvn test -DfailIfNoTests=false -Dstorm.version=${STORM_VERSION} -Dui.url=http://localhost:8744 diff --git a/integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java b/integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java new file mode 100644 index 00000000000..e7a901a9a94 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java @@ -0,0 +1,95 @@ +/* + * 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.hortonworks.storm; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.testing.TestWordSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; + +import java.util.Map; + +/** + * This is a basic example of a Storm topology. + */ +public class ExclamationTopology { + + public static final String WORD = "word"; + public static final String EXCLAIM_1 = "exclaim1"; + public static final String EXCLAIM_2 = "exclaim2"; + + public static class ExclamationBolt extends BaseRichBolt { + OutputCollector _collector; + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + @Override + public void execute(Tuple tuple) { + _collector.emit(tuple, new Values(tuple.getString(0) + "!!!")); + _collector.ack(tuple); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + + } + + public static void main(String[] args) throws Exception { + StormTopology topology = getStormTopology(); + + Config conf = new Config(); + conf.setDebug(true); + + if (args != null && args.length > 0) { + conf.setNumWorkers(3); + + StormSubmitter.submitTopologyWithProgressBar(args[0], conf, topology); + } + else { + + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", conf, topology); + Utils.sleep(10000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + + public static StormTopology getStormTopology() { + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(WORD, new TestWordSpout(), 10); + builder.setBolt(EXCLAIM_1, new ExclamationTopology.ExclamationBolt(), 3).shuffleGrouping(WORD); + builder.setBolt(EXCLAIM_2, new ExclamationTopology.ExclamationBolt(), 2).shuffleGrouping(EXCLAIM_1); + return builder.createTopology(); + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java b/integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java new file mode 100644 index 00000000000..01503b4a1fc --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java @@ -0,0 +1,42 @@ +/* + * 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.hortonworks.storm.debug; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.net.URLClassLoader; + +/** + * Created by temp on 6/3/16. + */ +public class DebugHelper { + private static final Logger LOG = LoggerFactory.getLogger(DebugHelper.class); + + public static void printClassPath() { + URL[] urls = getClassPaths(); + LOG.info("classpath:" + StringUtils.join(urls, ':')); + } + + public static URL[] getClassPaths() { + ClassLoader cl = ClassLoader.getSystemClassLoader(); + return ((URLClassLoader)cl).getURLs(); + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java new file mode 100644 index 00000000000..fc8d1b070c0 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java @@ -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.hortonworks.storm.st.topology; + +import org.apache.storm.generated.StormTopology; + +import java.util.List; + +/** + * Created by temp on 5/3/16. + */ +public interface TestableTopology { + String DUMMY_FIELD = "dummy"; + List getExpectedOutput(); + StormTopology newTopology(); + String getBoltName(); + String getSpoutName(); +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java new file mode 100644 index 00000000000..462705389cc --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java @@ -0,0 +1,167 @@ +/* + * 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.hortonworks.storm.st.topology.window; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.google.gson.GsonBuilder; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; +import org.hortonworks.storm.st.topology.TestableTopology; +import org.hortonworks.storm.st.topology.window.data.TimeData; +import org.hortonworks.storm.st.utils.StringDecorator; +import org.hortonworks.storm.st.utils.TimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * Computes sliding window sum + */ +public class SlidingTimeCorrectness implements TestableTopology { + private static final Logger LOG = LoggerFactory.getLogger(SlidingTimeCorrectness.class); + private final int windowSec; + private final int slideSec; + private final String spoutName; + private final String boltName; + + public SlidingTimeCorrectness(int windowSec, int slideSec) { + this.windowSec = windowSec; + this.slideSec = slideSec; + final String prefix = this.getClass().getSimpleName() + "-winSec" + windowSec + "slideSec" + slideSec; + spoutName = prefix + "IncrementingSpout"; + boltName = prefix + "VerificationBolt"; + } + + public String getBoltName() { + return boltName; + } + + public String getSpoutName() { + return spoutName; + } + + public StormTopology newTopology() { + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(getSpoutName(), new IncrementingSpout(), 2); + builder.setBolt(getBoltName(), + new VerificationBolt() + .withWindow(new BaseWindowedBolt.Duration(windowSec, TimeUnit.SECONDS), + new BaseWindowedBolt.Duration(slideSec, TimeUnit.SECONDS)) + .withTimestampField(TimeData.getTimestampFieldName()) + .withLag(new BaseWindowedBolt.Duration(10, TimeUnit.SECONDS)), + 1) + .globalGrouping(getSpoutName()); + return builder.createTopology(); + } + + public List getExpectedOutput() { + return Lists.newArrayList( + StringDecorator.decorate(getBoltName(), "tuplesInWindow.size() = " + windowSec), + StringDecorator.decorate(getBoltName(), "newTuples.size() = " + slideSec), + StringDecorator.decorate(getBoltName(), "expiredTuples.size() = " + slideSec) + ); + } + + public static class IncrementingSpout extends BaseRichSpout { + private static final Logger LOG = LoggerFactory.getLogger(IncrementingSpout.class); + private SpoutOutputCollector collector; + private static int currentNum; + private static Random rng = new Random(); + private String componentId; + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(TimeData.getFields()); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void nextTuple() { + TimeUtil.sleepMilliSec(rng.nextInt(800)); + currentNum++; + TimeData data = TimeData.newData(currentNum); + final Values tuple = data.getValues(); + collector.emit(tuple); + LOG.info(StringDecorator.decorate(componentId, data.toString())); + } + + @Override + public void ack(Object msgId) { + LOG.info("Received ACK for msgId : " + msgId); + } + + @Override + public void fail(Object msgId) { + LOG.info("Received FAIL for msgId : " + msgId); + } + } + + public static class VerificationBolt extends BaseWindowedBolt { + private OutputCollector collector; + private String componentId; + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void execute(TupleWindow inputWindow) { + List tuplesInWindow = inputWindow.get(); + List newTuples = inputWindow.getNew(); + List expiredTuples = inputWindow.getExpired(); + LOG.info("tuplesInWindow.size() = " + tuplesInWindow.size()); + LOG.info("newTuples.size() = " + newTuples.size()); + LOG.info("expiredTuples.size() = " + expiredTuples.size()); + Collection dataInWindow = Collections2.transform(tuplesInWindow, new Function() { + @Nullable + @Override + public TimeData apply(@Nullable Tuple input) { + return TimeData.fromTuple(input); + } + }); + final String jsonData = TimeData.toString(dataInWindow); + LOG.info(StringDecorator.decorate(componentId, jsonData)); + collector.emit(new Values("dummyValue")); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(DUMMY_FIELD)); + } + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java new file mode 100644 index 00000000000..016478cba05 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java @@ -0,0 +1,156 @@ +/* + * 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.hortonworks.storm.st.topology.window; + +import com.google.common.collect.Lists; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; +import org.hortonworks.storm.st.topology.TestableTopology; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.hortonworks.storm.st.utils.StringDecorator; +import org.hortonworks.storm.st.utils.TimeUtil; + +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +/** + * Computes sliding window sum + */ +public class SlidingWindowCorrectness implements TestableTopology { + private static final Logger LOG = LoggerFactory.getLogger(SlidingWindowCorrectness.class); + private static final String NUMBER_FIELD = "number"; + private static final String STRING_FIELD = "numAsStr"; + private final int windowSize; + private final int slideSize; + private final String spoutName; + private final String boltName; + + public SlidingWindowCorrectness(int windowSize, int slideSize) { + this.windowSize = windowSize; + this.slideSize = slideSize; + final String prefix = this.getClass().getSimpleName() + "-winSize" + windowSize + "slideSize" + slideSize; + spoutName = prefix + "IncrementingSpout"; + boltName = prefix + "VerificationBolt"; + } + + public String getBoltName() { + return boltName; + } + + public String getSpoutName() { + return spoutName; + } + + public StormTopology newTopology() { + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(getSpoutName(), new IncrementingSpout(), 1); + builder.setBolt(getBoltName(), + new VerificationBolt() + .withLag(new BaseWindowedBolt.Duration(10, TimeUnit.SECONDS)) + .withWindow(new BaseWindowedBolt.Count(windowSize), new BaseWindowedBolt.Count(slideSize)), + 1) + .shuffleGrouping(getSpoutName()); + return builder.createTopology(); + } + + public List getExpectedOutput() { + return Lists.newArrayList( + StringDecorator.decorate(getBoltName(), "tuplesInWindow.size() = " + windowSize), + StringDecorator.decorate(getBoltName(), "newTuples.size() = " + slideSize), + StringDecorator.decorate(getBoltName(), "expiredTuples.size() = " + slideSize) + ); + } + + public static class IncrementingSpout extends BaseRichSpout { + private static final Logger LOG = LoggerFactory.getLogger(IncrementingSpout.class); + private SpoutOutputCollector collector; + private static int currentNum; + private static Random rng = new Random(); + private String componentId; + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(NUMBER_FIELD, STRING_FIELD)); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void nextTuple() { + TimeUtil.sleepMilliSec(rng.nextInt(10)); + currentNum++; + final String numAsStr = "str(" + currentNum + ")str"; + final Values tuple = new Values(currentNum, numAsStr); + LOG.info(StringDecorator.decorate(componentId, tuple.toString())); + collector.emit(tuple, currentNum); + } + + @Override + public void ack(Object msgId) { + LOG.info("Received ACK for msgId : " + msgId); + } + + @Override + public void fail(Object msgId) { + LOG.info("Received FAIL for msgId : " + msgId); + } + } + + public static class VerificationBolt extends BaseWindowedBolt { + private OutputCollector collector; + private String componentId; + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void execute(TupleWindow inputWindow) { + List tuplesInWindow = inputWindow.get(); + List newTuples = inputWindow.getNew(); + List expiredTuples = inputWindow.getExpired(); + LOG.info("tuplesInWindow.size() = " + tuplesInWindow.size()); + LOG.info("newTuples.size() = " + newTuples.size()); + LOG.info("expiredTuples.size() = " + expiredTuples.size()); + LOG.info(StringDecorator.decorate(componentId, "tuplesInWindow = " + tuplesInWindow.toString())); + collector.emit(new Values("dummyValue")); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(DUMMY_FIELD)); + } + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java new file mode 100644 index 00000000000..da54ef388a6 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java @@ -0,0 +1,164 @@ +/* + * 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.hortonworks.storm.st.topology.window; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.google.gson.GsonBuilder; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; +import org.hortonworks.storm.st.topology.TestableTopology; +import org.hortonworks.storm.st.topology.window.data.TimeData; +import org.hortonworks.storm.st.utils.StringDecorator; +import org.hortonworks.storm.st.utils.TimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.*; +import java.util.concurrent.TimeUnit; + +/** + * Computes sliding window sum + */ +public class TumblingTimeCorrectness implements TestableTopology { + private static final Logger LOG = LoggerFactory.getLogger(TumblingTimeCorrectness.class); + private final int tumbleSec; + private final String spoutName; + private final String boltName; + + public TumblingTimeCorrectness(int timbleSec) { + this.tumbleSec = timbleSec; + final String prefix = this.getClass().getSimpleName() + "-timbleSec" + timbleSec; + spoutName = prefix + "IncrementingSpout"; + boltName = prefix + "VerificationBolt"; + } + + public String getBoltName() { + return boltName; + } + + public String getSpoutName() { + return spoutName; + } + + public StormTopology newTopology() { + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(getSpoutName(), new IncrementingSpout(), 2); + builder.setBolt(getBoltName(), + new VerificationBolt() + .withTumblingWindow(new BaseWindowedBolt.Duration(tumbleSec, TimeUnit.SECONDS)) + .withLag(new BaseWindowedBolt.Duration(10, TimeUnit.SECONDS)) + .withTimestampField(TimeData.getTimestampFieldName()), + 1) + .globalGrouping(getSpoutName()); + return builder.createTopology(); + } + + public List getExpectedOutput() { + return Lists.newArrayList( + StringDecorator.decorate(getBoltName(), "tuplesInWindow.size() = " + tumbleSec), + StringDecorator.decorate(getBoltName(), "newTuples.size() = " + tumbleSec), + StringDecorator.decorate(getBoltName(), "expiredTuples.size() = " + tumbleSec) + ); + } + + public static class IncrementingSpout extends BaseRichSpout { + private static final Logger LOG = LoggerFactory.getLogger(IncrementingSpout.class); + private SpoutOutputCollector collector; + private static int currentNum; + private static Random rng = new Random(); + private String componentId; + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(TimeData.getFields()); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void nextTuple() { + TimeUtil.sleepMilliSec(rng.nextInt(800)); + currentNum++; + TimeData data = TimeData.newData(currentNum); + final Values tuple = data.getValues(); + collector.emit(tuple); + LOG.info(StringDecorator.decorate(componentId, data.toString())); + } + + @Override + public void ack(Object msgId) { + LOG.info("Received ACK for msgId : " + msgId); + } + + @Override + public void fail(Object msgId) { + LOG.info("Received FAIL for msgId : " + msgId); + } + } + + public static class VerificationBolt extends BaseWindowedBolt { + private OutputCollector collector; + private String componentId; + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void execute(TupleWindow inputWindow) { + List tuplesInWindow = inputWindow.get(); + List newTuples = inputWindow.getNew(); + List expiredTuples = inputWindow.getExpired(); + LOG.info("tuplesInWindow.size() = " + tuplesInWindow.size()); + LOG.info("newTuples.size() = " + newTuples.size()); + LOG.info("expiredTuples.size() = " + expiredTuples.size()); + Collection dataInWindow = Collections2.transform(tuplesInWindow, new Function() { + @Nullable + @Override + public TimeData apply(@Nullable Tuple input) { + return TimeData.fromTuple(input); + } + }); + final String jsonData = TimeData.toString(dataInWindow); + LOG.info(StringDecorator.decorate(componentId, jsonData)); + collector.emit(new Values("dummyValue")); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(DUMMY_FIELD)); + } + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java new file mode 100644 index 00000000000..2ba2895254a --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java @@ -0,0 +1,153 @@ +/* + * 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.hortonworks.storm.st.topology.window; + +import com.google.common.collect.Lists; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; +import org.hortonworks.storm.st.topology.TestableTopology; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.hortonworks.storm.st.utils.StringDecorator; +import org.hortonworks.storm.st.utils.TimeUtil; + +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +/** + * Computes sliding window sum + */ +public class TumblingWindowCorrectness implements TestableTopology { + private static final Logger LOG = LoggerFactory.getLogger(TumblingWindowCorrectness.class); + private static final String NUMBER_FIELD = "number"; + private static final String STRING_FIELD = "numAsStr"; + private final int tumbleSize; + private final String spoutName; + private final String boltName; + + public TumblingWindowCorrectness(final int tumbleSize) { + this.tumbleSize = tumbleSize; + final String prefix = this.getClass().getSimpleName() + "-tubleSize" + tumbleSize; + spoutName = prefix + "IncrementingSpout"; + boltName = prefix + "VerificationBolt"; + } + + public String getBoltName() { + return boltName; + } + + public String getSpoutName() { + return spoutName; + } + + public StormTopology newTopology() { + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(getSpoutName(), new IncrementingSpout(), 1); + builder.setBolt(getBoltName(), + new VerificationBolt() + .withLag(new BaseWindowedBolt.Duration(10, TimeUnit.SECONDS)) + .withTumblingWindow(new BaseWindowedBolt.Count(tumbleSize)), 1) + .shuffleGrouping(getSpoutName()); + return builder.createTopology(); + } + + public List getExpectedOutput() { + return Lists.newArrayList( + StringDecorator.decorate(getBoltName(), "tuplesInWindow.size() = " + tumbleSize), + StringDecorator.decorate(getBoltName(), "newTuples.size() = " + tumbleSize), + StringDecorator.decorate(getBoltName(), "expiredTuples.size() = " + tumbleSize) + ); + } + + public static class IncrementingSpout extends BaseRichSpout { + private static final Logger LOG = LoggerFactory.getLogger(IncrementingSpout.class); + private SpoutOutputCollector collector; + private static int currentNum; + private static Random rng = new Random(); + private String componentId; + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(NUMBER_FIELD, STRING_FIELD)); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void nextTuple() { + TimeUtil.sleepMilliSec(rng.nextInt(10)); + currentNum++; + final String numAsStr = "str(" + currentNum + ")str"; + final Values tuple = new Values(currentNum, numAsStr); + LOG.info(StringDecorator.decorate(componentId, tuple.toString())); + collector.emit(tuple, currentNum); + } + + @Override + public void ack(Object msgId) { + LOG.info("Received ACK for msgId : " + msgId); + } + + @Override + public void fail(Object msgId) { + LOG.info("Received FAIL for msgId : " + msgId); + } + } + + public static class VerificationBolt extends BaseWindowedBolt { + private OutputCollector collector; + private String componentId; + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + componentId = context.getThisComponentId(); + this.collector = collector; + } + + @Override + public void execute(TupleWindow inputWindow) { + List tuplesInWindow = inputWindow.get(); + List newTuples = inputWindow.getNew(); + List expiredTuples = inputWindow.getExpired(); + LOG.info("tuplesInWindow.size() = " + tuplesInWindow.size()); + LOG.info("newTuples.size() = " + newTuples.size()); + LOG.info("expiredTuples.size() = " + expiredTuples.size()); + LOG.info(StringDecorator.decorate(componentId, "tuplesInWindow = " + tuplesInWindow.toString())); + collector.emit(new Values("dummyValue")); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields(DUMMY_FIELD)); + } + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java new file mode 100644 index 00000000000..6d163faae8c --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java @@ -0,0 +1,25 @@ +/* + * 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.hortonworks.storm.st.topology.window.data; + +/** + * Created by temp on 5/23/16. + */ +public interface FromJson { + T fromJson(String jsonStr); +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java new file mode 100644 index 00000000000..154f2b4d5ef --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java @@ -0,0 +1,113 @@ +/* + * 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.hortonworks.storm.st.topology.window.data; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; + +import java.util.Collection; +import java.util.Date; + +/** + * Created by temp on 5/12/16. + */ +public class TimeData implements Comparable, FromJson { + public static final TimeData CLS = new TimeData(-1); + private static final String NUMBER_FIELD_NAME = "number"; + private static final String STRING_FIELD_NAME = "dateAsStr"; + private static final String TIMESTAMP_FIELD_NAME = "date"; + static final Gson gson = new GsonBuilder().setDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'").create(); + private final int num; + private final Date now; + private final long timestamp; + + private TimeData(int num) { + this(num, new Date()); + } + + private TimeData(int num, Date date) { + this.num = num; + this.now = date; + this.timestamp = date.getTime(); + } + + public static TimeData newData(int num) { + return new TimeData(num); + } + + public static TimeData fromTuple(Tuple tuple) { + return new TimeData(tuple.getIntegerByField(NUMBER_FIELD_NAME), new Date(tuple.getLongByField(TIMESTAMP_FIELD_NAME))); + } + + public TimeData fromJson(String jsonStr) { + return gson.fromJson(jsonStr, TimeData.class); + } + + public String toString() { + return gson.toJson(this); + } + + public static String toString(Collection elements) { + return gson.toJson(elements); + } + + public Values getValues() { + return new Values(num, now.toString(), timestamp); + } + + public static String getTimestampFieldName() { + return TIMESTAMP_FIELD_NAME; + } + + public Date getDate() { + return now; + } + + public static Fields getFields() { + return new Fields(NUMBER_FIELD_NAME, STRING_FIELD_NAME, TIMESTAMP_FIELD_NAME); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TimeData data = (TimeData) o; + + if (num != data.num) return false; + if (timestamp != data.timestamp) return false; + return now.equals(data.now); + + } + + @Override + public int hashCode() { + int result = num; + result = 31 * result + now.hashCode(); + result = 31 * result + (int) (timestamp ^ (timestamp >>> 32)); + return result; + } + + @Override + public int compareTo(TimeData o) { + return Long.compare(timestamp, o.timestamp); + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java new file mode 100644 index 00000000000..65d67ddd900 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java @@ -0,0 +1,94 @@ +/* + * 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.hortonworks.storm.st.topology.window.data; + +import com.google.common.base.Predicate; +import com.google.common.collect.Collections2; +import com.google.gson.reflect.TypeToken; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Created by temp on 5/23/16. + */ +public class TimeDataWindow extends ArrayList implements FromJson { + public static final TimeDataWindow CLS = new TimeDataWindow(); + private static final Type listType = new TypeToken>() {}.getType(); + + private TimeDataWindow() { + } + + private TimeDataWindow(List data) { + super(data); + } + + public static TimeDataWindow newInstance(Collection data) { + final List dataCopy = new ArrayList<>(data); + Collections.sort(dataCopy); + return new TimeDataWindow(dataCopy); + } + + public static TimeDataWindow newInstance(Collection data, Predicate predicate) { + return newInstance(Collections2.filter(data, predicate)); + } + + public static TimeDataWindow newInstance(Collection data, final DateTime fromDate, final DateTime toDate) { + return TimeDataWindow.newInstance(data, new Predicate() { + @Override + public boolean apply(@Nullable TimeData input) { + if (input == null) { + return false; + } + final DateTime inputDate = new DateTime(input.getDate()); + return inputDate.isAfter(fromDate) && inputDate.isBefore(toDate.plusMillis(1)); + } + }); + } + + public TimeData first() { + return get(0); + } + + public TimeData last() { + return get(size()-1); + } + + public String getDescription() { + final int size = size(); + if (size > 0) { + final TimeData first = first(); + final TimeData last = last(); + return "Total " + size + " items: " + first + " to " + last; + } + return "Total " + size + " items."; + } + + public TimeDataWindow fromJson(String jsonStr) { + final List dataList = TimeData.gson.fromJson(jsonStr, listType); + Collections.sort(dataList); + return TimeDataWindow.newInstance(dataList); + } + + +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java b/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java new file mode 100644 index 00000000000..c005d3784ff --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java @@ -0,0 +1,58 @@ +/* + * 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.hortonworks.storm.st.utils; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +/** + * Created by Raghav Kumar Gautam on 6/7/16. + */ +public class HdfsUtils { + public static String joinPaths(String basePath, String firstPart, String... restParts) { + final String separator = "/"; + List cleanParts = new ArrayList<>(); + String cleanBasePath = basePath.replaceFirst(separator + "$", ""); + cleanParts.add(cleanBasePath); + final String cleanFirstPart1 = firstPart.replaceFirst("^" + separator, "").replaceFirst(separator + "$", ""); + cleanParts.add(cleanFirstPart1); + for (String onePart : restParts) { + final String cleanPart = onePart.replaceFirst("^" + separator, "").replaceFirst(separator + "$", ""); + cleanParts.add(cleanPart); + } + return StringUtils.join(cleanParts, separator); + } + + public static FileSystem getFileSystem() { + Configuration conf = new Configuration(); + //conf.setBoolean("fs.hdfs.impl.disable.cache", true); + //conf.set("fs.default.name", "hdfs://" + hadoopURL); + try { + return FileSystem.get(conf); + } catch (IOException e) { + throw new RuntimeException("Can't proceed without FileSystem object.", e); + } + } + +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java b/integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java new file mode 100644 index 00000000000..b6a31b45b89 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java @@ -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.hortonworks.storm.st.utils; + +import org.apache.commons.lang.StringUtils; + +/** + * Created by temp on 5/3/16. + */ +public class StringDecorator { + + private static final String UNIQUE_PREFIX = "---bed91874d79720f7e324c43d49dba4ff---"; + + public static String decorate(String componentId, String decorate) { + return componentId + UNIQUE_PREFIX + decorate; + } + + public static boolean isDecorated(String str) { + return str != null && str.contains(UNIQUE_PREFIX); + } + + public static String[] split2(String decoratedString) { + return StringUtils.splitByWholeSeparator(decoratedString, UNIQUE_PREFIX, 2); + } +} diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java b/integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java new file mode 100644 index 00000000000..09c98c85e13 --- /dev/null +++ b/integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java @@ -0,0 +1,57 @@ +/* + * 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.hortonworks.storm.st.utils; + +import org.apache.commons.lang.exception.ExceptionUtils; +import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeUnit; + +/** + * Created by temp on 4/28/16. + */ +public class TimeUtil { + private static Logger log = LoggerFactory.getLogger(TimeUtil.class); + + public static void sleepSec(int sec) { + try { + TimeUnit.SECONDS.sleep(sec); + } catch (InterruptedException e) { + log.warn("Caught exception: " + ExceptionUtils.getFullStackTrace(e)); + } + } + public static void sleepMilliSec(int milliSec) { + try { + TimeUnit.MILLISECONDS.sleep(milliSec); + } catch (InterruptedException e) { + log.warn("Caught exception: " + ExceptionUtils.getFullStackTrace(e)); + } + } + + public static DateTime floor(DateTime dateTime, int sec) { + long modValue = dateTime.getMillis() % (1000 * sec); + return dateTime.minus(modValue); + } + + public static DateTime ceil(DateTime dateTime, int sec) { + long modValue = dateTime.getMillis() % (1000 * sec); + return dateTime.minus(modValue).plusSeconds(sec); + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java b/integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java new file mode 100644 index 00000000000..edf27d16679 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java @@ -0,0 +1,87 @@ +/* + * 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 com.hortonworks.storm.st; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.Lists; +import com.hortonworks.storm.st.helper.AbstractTest; +import com.hortonworks.storm.st.utils.AssertUtil; +import com.hortonworks.storm.st.wrapper.TopoWrap; +import org.hortonworks.storm.ExclamationTopology; +import org.apache.storm.generated.TopologyInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import org.hortonworks.storm.st.utils.TimeUtil; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; + +public final class DemoTest extends AbstractTest { + private static Logger log = LoggerFactory.getLogger(DemoTest.class); + private static Collection words = Lists.newArrayList("nathan", "mike", "jackson", "golda", "bertels"); + private static Collection exclaim2Oputput = Collections2.transform(words, new Function() { + @Nullable + @Override + public String apply(@Nullable String input) { + return input + "!!!!!!"; + } + }); + protected final String topologyName = this.getClass().getSimpleName(); + private TopoWrap topo; + + @Test + public void testExclamationTopology() throws Exception { + topo = new TopoWrap(cluster, topologyName, ExclamationTopology.getStormTopology()); + topo.submitSuccessfully(); + final int minExclaim2Emits = 500; + final int minSpountEmits = 10000; + for(int i = 0; i < 10; ++i) { + TopologyInfo topologyInfo = topo.getInfo(); + log.info(topologyInfo.toString()); + long wordSpoutEmittedCount = topo.getAllTimeEmittedCount(ExclamationTopology.WORD); + long exclaim1EmittedCount = topo.getAllTimeEmittedCount(ExclamationTopology.EXCLAIM_1); + long exclaim2EmittedCount = topo.getAllTimeEmittedCount(ExclamationTopology.EXCLAIM_2); + log.info("wordSpoutEmittedCount for spout 'word' = " + wordSpoutEmittedCount); + log.info("exclaim1EmittedCount = " + exclaim1EmittedCount); + log.info("exclaim2EmittedCount = " + exclaim2EmittedCount); + if (exclaim2EmittedCount > minExclaim2Emits || wordSpoutEmittedCount > minSpountEmits) { + break; + } + TimeUtil.sleepSec(6); + } + List boltUrls = topo.getLogUrls(ExclamationTopology.WORD); + log.info(boltUrls.toString()); + final String actualOutput = topo.getLogs(ExclamationTopology.EXCLAIM_2); + for (String oneExpectedOutput : exclaim2Oputput) { + Assert.assertTrue(actualOutput.contains(oneExpectedOutput), "Couldn't find " + oneExpectedOutput + " in urls"); + } + } + + @AfterMethod + public void cleanup() throws Exception { + if (topo != null) { + topo.killQuietly(); + } + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java b/integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java new file mode 100644 index 00000000000..c16979a1762 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java @@ -0,0 +1,30 @@ +/* + * 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 com.hortonworks.storm.st.helper; + +import com.hortonworks.storm.st.wrapper.StormCluster; + +/** + * Created by temp on 5/2/16. + */ +public abstract class AbstractTest { + protected final StormCluster cluster = new StormCluster(); + static { + System.setProperty("user.timezone", "UTC"); + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java b/integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java new file mode 100644 index 00000000000..12529b12298 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java @@ -0,0 +1,103 @@ +/* + * 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 com.hortonworks.storm.st.meta; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.FilenameUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.log4j.Logger; +import org.apache.log4j.NDC; +import org.openqa.selenium.OutputType; +import org.openqa.selenium.TakesScreenshot; +import org.testng.IExecutionListener; +import org.testng.ITestContext; +import org.testng.ITestListener; +import org.testng.ITestResult; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; + +/** + * Testng listener class. This is useful for things that are applicable to all the tests as well + * taking actions that depend on test results. + */ +public class TestngListener implements ITestListener, IExecutionListener { + private static final Logger LOGGER = Logger.getLogger(TestngListener.class); + private final String hr = StringUtils.repeat("-", 100); + + private enum RunResult {SUCCESS, FAILED, SKIPPED, TestFailedButWithinSuccessPercentage } + + @Override + public void onTestStart(ITestResult result) { + LOGGER.info(hr); + LOGGER.info( + String.format("Testing going to start for: %s.%s(%s)", result.getTestClass().getName(), + result.getName(), Arrays.toString(result.getParameters()))); + NDC.push(result.getName()); + } + + private void endOfTestHook(ITestResult result, RunResult outcome) { + LOGGER.info( + String.format("Testing going to end for: %s.%s(%s) ----- Status: %s", result.getTestClass().getName(), + result.getName(), Arrays.toString(result.getParameters()), outcome)); + NDC.pop(); + LOGGER.info(hr); + } + + @Override + public void onTestSuccess(ITestResult result) { + endOfTestHook(result, RunResult.SUCCESS); + } + + @Override + public void onTestFailure(ITestResult result) { + endOfTestHook(result, RunResult.FAILED); + + LOGGER.info(ExceptionUtils.getStackTrace(result.getThrowable())); + LOGGER.info(hr); + } + + @Override + public void onTestSkipped(ITestResult result) { + endOfTestHook(result, RunResult.SKIPPED); + } + + @Override + public void onTestFailedButWithinSuccessPercentage(ITestResult result) { + endOfTestHook(result, RunResult.TestFailedButWithinSuccessPercentage); + } + + @Override + public void onStart(ITestContext context) { + } + + @Override + public void onFinish(ITestContext context) { + } + + @Override + public void onExecutionStart() { + } + + @Override + public void onExecutionFinish() { + } + +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java b/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java new file mode 100644 index 00000000000..9e25dd4bc25 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java @@ -0,0 +1,189 @@ +/* + * 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 com.hortonworks.storm.st.tests.window; + +import com.hortonworks.storm.st.helper.AbstractTest; +import com.hortonworks.storm.st.utils.AssertUtil; +import com.hortonworks.storm.st.wrapper.LogData; +import com.hortonworks.storm.st.wrapper.TopoWrap; +import org.apache.storm.thrift.TException; +import org.hortonworks.storm.st.topology.TestableTopology; +import org.hortonworks.storm.st.topology.window.SlidingTimeCorrectness; +import org.hortonworks.storm.st.topology.window.SlidingWindowCorrectness; +import org.hortonworks.storm.st.topology.window.data.TimeData; +import org.hortonworks.storm.st.topology.window.data.TimeDataWindow; +import org.hortonworks.storm.st.utils.TimeUtil; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.net.MalformedURLException; +import java.util.List; + +public final class SlidingWindowTest extends AbstractTest { + private static Logger log = LoggerFactory.getLogger(SlidingWindowTest.class); + private TopoWrap topo; + + @DataProvider + public static Object[][] generateCountWindows() { + final Object[][] objects = new Object[][]{ + {-1, 10}, + {10, -1}, + {0, 10}, + {10, 0}, + {0, 0}, + {-1, -1}, + {5, 10}, + {1, 1}, + {10, 5}, + {100, 10}, + {100, 100}, + {200, 100}, + {500, 100}, + }; + return objects; + } + + @Test(dataProvider = "generateCountWindows") + public void testWindowCount(int windowSize, int slideSize) throws Exception { + final SlidingWindowCorrectness testable = new SlidingWindowCorrectness(windowSize, slideSize); + final String topologyName = this.getClass().getSimpleName() + "w" + windowSize + "s" + slideSize; + if (windowSize <= 0 || slideSize <= 0) { + try { + testable.newTopology(); + Assert.fail("Expected IllegalArgumentException was not thrown."); + } catch (IllegalArgumentException ignore) { + return; + } + } + topo = new TopoWrap(cluster, topologyName, testable.newTopology()); + runAndVerifyCount(windowSize, slideSize, testable, topo); + } + + static void runAndVerifyCount(int windowSize, int slideSize, TestableTopology testable, TopoWrap topo) throws TException, MalformedURLException { + topo.submitSuccessfully(); + final int minSpoutEmits = 1000 + windowSize; + final int minBoltEmits = 5; + String boltName = testable.getBoltName(); + String spoutName = testable.getSpoutName(); + topo.waitForProgress(minSpoutEmits, spoutName, 180); + topo.waitForProgress(minBoltEmits, boltName, 180); + List boltUrls = topo.getLogUrls(boltName); + log.info(boltUrls.toString()); + final List allBoltData = topo.getLogData(boltName); + final List allSpoutData = topo.getLogData(spoutName); + Assert.assertTrue(allBoltData.size() >= minBoltEmits, + "Expecting min " + minBoltEmits + " bolt emits, found: " + allBoltData.size() + " \n\t" + allBoltData); + final int numberOfWindows = allBoltData.size() - windowSize / slideSize; + for(int i = 0; i < numberOfWindows; ++i ) { + log.info("Comparing window: " + (i + 1) + " of " + numberOfWindows); + final int toIndex = (i + 1) * slideSize; + final int fromIndex = toIndex - windowSize; + final int positiveFromIndex = fromIndex > 0 ? fromIndex : 0; + final List windowData = allSpoutData.subList(positiveFromIndex, toIndex); + final String actualString = allBoltData.get(i).toString(); + for (LogData oneLog : windowData) { + final String logStr = oneLog.getData(); + Assert.assertTrue(actualString.contains(logStr), + String.format("Missing: '%s' \nActual: '%s' \nCalculated window: '%s'", logStr, actualString, windowData)); + } + } + } + + @DataProvider + public static Object[][] generateTimeWindows() { + final Object[][] objects = new Object[][]{ + {-1, 10}, + {10, -1}, + {0, 10}, + {10, 0}, + {0, 0}, + {-1, -1}, + {1, 1}, + {5, 2}, + {2, 5}, + {20, 5}, + {20, 10}, + }; + return objects; + } + + @Test(dataProvider = "generateTimeWindows") + public void testTimeWindow(int windowSec, int slideSec) throws Exception { + final SlidingTimeCorrectness testable = new SlidingTimeCorrectness(windowSec, slideSec); + final String topologyName = this.getClass().getSimpleName() + "w" + windowSec + "s" + slideSec; + if (windowSec <= 0 || slideSec <= 0) { + try { + testable.newTopology(); + Assert.fail("Expected IllegalArgumentException was not thrown."); + } catch (IllegalArgumentException ignore) { + return; + } + } + topo = new TopoWrap(cluster, topologyName, testable.newTopology()); + runAndVerifyTime(windowSec, slideSec, testable, topo); + } + + static void runAndVerifyTime(int windowSec, int slideSec, TestableTopology testable, TopoWrap topo) throws TException, java.net.MalformedURLException { + topo.submitSuccessfully(); + final int minSpoutEmits = 1000 + windowSec; + final int minBoltEmits = 5; + String boltName = testable.getBoltName(); + String spoutName = testable.getSpoutName(); + topo.waitForProgress(minSpoutEmits, spoutName, 60 + 10 * (windowSec + slideSec)); + topo.waitForProgress(minBoltEmits, boltName, 60 + 10 * (windowSec + slideSec)); + final List allSpoutData = topo.getLogData(spoutName, TimeData.CLS); + final List allBoltLog = topo.getLogData(boltName); + final List allBoltData = topo.getLogData(boltName, TimeDataWindow.CLS); + Assert.assertTrue(allBoltLog.size() >= minBoltEmits, + "Expecting min " + minBoltEmits + " bolt emits, found: " + allBoltLog.size() + " \n\t" + allBoltLog); + final DateTime firstEndTime = TimeUtil.ceil(new DateTime(allSpoutData.get(0).getDate()).withZone(DateTimeZone.UTC), slideSec); + final int numberOfWindows = allBoltLog.size() - windowSec / slideSec; + for(int i = 0; i < numberOfWindows; ++i ) { + final DateTime toDate = firstEndTime.plusSeconds(i * slideSec); + final DateTime fromDate = toDate.minusSeconds(windowSec); + log.info("Comparing window: " + fromDate + " to " + toDate + " iter " + (i+1) + "/" + numberOfWindows); + final TimeDataWindow computedWindow = TimeDataWindow.newInstance(allSpoutData,fromDate, toDate); + final LogData oneBoltLog = allBoltLog.get(i); + final TimeDataWindow actualWindow = allBoltData.get(i); + log.info("Actual window: " + actualWindow.getDescription()); + log.info("Computed window: " + computedWindow.getDescription()); + for (TimeData oneLog : computedWindow) { + Assert.assertTrue(actualWindow.contains(oneLog), + String.format("Missing: '%s' \n\tActual: '%s' \n\tComputed window: '%s'", oneLog, oneBoltLog, computedWindow)); + } + for (TimeData oneLog : actualWindow) { + Assert.assertTrue(computedWindow.contains(oneLog), + String.format("Extra: '%s' \n\tActual: '%s' \n\tComputed window: '%s'", oneLog, oneBoltLog, computedWindow)); + } + } + } + + @AfterMethod + public void cleanup() throws Exception { + if (topo != null) { + topo.killQuietly(); + } + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java b/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java new file mode 100644 index 00000000000..6fd5fb9c250 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java @@ -0,0 +1,100 @@ +/* + * 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 com.hortonworks.storm.st.tests.window; + +import com.hortonworks.storm.st.helper.AbstractTest; +import com.hortonworks.storm.st.utils.AssertUtil; +import com.hortonworks.storm.st.wrapper.TopoWrap; +import org.hortonworks.storm.st.topology.window.TumblingTimeCorrectness; +import org.hortonworks.storm.st.topology.window.TumblingWindowCorrectness; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +public final class TumblingWindowTest extends AbstractTest { + private static Logger log = LoggerFactory.getLogger(TumblingWindowTest.class); + TopoWrap topo; + + @DataProvider + public static Object[][] generateWindows() { + final Object[][] objects = new Object[][]{ + {-1}, + {0}, + {1}, + {10}, + {250}, + {500}, + }; + return objects; + } + + @Test(dataProvider = "generateWindows") + public void testTumbleCount(int tumbleSize) throws Exception { + final TumblingWindowCorrectness testable = new TumblingWindowCorrectness(tumbleSize); + final String topologyName = this.getClass().getSimpleName() + "t" + tumbleSize; + if (tumbleSize <= 0) { + try { + testable.newTopology(); + Assert.fail("Expected IllegalArgumentException was not thrown."); + } catch (IllegalArgumentException ignore) { + return; + } + } + topo = new TopoWrap(cluster, topologyName, testable.newTopology()); + SlidingWindowTest.runAndVerifyCount(tumbleSize, tumbleSize, testable, topo); + } + + @DataProvider + public static Object[][] generateTumbleTimes() { + final Object[][] objects = new Object[][]{ + {-1}, + {0}, + {1}, + {2}, + {5}, + {10}, + }; + return objects; + } + + @Test(dataProvider = "generateTumbleTimes") + public void testTumbleTime(int tumbleSec) throws Exception { + final TumblingTimeCorrectness testable = new TumblingTimeCorrectness(tumbleSec); + final String topologyName = this.getClass().getSimpleName() + "t" + tumbleSec; + if (tumbleSec <= 0) { + try { + testable.newTopology(); + Assert.fail("Expected IllegalArgumentException was not thrown."); + } catch (IllegalArgumentException ignore) { + return; + } + } + topo = new TopoWrap(cluster, topologyName, testable.newTopology()); + SlidingWindowTest.runAndVerifyTime(tumbleSec, tumbleSec, testable, topo); + } + + @AfterMethod + public void cleanup() throws Exception { + if (topo != null) { + topo.killQuietly(); + } + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java new file mode 100644 index 00000000000..a999b63d12b --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java @@ -0,0 +1,74 @@ +/* + * 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 com.hortonworks.storm.st.utils; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + +import java.io.File; +import java.util.Collection; +import java.util.List; + +/** + * Created by temp on 4/28/16. + */ +public class AssertUtil { + private static Logger log = LoggerFactory.getLogger(AssertUtil.class); + + public static void empty(Collection collection) { + Assert.assertTrue(collection == null || collection.size() == 0, "Expected collection to be non-null, found: " + collection); + } + + public static void nonEmpty(Collection collection, String message) { + Assert.assertNotNull(collection, message + " Expected collection to be non-null, found: " + collection); + greater(collection.size(), 0, message + " Expected collection to be non-empty, found: " + collection); + } + + public static void greater(int actual, int expected, String message) { + Assert.assertTrue(actual > expected, message); + } + + public static void exists(File path) { + Assert.assertNotNull(path, "Supplied path was expected to be non null, found: " + path); + Assert.assertTrue(path.exists(), "Supplied path was expected to be non null, found: " + path); + } + + public static void assertOneElement(Collection collection) { + assertNElements(collection, 1); + } + + public static void assertNElements(Collection collection, int expectedCount) { + String message = "Unexpected number of elements in the collection: " + collection; + Assert.assertEquals(collection.size(), expectedCount, message); + } + + public static void assertTwoElements(Collection collection) { + assertNElements(collection, 2); + } + + public static void assertMatchCount(String actualOutput, List expectedOutput, int requiredMatchCount) { + for (String oneExpectedOutput : expectedOutput) { + final int matchCount = StringUtils.countMatches(actualOutput, oneExpectedOutput); + log.info("In output, found " + matchCount + " occurrences of: " + oneExpectedOutput); + Assert.assertTrue(matchCount > requiredMatchCount, + "Found " + matchCount + "occurrence of " + oneExpectedOutput + " in urls, expected" + requiredMatchCount); + } + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java new file mode 100644 index 00000000000..505bbdb3cf6 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java @@ -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. + */ + +package com.hortonworks.storm.st.utils; + +import org.apache.commons.lang.RandomStringUtils; + +import java.util.SortedSet; +import java.util.TreeSet; + + +/** + * Created by Raghav Kumar Gautam on 6/24/16. + */ +public class DataGenerator { + public static SortedSet generateSortedRandomTexts(int numOfString, int strLen) { + SortedSet randomTexts = new TreeSet<>(); + for (int i = 0; i < numOfString; i++) { + randomTexts.add(RandomStringUtils.randomAlphabetic(strLen)); + } + return randomTexts; + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java new file mode 100644 index 00000000000..3df5151cdc6 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java @@ -0,0 +1,53 @@ +/* + * 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 com.hortonworks.storm.st.utils; + +import org.hortonworks.storm.st.utils.TimeUtil; +import org.openqa.selenium.WebElement; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + +/** + * Created by Raghav Kumar Gautam on 7/6/16. + */ + +public final class UiAssert { + private UiAssert() { + throw new AssertionError("Instantiating utility class..."); + } + private static final Logger LOGGER = LoggerFactory.getLogger(UiAssert.class); + + public static void assertDisplayed(WebElement element, String webElementName) { + LOGGER.info(String.format("Checking if WebElement '%s' is displayed", webElementName)); + int timeoutSeconds = 5; + for (int i = 0; !element.isDisplayed() && i < timeoutSeconds * 10; i++) { + TimeUtil.sleepMilliSec(100); + } + Assert.assertTrue(element.isDisplayed(), + String.format("WebElement '%s' should have been displayed", webElementName)); + LOGGER.info(String.format("WebElement '%s' is displayed", webElementName)); + } + + public static void assertNotDisplayed(WebElement clusterForm, String webElementName) { + LOGGER.info(String.format("Checking if WebElement '%s' is displayed", webElementName)); + Assert.assertFalse(clusterForm.isDisplayed(), + String.format("WebElement '%s' should NOT have been displayed", webElementName)); + LOGGER.info(String.format("WebElement '%s' is not displayed", webElementName)); + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java new file mode 100644 index 00000000000..ef70db5fe30 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java @@ -0,0 +1,107 @@ +/* + * 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 com.hortonworks.storm.st.utils; + +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.log4j.Logger; +import org.hortonworks.storm.st.utils.TimeUtil; +import org.openqa.selenium.By; +import org.openqa.selenium.JavascriptExecutor; +import org.openqa.selenium.WebDriver; +import org.openqa.selenium.WebElement; + +import java.util.ArrayList; +import java.util.List; + +/** + * Utility class for UI related tasks. + */ +public final class UiUtil { + private UiUtil() { + throw new AssertionError("Instantiating utility class..."); + } + private static final Logger LOGGER = Logger.getLogger(UiUtil.class); + + /** + * Convert the element to string representation. Useful for debugging/development. + * @param element element to be converted + * @param limitDepth the depth to traverse. Typically <=3 is good value. + * @return + */ + protected static String elementToString(WebElement element, Integer limitDepth) { + final StringBuilder retVal = + new StringBuilder("String representation of the element(first line is format):\n"); + retVal.append("-> tagname") + .append("(id)") + .append("(classes)") + .append("[extra-info]") + .append("\t") + .append("text") + .append("\n"); + retVal.append(elementToString("", element, limitDepth)); + return retVal.toString(); + } + + private static StringBuilder elementToString(String prefix, WebElement element, Integer + limitDepth) { + if (limitDepth != null && limitDepth == 0) { + return new StringBuilder(); + } + final Integer newDepth = limitDepth == null ? null : limitDepth - 1; + final StringBuilder elementStr = new StringBuilder(prefix); + List extraInfo = new ArrayList<>(); + if (StringUtils.isNotBlank(element.getAttribute("ng-repeat"))) { + extraInfo.add("array"); + } + elementStr.append("-> ") + .append(element.getTagName()) + .append("(").append(element.getAttribute("id")).append(")") + .append("(").append(element.getAttribute("class")).append(")") + .append(extraInfo) + .append("\t").append(StringEscapeUtils.escapeJava(element.getText())); + final String childPrefix = prefix + "\t"; + final List childElements = element.findElements(By.xpath("./*")); + for (WebElement oneChildElement : childElements) { + StringBuilder childStr = elementToString(childPrefix, oneChildElement, newDepth); + if (childStr.length() > 0) { + elementStr.append("\n").append(childStr); + } + } + return elementStr; + } + + /** + * Highlight the element in the UI. Useful for development/debugging. + * Copied from http://www.testingdiaries.com/highlight-element-using-selenium-webdriver/ + * @param element the element to highlight + * @param driver the web driver in use + */ + public static void elementHighlight(WebElement element, WebDriver driver) { + for (int i = 0; i < 2; i++) { + JavascriptExecutor js = (JavascriptExecutor) driver; + js.executeScript( + "arguments[0].setAttribute('style', arguments[1]);", + element, "color: red; border: 3px solid red;"); + TimeUtil.sleepSec(2); + js.executeScript( + "arguments[0].setAttribute('style', arguments[1]);", + element, ""); + } + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java new file mode 100644 index 00000000000..03991b7b8de --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java @@ -0,0 +1,107 @@ +/* + * 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 com.hortonworks.storm.st.wrapper; + +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Ordering; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.SortedSet; + +/** + * Created by Raghav Kumar Gautam on 6/24/16. + */ +public class HdfsWrap { + private final FileSystem fileSystem; + private static Logger log = LoggerFactory.getLogger(HdfsWrap.class); + + private HdfsWrap(FileSystem fileSystem) { + this.fileSystem = fileSystem; + } + + public static HdfsWrap getInstance(final FileSystem fileSystem) { + return new HdfsWrap(fileSystem); + } + + + public void deleteDir(String hdfsDir) throws IOException { + Path hdfsPath = new Path(hdfsDir); + log.info("Deleting dir " + hdfsPath + " on filesystem " + fileSystem.getUri()); + fileSystem.delete(hdfsPath, true); + } + + public void createDir(String hdfsDir) throws IOException { + Path hdfsPath = new Path(hdfsDir); + log.info("Creating dir " + hdfsPath + " on filesystem " + fileSystem.getUri()); + fileSystem.mkdirs(hdfsPath); + } + + public SortedSet readSortedRecords(String hdfsDir, final String delimiterRegex) throws IOException { + final List rawOutput = readDataFromHdfs(hdfsDir); + return FluentIterable.from(rawOutput).transformAndConcat(new Function>() { + @Nullable + @Override + public List apply(@Nullable String s) { + if (StringUtils.isEmpty(s)) { + return new ArrayList<>(); + } + return Arrays.asList(s.split(delimiterRegex)); + } + }).toSortedSet(Ordering.natural()); + } + + private List readDataFromHdfs(String hdfsPath) throws IOException { + final File tempDir = new File(FileUtils.getTempDirectory(), "testHdfsSpoutBoltOutput"); + Assert.assertTrue(tempDir.mkdirs(), "temporary dir creation failed: " + tempDir); + fileSystem.copyToLocalFile(new Path(hdfsPath), new Path(tempDir.getAbsolutePath())); + List outputData = new ArrayList<>(); + for (File oneOutputFile : FileUtils.listFiles(tempDir, new String[] {"txt"}, true)) { + outputData.add(FileUtils.readFileToString(oneOutputFile, Charset.defaultCharset())); + } + FileUtils.deleteQuietly(tempDir); + return outputData; + } + + public void writeDataToHdfs(String data, String hdfsDir) throws IOException { + final File tempFile1 = File.createTempFile("data", ".txt"); + FileUtils.writeStringToFile(tempFile1, data, Charset.defaultCharset()); + fileSystem.copyFromLocalFile(new Path(tempFile1.getAbsolutePath()), new Path(hdfsDir)); + FileUtils.deleteQuietly(tempFile1); + } + + public void createHdfsDirWith777(String dir) throws IOException { + final Path path = new Path(dir); + createDir(dir); + fileSystem.setPermission(path, FsPermission.valueOf("drwxrwxrwx")); + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java new file mode 100644 index 00000000000..60268b83656 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java @@ -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 com.hortonworks.storm.st.wrapper; + +import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.commons.lang.StringUtils; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.hortonworks.storm.st.utils.StringDecorator; + +import java.util.Arrays; +import java.util.List; + +/** + * Created by temp on 5/5/16. + */ + +public class LogData implements Comparable { + private final DateTime logDate; + private final String data; + private static final int dateLen = "2016-05-04 23:38:10.702".length(); //format of date in worker logs + private static final DateTimeFormatter dateFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); + + public LogData(String logLine) { + DateTime tempDate; + final String[] pair = StringDecorator.split2(StringUtils.strip(logLine)); + final List pairList = Arrays.asList(pair); + AssertUtil.assertTwoElements(pairList); + tempDate = dateFormat.parseDateTime(pairList.get(0).substring(0, dateLen)); + this.logDate = tempDate; + this.data = pairList.get(1); + } + + @Override + public String toString() { + return "LogData{" + + "logDate=" + dateFormat.print(logDate) + + ", data='" + getData() + '\'' + + '}'; + } + + @Override + public int compareTo(LogData that) { + return this.logDate.compareTo(that.logDate); + } + + public String getData() { + return data; + } + + public DateTime getLogDate() { + return logDate; + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java new file mode 100644 index 00000000000..b38e10345bc --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java @@ -0,0 +1,114 @@ +/* + * 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 com.hortonworks.storm.st.wrapper; + +import com.google.common.base.Predicate; +import com.google.common.collect.Collections2; +import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.storm.generated.*; +import org.apache.storm.thrift.TException; +import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + +import javax.annotation.Nullable; +import java.util.*; + +/** + * Created by temp on 4/28/16. + */ +public class StormCluster { + private static Logger log = LoggerFactory.getLogger(StormCluster.class); + private final Nimbus.Client client; + + public StormCluster() { + Map conf = getConfig(); + this.client = NimbusClient.getConfiguredClient(conf).getClient(); + } + + public static Map getConfig() { + return Utils.readStormConfig(); + } + + public static boolean isSecure() { + final String thriftConfig = "" + getConfig().get("storm.thrift.transport"); + final String thriftConfigInSecCluster = "org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin"; + return thriftConfigInSecCluster.equals(thriftConfig.trim()); + } + + public List getSummaries() throws TException { + final ClusterSummary clusterInfo = client.getClusterInfo(); + log.info("Cluster info: " + clusterInfo); + return clusterInfo.get_topologies(); + } + + public List getActive() throws TException { + return getTopologiesWithStatus("active"); + } + + public List getKilled() throws TException { + return getTopologiesWithStatus("killed"); + } + + private List getTopologiesWithStatus(final String expectedStatus) throws TException { + Collection topologySummaries = getSummaries(); + Collection filteredSummary = Collections2.filter(topologySummaries, new Predicate() { + @Override + public boolean apply(@Nullable TopologySummary input) { + return input != null && input.get_status().toLowerCase().equals(expectedStatus.toLowerCase()); + } + }); + return new ArrayList<>(filteredSummary); + } + + public void killSilently(String topologyName) { + try { + client.killTopologyWithOpts(topologyName, new KillOptions()); + log.info("Topology killed: " + topologyName); + } catch (Throwable e){ + log.warn("Couldn't kill topology: " + topologyName + " Exception: " + ExceptionUtils.getFullStackTrace(e)); + } + } + + public TopologySummary getOneActive() throws TException { + List topoSummaries = getActive(); + AssertUtil.nonEmpty(topoSummaries, "Expecting one active topology."); + Assert.assertEquals(topoSummaries.size(), 1, "Expected one topology to be running, found: " + topoSummaries); + return topoSummaries.get(0); + } + + public TopologyInfo getInfo(TopologySummary topologySummary) throws TException { + return client.getTopologyInfo(topologySummary.get_id()); + } + + public Nimbus.Client getNimbusClient() { + return client; + } + + public void killActiveTopologies() throws TException { + List activeTopologies = getActive(); + for (TopologySummary activeTopology : activeTopologies) { + killSilently(activeTopology.get_name()); + } + + AssertUtil.empty(getActive()); + } +} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java new file mode 100644 index 00000000000..ddb59760bd7 --- /dev/null +++ b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java @@ -0,0 +1,362 @@ +/* + * 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 com.hortonworks.storm.st.wrapper; + +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.*; +import org.apache.storm.thrift.TException; +import org.hortonworks.storm.st.topology.window.data.FromJson; +import org.hortonworks.storm.st.utils.StringDecorator; +import org.hortonworks.storm.st.utils.TimeUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; +import java.text.NumberFormat; +import java.util.*; +import java.util.regex.Pattern; + +/** + * Created by temp on 4/29/16. + */ +public class TopoWrap { + private static Logger log = LoggerFactory.getLogger(TopoWrap.class); + private final StormCluster cluster; + + private final String name; + private final StormTopology topology; + private String id; + public static Map submitConf = getSubmitConf(); + static { + String jarFile = getJarPath(); + log.info("setting storm.jar to: " + jarFile); + System.setProperty("storm.jar", jarFile); + } + + public TopoWrap(StormCluster cluster, String name, StormTopology topology) { + this.cluster = cluster; + this.name = name; + this.topology = topology; + } + + public void submit(ImmutableMap of) throws AlreadyAliveException, InvalidTopologyException, AuthorizationException { + final HashMap newConfig = new HashMap<>(submitConf); + newConfig.putAll(of); + StormSubmitter.submitTopologyWithProgressBar(name, newConfig, topology); + } + + private static Map getSubmitConf() { + Map submitConf = new HashMap<>(); + submitConf.put("storm.zookeeper.topology.auth.scheme", "digest"); + submitConf.put("topology.workers", 3); + submitConf.put("topology.debug", true); + return submitConf; + } + + private static String getJarPath() { + final String USER_DIR = "user.dir"; + String userDirVal = System.getProperty(USER_DIR); + Assert.assertNotNull(userDirVal, "property " + USER_DIR + " was not set."); + File projectDir = new File(userDirVal); + AssertUtil.exists(projectDir); + Collection allJars = FileUtils.listFiles(projectDir, new String[]{"jar"}, true); + final Collection jarFiles = Collections2.filter(allJars, new Predicate() { + @Override + public boolean apply(@Nullable File input) { + return input != null && !input.getName().contains("surefirebooter"); + } + }); + log.info("Found jar files: " + jarFiles); + AssertUtil.nonEmpty(jarFiles, "The jar file is missing - did you run 'mvn clean package -DskipTests' before running tests ?"); + String jarFile = null; + for (File jarPath : jarFiles) { + log.info("jarPath = " + jarPath); + if (jarPath != null && !jarPath.getPath().contains("original")) { + AssertUtil.exists(jarPath); + jarFile = jarPath.getAbsolutePath(); + break; + } + } + Assert.assertNotNull(jarFile, "Couldn't detect a suitable jar file for uploading."); + log.info("jarFile = " + jarFile); + return jarFile; + } + + public void submitSuccessfully(ImmutableMap config) throws TException { + submit(config); + TopologySummary topologySummary = getSummary(); + Assert.assertEquals(topologySummary.get_status().toLowerCase(), "active", "Topology must be active."); + id = topologySummary.get_id(); + } + + public void submitSuccessfully() throws TException { + submitSuccessfully(ImmutableMap.of()); + } + + private TopologySummary getSummary() throws TException { + List allTopos = cluster.getSummaries(); + Collection oneTopo = Collections2.filter(allTopos, new Predicate() { + @Override + public boolean apply(@Nullable TopologySummary input) { + return input != null && input.get_name().equals(name); + } + }); + AssertUtil.assertOneElement(oneTopo); + return oneTopo.iterator().next(); + } + + public TopologyInfo getInfo() throws TException { + return cluster.getNimbusClient().getTopologyInfo(id); + } + + public long getAllTimeEmittedCount(final String componentId) throws TException { + TopologyInfo info = getInfo(); + final List executors = info.get_executors(); + List ackCounts = Lists.transform(executors, new Function() { + @Nullable + @Override + public Long apply(@Nullable ExecutorSummary input) { + if (input == null || !input.get_component_id().equals(componentId)) + return 0L; + String since = ":all-time"; + return getEmittedCount(input, since); + } + + //possible values for since are strings :all-time, 600, 10800, 86400 + public Long getEmittedCount(@Nonnull ExecutorSummary input, @Nonnull String since) { + ExecutorStats executorStats = input.get_stats(); + if (executorStats == null) + return 0L; + Map> emitted = executorStats.get_emitted(); + if (emitted == null) + return 0L; + Map allTime = emitted.get(since); + if (allTime == null) + return 0L; + return allTime.get("default"); + } + }); + return sum(ackCounts).longValue(); + } + + public List getLogUrls(final String componentId) throws TException, MalformedURLException { + ComponentPageInfo componentPageInfo = cluster.getNimbusClient().getComponentPageInfo(id, componentId, null, false); + Map windowToStats = componentPageInfo.get_window_to_stats(); + ComponentAggregateStats allTimeStats = windowToStats.get(":all-time"); + //Long emitted = (Long) allTimeStats.getFieldValue(ComponentAggregateStats._Fields.findByName("emitted")); + + + List execStats = componentPageInfo.get_exec_stats(); + Set urls = new HashSet<>(); + for (ExecutorAggregateStats execStat : execStats) { + ExecutorSummary execSummary = execStat.get_exec_summary(); + String host = execSummary.get_host(); + int executorPort = execSummary.get_port(); + //http://supervisor2:8000/download/DemoTest-26-1462229009%2F6703%2Fworker.log + //http://supervisor2:8000/log?file=SlidingWindowCountTest-9-1462388349%2F6703%2Fworker.log + int logViewerPort = 8000; + ExecutorURL executorURL = new ExecutorURL(componentId, host, logViewerPort, executorPort, id); + urls.add(executorURL); + } + return new ArrayList<>(urls); + } + + public void waitForProgress(int minEmits, String componentName, int maxWaitSec) throws TException { + for(int i = 0; i < (maxWaitSec+9)/10; ++i) { + log.info(getInfo().toString()); + long emitCount = getAllTimeEmittedCount(componentName); + log.info("Count for component " + componentName + " is " + emitCount); + if (emitCount > minEmits) { + break; + } + TimeUtil.sleepSec(10); + } + } + + public void assertProgress(int minEmits, String componentName, int maxWaitSec) throws TException { + waitForProgress(minEmits, componentName, maxWaitSec); + long emitCount = getAllTimeEmittedCount(componentName); + Assert.assertTrue(emitCount >= minEmits, "Count for component " + componentName + " is " + emitCount + " min is " + minEmits); + } + + public static class ExecutorURL { + private String componentId; + private URL viewUrl; + private URL downloadUrl; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ExecutorURL)) return false; + + ExecutorURL that = (ExecutorURL) o; + + if (componentId != null ? !componentId.equals(that.componentId) : that.componentId != null) return false; + if (getViewUrl() != null ? !getViewUrl().equals(that.getViewUrl()) : that.getViewUrl() != null) + return false; + return getDownloadUrl() != null ? getDownloadUrl().equals(that.getDownloadUrl()) : that.getDownloadUrl() == null; + + } + + @Override + public int hashCode() { + int result = componentId != null ? componentId.hashCode() : 0; + result = 31 * result + (getViewUrl() != null ? getViewUrl().hashCode() : 0); + result = 31 * result + (getDownloadUrl() != null ? getDownloadUrl().hashCode() : 0); + return result; + } + + public ExecutorURL(String componentId, String host, int logViewerPort, int executorPort, String topoId) throws MalformedURLException { + String sep = "%2F"; //hex of "/" + String viewUrlStr = String.format("http://%s:%s/log?file=", host, logViewerPort); + String downloadUrlStr = String.format("http://%s:%s/download", host, logViewerPort); + viewUrl = new URL(String.format("%s/%s%s%d%sworker.log", viewUrlStr, topoId, sep, executorPort, sep)); + downloadUrl = new URL(String.format("%s/%s%s%d%sworker.log", downloadUrlStr, topoId, sep, executorPort, sep)); + this.componentId = componentId; + } + + public URL getDownloadUrl() { + return downloadUrl; + } + + public URL getViewUrl() { + return viewUrl; + } + + @Override + public String toString() { + return "ExecutorURL{" + + "componentId='" + componentId + '\'' + + ", viewUrl=" + viewUrl + + ", downloadUrl=" + downloadUrl + + '}'; + } + } + + public > List getLogData(final String componentId, final FromJson cls) throws TException, MalformedURLException { + final List logData = getLogData(componentId); + final List data = new ArrayList<>( + Collections2.transform(logData, new Function() { + @Nullable + @Override + public T apply(@Nullable LogData input) { + Assert.assertNotNull(input, "Expected LogData to be non-null."); + return cls.fromJson(input.getData()); + } + })); + return data; + } + + public List getLogData(final String componentId) throws TException, MalformedURLException { + final String logs = getLogs(componentId); + final String dateRegex = "\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}\\.\\d{3}"; + Pattern pattern = Pattern.compile("(?=\\n" + dateRegex + ")"); + final String[] strings = pattern.split(logs); + final Collection interestingLogs = Collections2.filter(Arrays.asList(strings), new Predicate() { + @Override + public boolean apply(@Nullable String input) { + return input != null && StringDecorator.isDecorated(input); + } + }); + final Collection logData = Collections2.transform(interestingLogs, new Function() { + @Nullable + @Override + public LogData apply(@Nullable String input) { + return new LogData(input); + } + }); + final ArrayList sortedLogs = new ArrayList<>(logData); + Collections.sort(sortedLogs); + log.info("Found " + sortedLogs.size() + " items for component: " + componentId); + return sortedLogs; + } + + public String getLogs(final String componentId) throws TException, MalformedURLException { + log.info("Fetching logs for componentId = " + componentId); + List exclaim2Urls = getLogUrls(componentId); + log.info("Found " + exclaim2Urls.size() + " urls: " + exclaim2Urls.toString()); + Collection urlOuputs = Collections2.transform(exclaim2Urls, new Function() { + @Nullable + @Override + public String apply(@Nullable ExecutorURL executorURL) { + if (executorURL == null || executorURL.getDownloadUrl() == null) { + return ""; + } + String warnMessage = "Couldn't fetch executorURL: " + executorURL; + try { + log.info("Fetching: " + executorURL); + final URL downloadUrl = executorURL.downloadUrl; + final String urlContent = IOUtils.toString(downloadUrl); + if (urlContent.length() < 500) { + log.info("Fetched: " + urlContent); + } else { + log.info("Fetched: " + NumberFormat.getNumberInstance(Locale.US).format(urlContent.length()) + " bytes."); + } + if (System.getProperty("regression.downloadWorkerLogs").equalsIgnoreCase("true")) { + final String userDir = System.getProperty("user.dir"); + final File target = new File(userDir, "target"); + final File logDir = new File(target, "logs"); + final File logFile = new File(logDir, downloadUrl.getHost() + "-" + downloadUrl.getFile().split("/")[2]); + try { + FileUtils.forceMkdir(logDir); + FileUtils.write(logFile, urlContent); + } catch (Throwable throwable) { + log.info("Caught exteption: " + ExceptionUtils.getFullStackTrace(throwable)); + } + } + return urlContent; + } catch (IOException e) { + log.warn(warnMessage); + } + String stars = StringUtils.repeat("*", 30); + return stars + " " + warnMessage + " " + stars; + } + }); + return StringUtils.join(urlOuputs, '\n'); + } + + private Number sum(Collection nums) { + Double retVal = 0.0; + for (Number num : nums) { + if(num != null) { + retVal += num.doubleValue(); + } + } + return retVal; + } + + public void killQuietly() { + cluster.killSilently(name); + } +} diff --git a/integration-test/src/test/resources/storm-conf/storm.yaml b/integration-test/src/test/resources/storm-conf/storm.yaml new file mode 100644 index 00000000000..8557188b131 --- /dev/null +++ b/integration-test/src/test/resources/storm-conf/storm.yaml @@ -0,0 +1,95 @@ +atlas.cluster.name : 'tstc' +client.jartransformer.class : 'org.apache.storm.hack.StormShadeTransformer' +dev.zookeeper.path : '/tmp/dev-storm-zookeeper' +drpc.childopts : '-Xmx768m ' +drpc.invocations.port : 3773 +drpc.port : 3772 +drpc.queue.size : 128 +drpc.request.timeout.secs : 600 +drpc.worker.threads : 64 +java.library.path : '/usr/local/lib:/opt/local/lib:/usr/lib:/usr/hdp/current/storm-client/lib' +logviewer.appender.name : 'A1' +logviewer.childopts : '-Xmx128m ' +logviewer.port : 8000 +nimbus.childopts : '-Xmx1024m -javaagent:/usr/hdp/current/storm-nimbus/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8649,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-nimbus/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM' +nimbus.cleanup.inbox.freq.secs : 600 +nimbus.file.copy.expiration.secs : 600 +nimbus.inbox.jar.expiration.secs : 3600 +nimbus.monitor.freq.secs : 120 +nimbus.reassign : true +nimbus.seeds : ['c6402.ambari.apache.org'] +nimbus.supervisor.timeout.secs : 60 +nimbus.task.launch.secs : 120 +nimbus.task.timeout.secs : 30 +nimbus.thrift.max_buffer_size : 1048576 +nimbus.thrift.port : 6627 +nimbus.topology.validator : 'org.apache.storm.nimbus.DefaultTopologyValidator' +storm.cluster.mode : 'distributed' +storm.local.dir : '/hadoop/storm' +storm.local.mode.zmq : false +storm.log.dir : '/var/log/storm' +storm.messaging.netty.buffer_size : 5242880 +storm.messaging.netty.client_worker_threads : 1 +storm.messaging.netty.max_retries : 30 +storm.messaging.netty.max_wait_ms : 1000 +storm.messaging.netty.min_wait_ms : 100 +storm.messaging.netty.server_worker_threads : 1 +storm.messaging.transport : 'org.apache.storm.messaging.netty.Context' +storm.thrift.transport : 'org.apache.storm.security.auth.SimpleTransportPlugin' +storm.zookeeper.connection.timeout : 15000 +storm.zookeeper.port : 2181 +storm.zookeeper.retry.interval : 1000 +storm.zookeeper.retry.intervalceiling.millis : 30000 +storm.zookeeper.retry.times : 5 +storm.zookeeper.root : '/storm' +storm.zookeeper.servers : ['c6401.ambari.apache.org','c6402.ambari.apache.org','c6403.ambari.apache.org'] +storm.zookeeper.session.timeout : 20000 +supervisor.childopts : '-Xmx256m -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=56431 -javaagent:/usr/hdp/current/storm-supervisor/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-supervisor/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM' +supervisor.heartbeat.frequency.secs : 5 +supervisor.monitor.frequency.secs : 3 +supervisor.slots.ports : [6700, 6701] +supervisor.worker.start.timeout.secs : 120 +supervisor.worker.timeout.secs : 30 +task.heartbeat.frequency.secs : 3 +task.refresh.poll.secs : 10 +topology.acker.executors : null +topology.builtin.metrics.bucket.size.secs : 60 +topology.debug : false +topology.disruptor.wait.strategy : 'com.lmax.disruptor.BlockingWaitStrategy' +topology.enable.message.timeouts : true +topology.error.throttle.interval.secs : 10 +topology.executor.receive.buffer.size : 1024 +topology.executor.send.buffer.size : 1024 +topology.fall.back.on.java.serialization : true +topology.kryo.factory : 'org.apache.storm.serialization.DefaultKryoFactory' +topology.max.error.report.per.interval : 5 +topology.max.replication.wait.time.sec : 60 +topology.max.spout.pending : 1000 +topology.max.task.parallelism : null +topology.message.timeout.secs : 30 +topology.min.replication.count : 1 +topology.optimize : true +topology.receiver.buffer.size : 8 +topology.skip.missing.kryo.registrations : false +topology.sleep.spout.wait.strategy.time.ms : 1 +topology.spout.wait.strategy : 'org.apache.storm.spout.SleepSpoutWaitStrategy' +topology.state.synchronization.timeout.secs : 60 +topology.stats.sample.rate : 0.05 +topology.tick.tuple.freq.secs : null +topology.transfer.buffer.size : 1024 +topology.trident.batch.emit.interval.millis : 500 +topology.tuple.serializer : 'org.apache.storm.serialization.types.ListDelegateSerializer' +topology.worker.childopts : null +topology.worker.shared.thread.pool.size : 4 +topology.workers : 1 +transactional.zookeeper.port : null +transactional.zookeeper.root : '/transactional' +transactional.zookeeper.servers : null +ui.childopts : '-Xmx768m ' +ui.filter : null +ui.port : 8744 +worker.childopts : '-Xmx768m -javaagent:/usr/hdp/current/storm-client/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-client/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM' +worker.heartbeat.frequency.secs : 1 +zmq.hwm : 0 +zmq.linger.millis : 5000 +zmq.threads : 1 From ce1cdd27974f397faf4ed23eebb8d09e8f79c1c2 Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Mon, 19 Sep 2016 13:41:31 -0700 Subject: [PATCH 2/6] fix for java_home issue --- integration-test/run-it.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/integration-test/run-it.sh b/integration-test/run-it.sh index eed6025743d..3b4a86167e9 100755 --- a/integration-test/run-it.sh +++ b/integration-test/run-it.sh @@ -62,9 +62,11 @@ list_storm_processes || true sudo bash ${SCRIPT_DIR}/config/common.sh sudo bash ${SCRIPT_DIR}/config/install-zookeeper.sh sudo bash ${SCRIPT_DIR}/config/install-storm.sh $storm_binary_zip +export JAVA_HOME +env function start_storm_process() { echo starting: storm $1 - sudo su storm -c "cd /home/storm && storm $1" & + sudo su storm -c "export JAVA_HOME=${JAVA_HOME} && cd /home/storm && storm $1" & } start_storm_process nimbus start_storm_process ui From 9c793071ef75932190df62107333ae17b639cf8a Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Tue, 20 Sep 2016 14:57:52 -0700 Subject: [PATCH 3/6] moving to apache package --- integration-test/README.md | 40 ++++--- integration-test/config/etc-hosts | 5 +- integration-test/pom.xml | 2 +- .../storm/ExclamationTopology.java | 6 +- .../storm/debug/DebugHelper.java | 2 +- .../storm/st/topology/TestableTopology.java | 2 +- .../window/SlidingTimeCorrectness.java | 12 +- .../window/SlidingWindowCorrectness.java | 9 +- .../window/TumblingTimeCorrectness.java | 12 +- .../window/TumblingWindowCorrectness.java | 9 +- .../st/topology/window/data/FromJson.java | 2 +- .../st/topology/window/data/TimeData.java | 2 +- .../topology/window/data/TimeDataWindow.java | 2 +- .../storm/st/utils/StringDecorator.java | 2 +- .../storm/st/utils/TimeUtil.java | 2 +- .../hortonworks/storm/st/utils/HdfsUtils.java | 58 --------- .../storm/st/utils/DataGenerator.java | 37 ------ .../hortonworks/storm/st/utils/UiAssert.java | 53 --------- .../hortonworks/storm/st/utils/UiUtil.java | 107 ----------------- .../storm/st/wrapper/HdfsWrap.java | 107 ----------------- .../apache}/storm/st/DemoTest.java | 12 +- .../apache}/storm/st/helper/AbstractTest.java | 4 +- .../apache}/storm/st/meta/TestngListener.java | 8 +- .../st/tests/window/SlidingWindowTest.java | 22 ++-- .../st/tests/window/TumblingWindowTest.java | 11 +- .../apache}/storm/st/utils/AssertUtil.java | 2 +- .../apache}/storm/st/wrapper/LogData.java | 6 +- .../storm/st/wrapper/StormCluster.java | 4 +- .../apache}/storm/st/wrapper/TopoWrap.java | 10 +- .../src/test/resources/storm-conf/storm.yaml | 112 +++--------------- 30 files changed, 109 insertions(+), 553 deletions(-) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/ExclamationTopology.java (95%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/debug/DebugHelper.java (97%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/TestableTopology.java (96%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/SlidingTimeCorrectness.java (95%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/SlidingWindowCorrectness.java (96%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/TumblingTimeCorrectness.java (95%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/TumblingWindowCorrectness.java (96%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/data/FromJson.java (94%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/data/TimeData.java (98%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/topology/window/data/TimeDataWindow.java (98%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/utils/StringDecorator.java (97%) rename integration-test/src/main/java/org/{hortonworks => apache}/storm/st/utils/TimeUtil.java (98%) delete mode 100644 integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java delete mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java delete mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java delete mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java delete mode 100644 integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/DemoTest.java (91%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/helper/AbstractTest.java (91%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/meta/TestngListener.java (92%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/tests/window/SlidingWindowTest.java (92%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/tests/window/TumblingWindowTest.java (90%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/utils/AssertUtil.java (98%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/wrapper/LogData.java (93%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/wrapper/StormCluster.java (97%) rename integration-test/src/test/java/{com/hortonworks => org/apache}/storm/st/wrapper/TopoWrap.java (98%) diff --git a/integration-test/README.md b/integration-test/README.md index 42a086c0b56..36c70a1c3a0 100644 --- a/integration-test/README.md +++ b/integration-test/README.md @@ -1,18 +1,31 @@ -End to end storm-integration-tests +End to end storm integration tests ================================== -Bring up a cluster ------------------- -Vagrant setup can be used for bringing up unsecure Storm 1.0 -https://github.com/raghavgautam/storm-vagrant +Running tests end-to-end +------------------------ +Assumption: +A single version of storm binary zip such as `storm-dist/binary/target/apache-storm-2.0.0-SNAPSHOT.zip` is present +The following command will bring up a vagrant cluster. +```sh +cd integration-test/config +vagrant up +``` +This automatically will run `integration-test/run-it.sh`. +This brings up a vagrant machine, with storm and zookeeper daemons. +And runs all the tests against it. -Configs for running +Running tests for development & debugging +========================================= +```vagrant up``` command is steup as a complete auto-pilot. +Following describes how we can run individual tests against this vagrant cluster or any other cluster. + +Configs for running ------------------- -Change following storm.yaml: -storm-integration-test/src/test/resources/storm.yaml +The supplied configuration will run tests against vagrant setup. However, it can be changed to use a different cluster. +Change `integration-test/src/test/resources/storm.yaml` as necessary. -Running tests end to end from Commandline ------------------------------------------ +Running all tests manually +-------------------------- To run all tests: ```sh mvn clean package -DskipTests && mvn test @@ -25,6 +38,7 @@ mvn clean package -DskipTests && mvn test -Dtest=SlidingWindowCountTest Running tests from IDE ---------------------- +You might have to enable intellij profile to make your IDE happy. Make sure that the following is run before tests are launched. ```sh mvn package -DskipTests @@ -40,10 +54,6 @@ mvn test -Dtest=DemoTest -Dstorm.version= To find version of the storm that you are running run `storm version` command. -Submiting topologies --------------------- -All the topologies can be submitted using `storm jar` command. - Code ---- -Start off by looking at file [DemoTest.java](https://github.com/raghavgautam/storm-integration-test/blob/master/src/test/java/com/hortonworks/storm/st/DemoTest.java). +Start off by looking at file [DemoTest.java](https://github.com/apache/storm/integration-test/blob/master/src/test/java/org/apache/storm/st/DemoTest.java). diff --git a/integration-test/config/etc-hosts b/integration-test/config/etc-hosts index 597d9813d16..3b9032181b1 100644 --- a/integration-test/config/etc-hosts +++ b/integration-test/config/etc-hosts @@ -1,5 +1,2 @@ 127.0.0.1 localhost localhost -192.168.50.3 zookeeper zookeeper -192.168.50.4 nimbus nimbus -192.168.50.5 supervisor1 supervisor1 -192.168.50.6 supervisor2 supervisor2 +192.168.100.100 node-1 diff --git a/integration-test/pom.xml b/integration-test/pom.xml index 50c90359d8b..47eb2446d0e 100755 --- a/integration-test/pom.xml +++ b/integration-test/pom.xml @@ -179,7 +179,7 @@ listener - com.hortonworks.storm.st.meta.TestngListener + org.apache.storm.st.meta.TestngListener diff --git a/integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java similarity index 95% rename from integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java rename to integration-test/src/main/java/org/apache/storm/ExclamationTopology.java index e7a901a9a94..d4646087501 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/ExclamationTopology.java +++ b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java @@ -14,11 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.hortonworks.storm; -import org.apache.storm.Config; -import org.apache.storm.LocalCluster; -import org.apache.storm.StormSubmitter; +package org.apache.storm; + import org.apache.storm.generated.StormTopology; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; diff --git a/integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java b/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java similarity index 97% rename from integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java rename to integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java index 01503b4a1fc..3ace3cc4881 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/debug/DebugHelper.java +++ b/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.debug; +package org.apache.storm.debug; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java b/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java similarity index 96% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java rename to integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java index fc8d1b070c0..75777ec1e48 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/TestableTopology.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.topology; +package org.apache.storm.st.topology; import org.apache.storm.generated.StormTopology; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java similarity index 95% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java index 462705389cc..a3c9e33c06e 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingTimeCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java @@ -14,14 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.hortonworks.storm.st.topology.window; + +package org.apache.storm.st.topology.window; import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; -import com.google.gson.GsonBuilder; import org.apache.storm.generated.StormTopology; import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.st.topology.TestableTopology; +import org.apache.storm.st.topology.window.data.TimeData; +import org.apache.storm.st.utils.StringDecorator; +import org.apache.storm.st.utils.TimeUtil; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; @@ -32,10 +36,6 @@ import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; import org.apache.storm.windowing.TupleWindow; -import org.hortonworks.storm.st.topology.TestableTopology; -import org.hortonworks.storm.st.topology.window.data.TimeData; -import org.hortonworks.storm.st.utils.StringDecorator; -import org.hortonworks.storm.st.utils.TimeUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingWindowCorrectness.java similarity index 96% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingWindowCorrectness.java index 016478cba05..33ee00482c8 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/SlidingWindowCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingWindowCorrectness.java @@ -14,7 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.hortonworks.storm.st.topology.window; + +package org.apache.storm.st.topology.window; import com.google.common.collect.Lists; import org.apache.storm.generated.StormTopology; @@ -29,11 +30,11 @@ import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; import org.apache.storm.windowing.TupleWindow; -import org.hortonworks.storm.st.topology.TestableTopology; +import org.apache.storm.st.topology.TestableTopology; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.hortonworks.storm.st.utils.StringDecorator; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.utils.StringDecorator; +import org.apache.storm.st.utils.TimeUtil; import java.util.List; import java.util.Map; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java similarity index 95% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java index da54ef388a6..df2aa2df9bb 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingTimeCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java @@ -14,14 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.hortonworks.storm.st.topology.window; + +package org.apache.storm.st.topology.window; import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; -import com.google.gson.GsonBuilder; import org.apache.storm.generated.StormTopology; import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.st.topology.TestableTopology; +import org.apache.storm.st.topology.window.data.TimeData; +import org.apache.storm.st.utils.TimeUtil; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; @@ -32,10 +35,7 @@ import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; import org.apache.storm.windowing.TupleWindow; -import org.hortonworks.storm.st.topology.TestableTopology; -import org.hortonworks.storm.st.topology.window.data.TimeData; -import org.hortonworks.storm.st.utils.StringDecorator; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.utils.StringDecorator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingWindowCorrectness.java similarity index 96% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingWindowCorrectness.java index 2ba2895254a..22c6d75645d 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/TumblingWindowCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingWindowCorrectness.java @@ -14,7 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.hortonworks.storm.st.topology.window; + +package org.apache.storm.st.topology.window; import com.google.common.collect.Lists; import org.apache.storm.generated.StormTopology; @@ -29,11 +30,11 @@ import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; import org.apache.storm.windowing.TupleWindow; -import org.hortonworks.storm.st.topology.TestableTopology; +import org.apache.storm.st.topology.TestableTopology; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.hortonworks.storm.st.utils.StringDecorator; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.utils.StringDecorator; +import org.apache.storm.st.utils.TimeUtil; import java.util.List; import java.util.Map; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java similarity index 94% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java index 6d163faae8c..232de09d3f4 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/FromJson.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.topology.window.data; +package org.apache.storm.st.topology.window.data; /** * Created by temp on 5/23/16. diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java similarity index 98% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java index 154f2b4d5ef..5a29c228f20 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeData.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.topology.window.data; +package org.apache.storm.st.topology.window.data; import com.google.gson.Gson; import com.google.gson.GsonBuilder; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java similarity index 98% rename from integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java rename to integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java index 65d67ddd900..6c01e93b18b 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/topology/window/data/TimeDataWindow.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.topology.window.data; +package org.apache.storm.st.topology.window.data; import com.google.common.base.Predicate; import com.google.common.collect.Collections2; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java b/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java similarity index 97% rename from integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java rename to integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java index b6a31b45b89..0a9cb6dbc87 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/utils/StringDecorator.java +++ b/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.utils; +package org.apache.storm.st.utils; import org.apache.commons.lang.StringUtils; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java b/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java similarity index 98% rename from integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java rename to integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java index 09c98c85e13..dd97cf33f0a 100644 --- a/integration-test/src/main/java/org/hortonworks/storm/st/utils/TimeUtil.java +++ b/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.hortonworks.storm.st.utils; +package org.apache.storm.st.utils; import org.apache.commons.lang.exception.ExceptionUtils; import org.joda.time.DateTime; diff --git a/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java b/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java deleted file mode 100644 index c005d3784ff..00000000000 --- a/integration-test/src/main/java/org/hortonworks/storm/st/utils/HdfsUtils.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.hortonworks.storm.st.utils; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; - -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.List; - -/** - * Created by Raghav Kumar Gautam on 6/7/16. - */ -public class HdfsUtils { - public static String joinPaths(String basePath, String firstPart, String... restParts) { - final String separator = "/"; - List cleanParts = new ArrayList<>(); - String cleanBasePath = basePath.replaceFirst(separator + "$", ""); - cleanParts.add(cleanBasePath); - final String cleanFirstPart1 = firstPart.replaceFirst("^" + separator, "").replaceFirst(separator + "$", ""); - cleanParts.add(cleanFirstPart1); - for (String onePart : restParts) { - final String cleanPart = onePart.replaceFirst("^" + separator, "").replaceFirst(separator + "$", ""); - cleanParts.add(cleanPart); - } - return StringUtils.join(cleanParts, separator); - } - - public static FileSystem getFileSystem() { - Configuration conf = new Configuration(); - //conf.setBoolean("fs.hdfs.impl.disable.cache", true); - //conf.set("fs.default.name", "hdfs://" + hadoopURL); - try { - return FileSystem.get(conf); - } catch (IOException e) { - throw new RuntimeException("Can't proceed without FileSystem object.", e); - } - } - -} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java deleted file mode 100644 index 505bbdb3cf6..00000000000 --- a/integration-test/src/test/java/com/hortonworks/storm/st/utils/DataGenerator.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 com.hortonworks.storm.st.utils; - -import org.apache.commons.lang.RandomStringUtils; - -import java.util.SortedSet; -import java.util.TreeSet; - - -/** - * Created by Raghav Kumar Gautam on 6/24/16. - */ -public class DataGenerator { - public static SortedSet generateSortedRandomTexts(int numOfString, int strLen) { - SortedSet randomTexts = new TreeSet<>(); - for (int i = 0; i < numOfString; i++) { - randomTexts.add(RandomStringUtils.randomAlphabetic(strLen)); - } - return randomTexts; - } -} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java deleted file mode 100644 index 3df5151cdc6..00000000000 --- a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiAssert.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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 com.hortonworks.storm.st.utils; - -import org.hortonworks.storm.st.utils.TimeUtil; -import org.openqa.selenium.WebElement; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.Assert; - -/** - * Created by Raghav Kumar Gautam on 7/6/16. - */ - -public final class UiAssert { - private UiAssert() { - throw new AssertionError("Instantiating utility class..."); - } - private static final Logger LOGGER = LoggerFactory.getLogger(UiAssert.class); - - public static void assertDisplayed(WebElement element, String webElementName) { - LOGGER.info(String.format("Checking if WebElement '%s' is displayed", webElementName)); - int timeoutSeconds = 5; - for (int i = 0; !element.isDisplayed() && i < timeoutSeconds * 10; i++) { - TimeUtil.sleepMilliSec(100); - } - Assert.assertTrue(element.isDisplayed(), - String.format("WebElement '%s' should have been displayed", webElementName)); - LOGGER.info(String.format("WebElement '%s' is displayed", webElementName)); - } - - public static void assertNotDisplayed(WebElement clusterForm, String webElementName) { - LOGGER.info(String.format("Checking if WebElement '%s' is displayed", webElementName)); - Assert.assertFalse(clusterForm.isDisplayed(), - String.format("WebElement '%s' should NOT have been displayed", webElementName)); - LOGGER.info(String.format("WebElement '%s' is not displayed", webElementName)); - } -} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java b/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java deleted file mode 100644 index ef70db5fe30..00000000000 --- a/integration-test/src/test/java/com/hortonworks/storm/st/utils/UiUtil.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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 com.hortonworks.storm.st.utils; - -import org.apache.commons.lang.StringEscapeUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; -import org.hortonworks.storm.st.utils.TimeUtil; -import org.openqa.selenium.By; -import org.openqa.selenium.JavascriptExecutor; -import org.openqa.selenium.WebDriver; -import org.openqa.selenium.WebElement; - -import java.util.ArrayList; -import java.util.List; - -/** - * Utility class for UI related tasks. - */ -public final class UiUtil { - private UiUtil() { - throw new AssertionError("Instantiating utility class..."); - } - private static final Logger LOGGER = Logger.getLogger(UiUtil.class); - - /** - * Convert the element to string representation. Useful for debugging/development. - * @param element element to be converted - * @param limitDepth the depth to traverse. Typically <=3 is good value. - * @return - */ - protected static String elementToString(WebElement element, Integer limitDepth) { - final StringBuilder retVal = - new StringBuilder("String representation of the element(first line is format):\n"); - retVal.append("-> tagname") - .append("(id)") - .append("(classes)") - .append("[extra-info]") - .append("\t") - .append("text") - .append("\n"); - retVal.append(elementToString("", element, limitDepth)); - return retVal.toString(); - } - - private static StringBuilder elementToString(String prefix, WebElement element, Integer - limitDepth) { - if (limitDepth != null && limitDepth == 0) { - return new StringBuilder(); - } - final Integer newDepth = limitDepth == null ? null : limitDepth - 1; - final StringBuilder elementStr = new StringBuilder(prefix); - List extraInfo = new ArrayList<>(); - if (StringUtils.isNotBlank(element.getAttribute("ng-repeat"))) { - extraInfo.add("array"); - } - elementStr.append("-> ") - .append(element.getTagName()) - .append("(").append(element.getAttribute("id")).append(")") - .append("(").append(element.getAttribute("class")).append(")") - .append(extraInfo) - .append("\t").append(StringEscapeUtils.escapeJava(element.getText())); - final String childPrefix = prefix + "\t"; - final List childElements = element.findElements(By.xpath("./*")); - for (WebElement oneChildElement : childElements) { - StringBuilder childStr = elementToString(childPrefix, oneChildElement, newDepth); - if (childStr.length() > 0) { - elementStr.append("\n").append(childStr); - } - } - return elementStr; - } - - /** - * Highlight the element in the UI. Useful for development/debugging. - * Copied from http://www.testingdiaries.com/highlight-element-using-selenium-webdriver/ - * @param element the element to highlight - * @param driver the web driver in use - */ - public static void elementHighlight(WebElement element, WebDriver driver) { - for (int i = 0; i < 2; i++) { - JavascriptExecutor js = (JavascriptExecutor) driver; - js.executeScript( - "arguments[0].setAttribute('style', arguments[1]);", - element, "color: red; border: 3px solid red;"); - TimeUtil.sleepSec(2); - js.executeScript( - "arguments[0].setAttribute('style', arguments[1]);", - element, ""); - } - } -} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java b/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java deleted file mode 100644 index 03991b7b8de..00000000000 --- a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/HdfsWrap.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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 com.hortonworks.storm.st.wrapper; - -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Ordering; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.Assert; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.SortedSet; - -/** - * Created by Raghav Kumar Gautam on 6/24/16. - */ -public class HdfsWrap { - private final FileSystem fileSystem; - private static Logger log = LoggerFactory.getLogger(HdfsWrap.class); - - private HdfsWrap(FileSystem fileSystem) { - this.fileSystem = fileSystem; - } - - public static HdfsWrap getInstance(final FileSystem fileSystem) { - return new HdfsWrap(fileSystem); - } - - - public void deleteDir(String hdfsDir) throws IOException { - Path hdfsPath = new Path(hdfsDir); - log.info("Deleting dir " + hdfsPath + " on filesystem " + fileSystem.getUri()); - fileSystem.delete(hdfsPath, true); - } - - public void createDir(String hdfsDir) throws IOException { - Path hdfsPath = new Path(hdfsDir); - log.info("Creating dir " + hdfsPath + " on filesystem " + fileSystem.getUri()); - fileSystem.mkdirs(hdfsPath); - } - - public SortedSet readSortedRecords(String hdfsDir, final String delimiterRegex) throws IOException { - final List rawOutput = readDataFromHdfs(hdfsDir); - return FluentIterable.from(rawOutput).transformAndConcat(new Function>() { - @Nullable - @Override - public List apply(@Nullable String s) { - if (StringUtils.isEmpty(s)) { - return new ArrayList<>(); - } - return Arrays.asList(s.split(delimiterRegex)); - } - }).toSortedSet(Ordering.natural()); - } - - private List readDataFromHdfs(String hdfsPath) throws IOException { - final File tempDir = new File(FileUtils.getTempDirectory(), "testHdfsSpoutBoltOutput"); - Assert.assertTrue(tempDir.mkdirs(), "temporary dir creation failed: " + tempDir); - fileSystem.copyToLocalFile(new Path(hdfsPath), new Path(tempDir.getAbsolutePath())); - List outputData = new ArrayList<>(); - for (File oneOutputFile : FileUtils.listFiles(tempDir, new String[] {"txt"}, true)) { - outputData.add(FileUtils.readFileToString(oneOutputFile, Charset.defaultCharset())); - } - FileUtils.deleteQuietly(tempDir); - return outputData; - } - - public void writeDataToHdfs(String data, String hdfsDir) throws IOException { - final File tempFile1 = File.createTempFile("data", ".txt"); - FileUtils.writeStringToFile(tempFile1, data, Charset.defaultCharset()); - fileSystem.copyFromLocalFile(new Path(tempFile1.getAbsolutePath()), new Path(hdfsDir)); - FileUtils.deleteQuietly(tempFile1); - } - - public void createHdfsDirWith777(String dir) throws IOException { - final Path path = new Path(dir); - createDir(dir); - fileSystem.setPermission(path, FsPermission.valueOf("drwxrwxrwx")); - } -} diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java b/integration-test/src/test/java/org/apache/storm/st/DemoTest.java similarity index 91% rename from integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java rename to integration-test/src/test/java/org/apache/storm/st/DemoTest.java index edf27d16679..b0112541a1a 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/DemoTest.java +++ b/integration-test/src/test/java/org/apache/storm/st/DemoTest.java @@ -15,23 +15,21 @@ * limitations under the License. */ -package com.hortonworks.storm.st; +package org.apache.storm.st; import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; -import com.hortonworks.storm.st.helper.AbstractTest; -import com.hortonworks.storm.st.utils.AssertUtil; -import com.hortonworks.storm.st.wrapper.TopoWrap; -import org.hortonworks.storm.ExclamationTopology; +import org.apache.storm.st.helper.AbstractTest; +import org.apache.storm.st.wrapper.TopoWrap; +import org.apache.storm.ExclamationTopology; import org.apache.storm.generated.TopologyInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.utils.TimeUtil; import javax.annotation.Nullable; import java.util.Collection; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java b/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java similarity index 91% rename from integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java rename to integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java index c16979a1762..7c6384c0c4b 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/helper/AbstractTest.java +++ b/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package com.hortonworks.storm.st.helper; +package org.apache.storm.st.helper; -import com.hortonworks.storm.st.wrapper.StormCluster; +import org.apache.storm.st.wrapper.StormCluster; /** * Created by temp on 5/2/16. diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java b/integration-test/src/test/java/org/apache/storm/st/meta/TestngListener.java similarity index 92% rename from integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java rename to integration-test/src/test/java/org/apache/storm/st/meta/TestngListener.java index 12529b12298..ed2f9ce4db2 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/meta/TestngListener.java +++ b/integration-test/src/test/java/org/apache/storm/st/meta/TestngListener.java @@ -15,23 +15,17 @@ * limitations under the License. */ -package com.hortonworks.storm.st.meta; +package org.apache.storm.st.meta; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.log4j.Logger; import org.apache.log4j.NDC; -import org.openqa.selenium.OutputType; -import org.openqa.selenium.TakesScreenshot; import org.testng.IExecutionListener; import org.testng.ITestContext; import org.testng.ITestListener; import org.testng.ITestResult; -import java.io.File; -import java.io.IOException; import java.util.Arrays; /** diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java b/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java similarity index 92% rename from integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java rename to integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java index 9e25dd4bc25..c4200a0cc34 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/SlidingWindowTest.java +++ b/integration-test/src/test/java/org/apache/storm/st/tests/window/SlidingWindowTest.java @@ -15,26 +15,24 @@ * limitations under the License. */ -package com.hortonworks.storm.st.tests.window; +package org.apache.storm.st.tests.window; -import com.hortonworks.storm.st.helper.AbstractTest; -import com.hortonworks.storm.st.utils.AssertUtil; -import com.hortonworks.storm.st.wrapper.LogData; -import com.hortonworks.storm.st.wrapper.TopoWrap; +import org.apache.storm.st.helper.AbstractTest; +import org.apache.storm.st.wrapper.LogData; +import org.apache.storm.st.wrapper.TopoWrap; import org.apache.storm.thrift.TException; -import org.hortonworks.storm.st.topology.TestableTopology; -import org.hortonworks.storm.st.topology.window.SlidingTimeCorrectness; -import org.hortonworks.storm.st.topology.window.SlidingWindowCorrectness; -import org.hortonworks.storm.st.topology.window.data.TimeData; -import org.hortonworks.storm.st.topology.window.data.TimeDataWindow; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.topology.TestableTopology; +import org.apache.storm.st.topology.window.SlidingTimeCorrectness; +import org.apache.storm.st.topology.window.SlidingWindowCorrectness; +import org.apache.storm.st.topology.window.data.TimeData; +import org.apache.storm.st.topology.window.data.TimeDataWindow; +import org.apache.storm.st.utils.TimeUtil; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java b/integration-test/src/test/java/org/apache/storm/st/tests/window/TumblingWindowTest.java similarity index 90% rename from integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java rename to integration-test/src/test/java/org/apache/storm/st/tests/window/TumblingWindowTest.java index 6fd5fb9c250..450219d8f7d 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/tests/window/TumblingWindowTest.java +++ b/integration-test/src/test/java/org/apache/storm/st/tests/window/TumblingWindowTest.java @@ -15,13 +15,12 @@ * limitations under the License. */ -package com.hortonworks.storm.st.tests.window; +package org.apache.storm.st.tests.window; -import com.hortonworks.storm.st.helper.AbstractTest; -import com.hortonworks.storm.st.utils.AssertUtil; -import com.hortonworks.storm.st.wrapper.TopoWrap; -import org.hortonworks.storm.st.topology.window.TumblingTimeCorrectness; -import org.hortonworks.storm.st.topology.window.TumblingWindowCorrectness; +import org.apache.storm.st.helper.AbstractTest; +import org.apache.storm.st.wrapper.TopoWrap; +import org.apache.storm.st.topology.window.TumblingTimeCorrectness; +import org.apache.storm.st.topology.window.TumblingWindowCorrectness; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java b/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java similarity index 98% rename from integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java rename to integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java index a999b63d12b..f649cc8b333 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/utils/AssertUtil.java +++ b/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package com.hortonworks.storm.st.utils; +package org.apache.storm.st.utils; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java similarity index 93% rename from integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java rename to integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java index 60268b83656..68ba74954ec 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/LogData.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package com.hortonworks.storm.st.wrapper; +package org.apache.storm.st.wrapper; -import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.storm.st.utils.AssertUtil; import org.apache.commons.lang.StringUtils; import org.joda.time.DateTime; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import org.hortonworks.storm.st.utils.StringDecorator; +import org.apache.storm.st.utils.StringDecorator; import java.util.Arrays; import java.util.List; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java similarity index 97% rename from integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java rename to integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java index b38e10345bc..e9fe5240b0d 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/StormCluster.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package com.hortonworks.storm.st.wrapper; +package org.apache.storm.st.wrapper; import com.google.common.base.Predicate; import com.google.common.collect.Collections2; -import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.storm.st.utils.AssertUtil; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.storm.generated.*; import org.apache.storm.thrift.TException; diff --git a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java similarity index 98% rename from integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java rename to integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java index ddb59760bd7..940622457af 100644 --- a/integration-test/src/test/java/com/hortonworks/storm/st/wrapper/TopoWrap.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package com.hortonworks.storm.st.wrapper; +package org.apache.storm.st.wrapper; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.hortonworks.storm.st.utils.AssertUtil; +import org.apache.storm.st.utils.AssertUtil; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; @@ -30,9 +30,9 @@ import org.apache.storm.StormSubmitter; import org.apache.storm.generated.*; import org.apache.storm.thrift.TException; -import org.hortonworks.storm.st.topology.window.data.FromJson; -import org.hortonworks.storm.st.utils.StringDecorator; -import org.hortonworks.storm.st.utils.TimeUtil; +import org.apache.storm.st.topology.window.data.FromJson; +import org.apache.storm.st.utils.StringDecorator; +import org.apache.storm.st.utils.TimeUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; diff --git a/integration-test/src/test/resources/storm-conf/storm.yaml b/integration-test/src/test/resources/storm-conf/storm.yaml index 8557188b131..3a9df98f01d 100644 --- a/integration-test/src/test/resources/storm-conf/storm.yaml +++ b/integration-test/src/test/resources/storm-conf/storm.yaml @@ -1,95 +1,17 @@ -atlas.cluster.name : 'tstc' -client.jartransformer.class : 'org.apache.storm.hack.StormShadeTransformer' -dev.zookeeper.path : '/tmp/dev-storm-zookeeper' -drpc.childopts : '-Xmx768m ' -drpc.invocations.port : 3773 -drpc.port : 3772 -drpc.queue.size : 128 -drpc.request.timeout.secs : 600 -drpc.worker.threads : 64 -java.library.path : '/usr/local/lib:/opt/local/lib:/usr/lib:/usr/hdp/current/storm-client/lib' -logviewer.appender.name : 'A1' -logviewer.childopts : '-Xmx128m ' -logviewer.port : 8000 -nimbus.childopts : '-Xmx1024m -javaagent:/usr/hdp/current/storm-nimbus/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8649,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-nimbus/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM' -nimbus.cleanup.inbox.freq.secs : 600 -nimbus.file.copy.expiration.secs : 600 -nimbus.inbox.jar.expiration.secs : 3600 -nimbus.monitor.freq.secs : 120 -nimbus.reassign : true -nimbus.seeds : ['c6402.ambari.apache.org'] -nimbus.supervisor.timeout.secs : 60 -nimbus.task.launch.secs : 120 -nimbus.task.timeout.secs : 30 -nimbus.thrift.max_buffer_size : 1048576 -nimbus.thrift.port : 6627 -nimbus.topology.validator : 'org.apache.storm.nimbus.DefaultTopologyValidator' -storm.cluster.mode : 'distributed' -storm.local.dir : '/hadoop/storm' -storm.local.mode.zmq : false -storm.log.dir : '/var/log/storm' -storm.messaging.netty.buffer_size : 5242880 -storm.messaging.netty.client_worker_threads : 1 -storm.messaging.netty.max_retries : 30 -storm.messaging.netty.max_wait_ms : 1000 -storm.messaging.netty.min_wait_ms : 100 -storm.messaging.netty.server_worker_threads : 1 -storm.messaging.transport : 'org.apache.storm.messaging.netty.Context' -storm.thrift.transport : 'org.apache.storm.security.auth.SimpleTransportPlugin' -storm.zookeeper.connection.timeout : 15000 -storm.zookeeper.port : 2181 -storm.zookeeper.retry.interval : 1000 -storm.zookeeper.retry.intervalceiling.millis : 30000 -storm.zookeeper.retry.times : 5 -storm.zookeeper.root : '/storm' -storm.zookeeper.servers : ['c6401.ambari.apache.org','c6402.ambari.apache.org','c6403.ambari.apache.org'] -storm.zookeeper.session.timeout : 20000 -supervisor.childopts : '-Xmx256m -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=56431 -javaagent:/usr/hdp/current/storm-supervisor/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-supervisor/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM' -supervisor.heartbeat.frequency.secs : 5 -supervisor.monitor.frequency.secs : 3 -supervisor.slots.ports : [6700, 6701] -supervisor.worker.start.timeout.secs : 120 -supervisor.worker.timeout.secs : 30 -task.heartbeat.frequency.secs : 3 -task.refresh.poll.secs : 10 -topology.acker.executors : null -topology.builtin.metrics.bucket.size.secs : 60 -topology.debug : false -topology.disruptor.wait.strategy : 'com.lmax.disruptor.BlockingWaitStrategy' -topology.enable.message.timeouts : true -topology.error.throttle.interval.secs : 10 -topology.executor.receive.buffer.size : 1024 -topology.executor.send.buffer.size : 1024 -topology.fall.back.on.java.serialization : true -topology.kryo.factory : 'org.apache.storm.serialization.DefaultKryoFactory' -topology.max.error.report.per.interval : 5 -topology.max.replication.wait.time.sec : 60 -topology.max.spout.pending : 1000 -topology.max.task.parallelism : null -topology.message.timeout.secs : 30 -topology.min.replication.count : 1 -topology.optimize : true -topology.receiver.buffer.size : 8 -topology.skip.missing.kryo.registrations : false -topology.sleep.spout.wait.strategy.time.ms : 1 -topology.spout.wait.strategy : 'org.apache.storm.spout.SleepSpoutWaitStrategy' -topology.state.synchronization.timeout.secs : 60 -topology.stats.sample.rate : 0.05 -topology.tick.tuple.freq.secs : null -topology.transfer.buffer.size : 1024 -topology.trident.batch.emit.interval.millis : 500 -topology.tuple.serializer : 'org.apache.storm.serialization.types.ListDelegateSerializer' -topology.worker.childopts : null -topology.worker.shared.thread.pool.size : 4 -topology.workers : 1 -transactional.zookeeper.port : null -transactional.zookeeper.root : '/transactional' -transactional.zookeeper.servers : null -ui.childopts : '-Xmx768m ' -ui.filter : null -ui.port : 8744 -worker.childopts : '-Xmx768m -javaagent:/usr/hdp/current/storm-client/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-client/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM' -worker.heartbeat.frequency.secs : 1 -zmq.hwm : 0 -zmq.linger.millis : 5000 -zmq.threads : 1 + +storm.zookeeper.servers: + - "node1" + +nimbus.seeds: ["node1"] + +# netty transport +storm.messaging.transport: "org.apache.storm.messaging.netty.Context" +storm.messaging.netty.buffer_size: 16384 +storm.messaging.netty.max_retries: 10 +storm.messaging.netty.min_wait_ms: 1000 +storm.messaging.netty.max_wait_ms: 5000 + +drpc.servers: + - "node1" + +supervisor.slots.ports: [6700, 6701, 6702, 6703, 6704, 6705, 6706, 6707, 6708, 6709] From 0d1336619b308db1ee058906bd695c16346c006b Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Wed, 21 Sep 2016 10:56:32 -0700 Subject: [PATCH 4/6] removing mass imports --- .../org/apache/storm/debug/DebugHelper.java | 3 --- .../storm/st/topology/TestableTopology.java | 3 --- .../window/SlidingTimeCorrectness.java | 5 +++- .../window/TumblingTimeCorrectness.java | 5 +++- .../st/topology/window/data/FromJson.java | 3 --- .../st/topology/window/data/TimeData.java | 3 --- .../topology/window/data/TimeDataWindow.java | 3 --- .../storm/st/utils/StringDecorator.java | 3 --- .../org/apache/storm/st/utils/TimeUtil.java | 3 --- .../apache/storm/st/helper/AbstractTest.java | 3 --- .../org/apache/storm/st/utils/AssertUtil.java | 3 --- .../org/apache/storm/st/wrapper/LogData.java | 4 --- .../apache/storm/st/wrapper/StormCluster.java | 14 ++++++---- .../org/apache/storm/st/wrapper/TopoWrap.java | 26 +++++++++++++++---- 14 files changed, 38 insertions(+), 43 deletions(-) diff --git a/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java b/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java index 3ace3cc4881..97c05544090 100644 --- a/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java +++ b/integration-test/src/main/java/org/apache/storm/debug/DebugHelper.java @@ -24,9 +24,6 @@ import java.net.URL; import java.net.URLClassLoader; -/** - * Created by temp on 6/3/16. - */ public class DebugHelper { private static final Logger LOG = LoggerFactory.getLogger(DebugHelper.class); diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java b/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java index 75777ec1e48..38233109174 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/TestableTopology.java @@ -21,9 +21,6 @@ import java.util.List; -/** - * Created by temp on 5/3/16. - */ public interface TestableTopology { String DUMMY_FIELD = "dummy"; List getExpectedOutput(); diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java index a3c9e33c06e..430449b4e5a 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/SlidingTimeCorrectness.java @@ -40,7 +40,10 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.util.*; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; import java.util.concurrent.TimeUnit; /** diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java index df2aa2df9bb..a77836f865a 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/TumblingTimeCorrectness.java @@ -40,7 +40,10 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.util.*; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; import java.util.concurrent.TimeUnit; /** diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java index 232de09d3f4..d749abfae5b 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/FromJson.java @@ -17,9 +17,6 @@ package org.apache.storm.st.topology.window.data; -/** - * Created by temp on 5/23/16. - */ public interface FromJson { T fromJson(String jsonStr); } diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java index 5a29c228f20..cd2c7a5d055 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeData.java @@ -26,9 +26,6 @@ import java.util.Collection; import java.util.Date; -/** - * Created by temp on 5/12/16. - */ public class TimeData implements Comparable, FromJson { public static final TimeData CLS = new TimeData(-1); private static final String NUMBER_FIELD_NAME = "number"; diff --git a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java index 6c01e93b18b..d6cb9d69cca 100644 --- a/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java +++ b/integration-test/src/main/java/org/apache/storm/st/topology/window/data/TimeDataWindow.java @@ -29,9 +29,6 @@ import java.util.Collections; import java.util.List; -/** - * Created by temp on 5/23/16. - */ public class TimeDataWindow extends ArrayList implements FromJson { public static final TimeDataWindow CLS = new TimeDataWindow(); private static final Type listType = new TypeToken>() {}.getType(); diff --git a/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java b/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java index 0a9cb6dbc87..34c2b65592a 100644 --- a/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java +++ b/integration-test/src/main/java/org/apache/storm/st/utils/StringDecorator.java @@ -19,9 +19,6 @@ import org.apache.commons.lang.StringUtils; -/** - * Created by temp on 5/3/16. - */ public class StringDecorator { private static final String UNIQUE_PREFIX = "---bed91874d79720f7e324c43d49dba4ff---"; diff --git a/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java b/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java index dd97cf33f0a..36bdfdd8610 100644 --- a/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java +++ b/integration-test/src/main/java/org/apache/storm/st/utils/TimeUtil.java @@ -24,9 +24,6 @@ import java.util.concurrent.TimeUnit; -/** - * Created by temp on 4/28/16. - */ public class TimeUtil { private static Logger log = LoggerFactory.getLogger(TimeUtil.class); diff --git a/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java b/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java index 7c6384c0c4b..57c4930d6ab 100644 --- a/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java +++ b/integration-test/src/test/java/org/apache/storm/st/helper/AbstractTest.java @@ -19,9 +19,6 @@ import org.apache.storm.st.wrapper.StormCluster; -/** - * Created by temp on 5/2/16. - */ public abstract class AbstractTest { protected final StormCluster cluster = new StormCluster(); static { diff --git a/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java b/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java index f649cc8b333..2d918921f05 100644 --- a/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java +++ b/integration-test/src/test/java/org/apache/storm/st/utils/AssertUtil.java @@ -26,9 +26,6 @@ import java.util.Collection; import java.util.List; -/** - * Created by temp on 4/28/16. - */ public class AssertUtil { private static Logger log = LoggerFactory.getLogger(AssertUtil.class); diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java index 68ba74954ec..b042713e836 100644 --- a/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/LogData.java @@ -27,10 +27,6 @@ import java.util.Arrays; import java.util.List; -/** - * Created by temp on 5/5/16. - */ - public class LogData implements Comparable { private final DateTime logDate; private final String data; diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java index e9fe5240b0d..7311d5b320b 100644 --- a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java @@ -19,9 +19,13 @@ import com.google.common.base.Predicate; import com.google.common.collect.Collections2; +import org.apache.storm.generated.ClusterSummary; +import org.apache.storm.generated.KillOptions; +import org.apache.storm.generated.Nimbus; +import org.apache.storm.generated.TopologyInfo; +import org.apache.storm.generated.TopologySummary; import org.apache.storm.st.utils.AssertUtil; import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.storm.generated.*; import org.apache.storm.thrift.TException; import org.apache.storm.utils.NimbusClient; import org.apache.storm.utils.Utils; @@ -30,11 +34,11 @@ import org.testng.Assert; import javax.annotation.Nullable; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; -/** - * Created by temp on 4/28/16. - */ public class StormCluster { private static Logger log = LoggerFactory.getLogger(StormCluster.class); private final Nimbus.Client client; diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java index 940622457af..be1db6b84cf 100644 --- a/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java +++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/TopoWrap.java @@ -22,13 +22,23 @@ import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import org.apache.storm.generated.AlreadyAliveException; +import org.apache.storm.generated.AuthorizationException; +import org.apache.storm.generated.ComponentAggregateStats; +import org.apache.storm.generated.ComponentPageInfo; +import org.apache.storm.generated.ExecutorAggregateStats; +import org.apache.storm.generated.ExecutorStats; +import org.apache.storm.generated.ExecutorSummary; +import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.generated.TopologyInfo; +import org.apache.storm.generated.TopologySummary; import org.apache.storm.st.utils.AssertUtil; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.storm.StormSubmitter; -import org.apache.storm.generated.*; import org.apache.storm.thrift.TException; import org.apache.storm.st.topology.window.data.FromJson; import org.apache.storm.st.utils.StringDecorator; @@ -44,12 +54,18 @@ import java.net.MalformedURLException; import java.net.URL; import java.text.NumberFormat; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; import java.util.regex.Pattern; -/** - * Created by temp on 4/29/16. - */ public class TopoWrap { private static Logger log = LoggerFactory.getLogger(TopoWrap.class); private final StormCluster cluster; From 1b231f5ef9d57b41ab5bb312e69df9eed4b4231c Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Wed, 21 Sep 2016 12:09:46 -0700 Subject: [PATCH 5/6] addressing rat issues --- .gitignore | 3 +++ integration-test/config/common.sh | 16 ++++++++++++++++ integration-test/config/config-supervisord.sh | 5 ----- integration-test/config/etc-hosts | 16 ++++++++++++++++ integration-test/config/install-storm.sh | 16 ++++++++++++++++ integration-test/config/install-zookeeper.sh | 16 ++++++++++++++++ integration-test/config/start-supervisord.sh | 1 - integration-test/config/storm.yaml | 16 ++++++++++++++++ integration-test/config/user-script.sh | 1 - .../src/test/resources/storm-conf/storm.yaml | 16 ++++++++++++++++ pom.xml | 3 +++ 11 files changed, 102 insertions(+), 7 deletions(-) delete mode 100644 integration-test/config/config-supervisord.sh delete mode 100644 integration-test/config/start-supervisord.sh delete mode 100644 integration-test/config/user-script.sh diff --git a/.gitignore b/.gitignore index 5c66aba74c8..636570f7e76 100644 --- a/.gitignore +++ b/.gitignore @@ -54,3 +54,6 @@ logs # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml hs_err_pid* + +# ignore vagrant files +/integration-test/config/.vagrant/ \ No newline at end of file diff --git a/integration-test/config/common.sh b/integration-test/config/common.sh index 372c1a2bbc5..c89319c7513 100644 --- a/integration-test/config/common.sh +++ b/integration-test/config/common.sh @@ -1,3 +1,19 @@ +# +# 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. +# USER_SCRIPT="user-script.sh" [[ -f $USER_SCRIPT ]] && echo "Running ${USER_SCRIPT}" && bash ${USER_SCRIPT} || echo "${USER_SCRIPT} not found/executed, continuing." #apt-get update diff --git a/integration-test/config/config-supervisord.sh b/integration-test/config/config-supervisord.sh deleted file mode 100644 index 1a286b1912c..00000000000 --- a/integration-test/config/config-supervisord.sh +++ /dev/null @@ -1,5 +0,0 @@ -echo [program:storm-$1] | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf -echo command=storm $1 | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf -echo directory=/home/storm | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf -echo autorestart=true | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf -echo user=storm | sudo tee -a /etc/supervisor/conf.d/storm-$1.conf \ No newline at end of file diff --git a/integration-test/config/etc-hosts b/integration-test/config/etc-hosts index 3b9032181b1..45e2aa69786 100644 --- a/integration-test/config/etc-hosts +++ b/integration-test/config/etc-hosts @@ -1,2 +1,18 @@ +# +# 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. +# 127.0.0.1 localhost localhost 192.168.100.100 node-1 diff --git a/integration-test/config/install-storm.sh b/integration-test/config/install-storm.sh index 20391e1b718..2731abe5ea9 100644 --- a/integration-test/config/install-storm.sh +++ b/integration-test/config/install-storm.sh @@ -1,3 +1,19 @@ +# +# 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. +# # $1 is the storm binary zip file SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" diff --git a/integration-test/config/install-zookeeper.sh b/integration-test/config/install-zookeeper.sh index 45d186bee56..a81a07c55ee 100644 --- a/integration-test/config/install-zookeeper.sh +++ b/integration-test/config/install-zookeeper.sh @@ -1,3 +1,19 @@ +# +# 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. +# apt-get --yes install zookeeper=3.3.5* zookeeperd=3.3.5* service zookeeper stop echo maxClientCnxns=200 >> /etc/zookeeper/conf/zoo.cfg diff --git a/integration-test/config/start-supervisord.sh b/integration-test/config/start-supervisord.sh deleted file mode 100644 index 37b4cf8c5ef..00000000000 --- a/integration-test/config/start-supervisord.sh +++ /dev/null @@ -1 +0,0 @@ -/etc/init.d/supervisor start \ No newline at end of file diff --git a/integration-test/config/storm.yaml b/integration-test/config/storm.yaml index 3a9df98f01d..eca352fb9a9 100644 --- a/integration-test/config/storm.yaml +++ b/integration-test/config/storm.yaml @@ -1,3 +1,19 @@ +# +# 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. +# storm.zookeeper.servers: - "node1" diff --git a/integration-test/config/user-script.sh b/integration-test/config/user-script.sh deleted file mode 100644 index 0dcf94dfad0..00000000000 --- a/integration-test/config/user-script.sh +++ /dev/null @@ -1 +0,0 @@ -apt-get --yes install emacs tmux diff --git a/integration-test/src/test/resources/storm-conf/storm.yaml b/integration-test/src/test/resources/storm-conf/storm.yaml index 3a9df98f01d..eca352fb9a9 100644 --- a/integration-test/src/test/resources/storm-conf/storm.yaml +++ b/integration-test/src/test/resources/storm-conf/storm.yaml @@ -1,3 +1,19 @@ +# +# 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. +# storm.zookeeper.servers: - "node1" diff --git a/pom.xml b/pom.xml index 5c47f757b7e..7bc566dd3d1 100644 --- a/pom.xml +++ b/pom.xml @@ -429,6 +429,9 @@ **/src/test/resources/FixedAvroSerializer.config + + + integration-test/config/.vagrant/** From 6b32ff2847d819dc8896df932ba16799aab5fa51 Mon Sep 17 00:00:00 2001 From: Raghav Kumar Gautam Date: Wed, 21 Sep 2016 13:31:44 -0700 Subject: [PATCH 6/6] fail fast --- .travis.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 286f1d4b6b5..3aca6a8a42c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -26,8 +26,7 @@ before_install: - nvm use 0.12.2 install: /bin/bash ./dev-tools/travis/travis-install.sh `pwd` script: - - /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES - - /bin/bash ./integration-test/run-it.sh + - /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES && /bin/bash ./integration-test/run-it.sh sudo: true cache: directories: