You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2018/01/02 16:56:01 UTC

[10/37] ambari git commit: AMBARI-22514, AMBARI-22653. Ambari Infra Manager: solr data exporting jobs and integration test environment. (Krisztian Kasa via swagle)

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrDocumentSource.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrDocumentSource.java b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrDocumentSource.java
index 2181ba3..5ded9ac 100644
--- a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrDocumentSource.java
+++ b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrDocumentSource.java
@@ -18,6 +18,8 @@
  */
 package org.apache.ambari.infra.job.archive;
 
+import org.apache.ambari.infra.job.CloseableIterator;
+import org.apache.ambari.infra.job.ObjectSource;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -29,27 +31,29 @@ import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.time.format.DateTimeFormatter;
 
-public class SolrDocumentSource implements DocumentSource {
+public class SolrDocumentSource implements ObjectSource<Document> {
   public static final DateTimeFormatter SOLR_DATETIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX");
   private static final Logger LOG = LoggerFactory.getLogger(SolrDocumentSource.class);
 
-  private final String zkHost;
+  private final String zkConnectionString;
   private final SolrQueryProperties properties;
-  private final String endValue;
+  private final String start;
+  private final String end;
 
-  public SolrDocumentSource(String zkHost, SolrQueryProperties properties, String endValue) {
-    this.zkHost = zkHost;
+  public SolrDocumentSource(String zkConnectionString, SolrQueryProperties properties, String start, String end) {
+    this.zkConnectionString = zkConnectionString;
     this.properties = properties;
-    this.endValue = endValue;
+    this.start = start;
+    this.end = end;
   }
 
   @Override
-  public DocumentIterator open(Document current, int rows) {
-    CloudSolrClient client = new CloudSolrClient.Builder().withZkHost(zkHost).build();
+  public CloseableIterator<Document> open(Document current, int rows) {
+    CloudSolrClient client = new CloudSolrClient.Builder().withZkHost(zkConnectionString).build();
     client.setDefaultCollection(properties.getCollection());
 
     SolrQuery query = properties.toQueryBuilder()
-            .setEndValue(endValue)
+            .setInterval(start, end)
             .setDocument(current)
             .build();
     query.setRows(rows);

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryBuilder.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryBuilder.java b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryBuilder.java
index d0f6d40..b3ea14e 100644
--- a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryBuilder.java
+++ b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryBuilder.java
@@ -25,6 +25,7 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import static org.apache.commons.lang.StringUtils.isBlank;
 import static org.apache.solr.client.solrj.SolrQuery.ORDER.asc;
 
 public class SolrQueryBuilder {
@@ -32,6 +33,7 @@ public class SolrQueryBuilder {
   public static final Pattern PARAMETER_PATTERN = Pattern.compile("\\$\\{[a-z]+\\}");
 
   private String queryText;
+  private String startValue;
   private String endValue;
   private String filterQueryText;
   private Document document;
@@ -51,6 +53,12 @@ public class SolrQueryBuilder {
     return this;
   }
 
+  public SolrQueryBuilder setInterval(String startValue, String endValue) {
+    this.startValue = startValue;
+    this.endValue = endValue;
+    return this;
+  }
+
   public SolrQueryBuilder setFilterQueryText(String filterQueryText) {
     this.filterQueryText = filterQueryText;
     return this;
@@ -71,19 +79,21 @@ public class SolrQueryBuilder {
     SolrQuery solrQuery = new SolrQuery();
 
     String query = queryText;
-    query = setEndValueOn(query);
+    query = setValueOn(query, "${start}", startValue);
+    query = setValueOn(query, "${end}", endValue);
 
     solrQuery.setQuery(query);
 
     if (filterQueryText != null) {
       String filterQuery = filterQueryText;
-      filterQuery = setEndValueOn(filterQuery);
+      filterQuery = setValueOn(filterQuery, "${start}", startValue);
+      filterQuery = setValueOn(filterQuery, "${end}", endValue);
 
       Set<String> paramNames = collectParamNames(filterQuery);
       if (document != null) {
         for (String parameter : paramNames) {
           if (document.get(parameter) != null)
-            filterQuery = filterQuery.replace(String.format("${%s}", parameter), document.get(parameter));
+            filterQuery = filterQuery.replace(String.format("${%s}", parameter), String.format("\"%s\"", document.get(parameter)));
         }
       }
 
@@ -99,10 +109,14 @@ public class SolrQueryBuilder {
     return solrQuery;
   }
 
-  private String setEndValueOn(String query) {
-    if (endValue != null)
-      query = query.replace("${end}", endValue);
-    return query;
+  private String setValueOn(String query, String placeHolder, String value) {
+    if (isBlank(value)) {
+      value = "*";
+    }
+    else {
+      value = '"' + value + '"';
+    }
+    return query.replace(placeHolder, value);
   }
 
   private Set<String> collectParamNames(String filterQuery) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryProperties.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryProperties.java b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryProperties.java
index 444a15b..f062879 100644
--- a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryProperties.java
+++ b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/SolrQueryProperties.java
@@ -19,6 +19,12 @@
 package org.apache.ambari.infra.job.archive;
 
 import org.hibernate.validator.constraints.NotBlank;
+import org.springframework.batch.core.JobParameters;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.commons.lang.StringUtils.isBlank;
 
 public class SolrQueryProperties {
   @NotBlank
@@ -26,7 +32,7 @@ public class SolrQueryProperties {
   @NotBlank
   private String queryText;
   private String filterQueryText;
-  private String[] sort;
+  private String[] sortColumn;
 
   public String getCollection() {
     return collection;
@@ -52,18 +58,40 @@ public class SolrQueryProperties {
     this.filterQueryText = filterQueryText;
   }
 
-  public String[] getSort() {
-    return sort;
+  public String[] getSortColumn() {
+    return sortColumn;
   }
 
-  public void setSort(String[] sort) {
-    this.sort = sort;
+  public void setSortColumn(String[] sortColumn) {
+    this.sortColumn = sortColumn;
   }
 
   public SolrQueryBuilder toQueryBuilder() {
     return new SolrQueryBuilder().
             setQueryText(queryText)
             .setFilterQueryText(filterQueryText)
-            .addSort(sort);
+            .addSort(sortColumn);
+  }
+
+  public void apply(JobParameters jobParameters) {
+    collection = jobParameters.getString("collection", collection);
+    queryText = jobParameters.getString("queryText", queryText);
+    filterQueryText = jobParameters.getString("filterQueryText", filterQueryText);
+
+    String sortValue;
+    List<String> sortColumns = new ArrayList<>();
+    int i = 0;
+    while ((sortValue = jobParameters.getString(String.format("sortColumn[%d]", i))) != null) {
+      sortColumns.add(sortValue);
+      ++i;
+    }
+
+    if (sortColumns.size() > 0)
+      sortColumn = sortColumns.toArray(new String[sortColumns.size()]);
+  }
+
+  public void validate() {
+    if (isBlank(collection))
+      throw new IllegalArgumentException("The property collection can not be null or empty string!");
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/TarGzCompressor.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/TarGzCompressor.java b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/TarGzCompressor.java
index 8e34ca9..55ba58a 100644
--- a/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/TarGzCompressor.java
+++ b/ambari-infra/ambari-infra-manager/src/main/java/org/apache/ambari/infra/job/archive/TarGzCompressor.java
@@ -25,7 +25,7 @@ import org.apache.commons.io.IOUtils;
 
 import java.io.*;
 
-public class TarGzCompressor implements FileAction {
+public class TarGzCompressor extends AbstractFileAction {
   @Override
   public File perform(File inputFile) {
     File tarGzFile = new File(inputFile.getParent(), inputFile.getName() + ".tar.gz");

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/main/resources/infra-manager.properties
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/main/resources/infra-manager.properties b/ambari-infra/ambari-infra-manager/src/main/resources/infra-manager.properties
index 7ef70aa..27b36b3 100644
--- a/ambari-infra/ambari-infra-manager/src/main/resources/infra-manager.properties
+++ b/ambari-infra/ambari-infra-manager/src/main/resources/infra-manager.properties
@@ -20,13 +20,41 @@ management.security.enabled=false
 management.health.solr.enabled=false
 infra-manager.server.data.folder=/tmp
 
-infra-manager.jobs.solr_data_export.zoo_keeper_socket=zookeeper:2181
-infra-manager.jobs.solr_data_export.read_block_size=100
-infra-manager.jobs.solr_data_export.write_block_size=150
-infra-manager.jobs.solr_data_export.file_name_suffix_column=logtime
-infra-manager.jobs.solr_data_export.destination_directory_path=/tmp/ambariInfraManager
-infra-manager.jobs.solr_data_export.query.collection=hadoop_logs
-infra-manager.jobs.solr_data_export.query.query_text=logtime:[* TO "${end}"]
-infra-manager.jobs.solr_data_export.query.filter_query_text=(logtime:"${logtime}" AND id:{"${id}" TO *]) OR logtime:{"${logtime}" TO "${end}"]
-infra-manager.jobs.solr_data_export.query.sort[0]=logtime
-infra-manager.jobs.solr_data_export.query.sort[1]=id
+infra-manager.jobs.solr_data_export.export_service_logs.zoo_keeper_connection_string=zookeeper:2181
+infra-manager.jobs.solr_data_export.export_service_logs.read_block_size=100
+infra-manager.jobs.solr_data_export.export_service_logs.write_block_size=150
+infra-manager.jobs.solr_data_export.export_service_logs.file_name_suffix_column=logtime
+infra-manager.jobs.solr_data_export.export_service_logs.destination_directory_path=/tmp/ambariInfraManager
+infra-manager.jobs.solr_data_export.export_service_logs.query.collection=hadoop_logs
+infra-manager.jobs.solr_data_export.export_service_logs.query.query_text=logtime:[${start} TO ${end}]
+infra-manager.jobs.solr_data_export.export_service_logs.query.filter_query_text=(logtime:${logtime} AND id:{${id} TO *]) OR logtime:{${logtime} TO ${end}]
+infra-manager.jobs.solr_data_export.export_service_logs.query.sort_column[0]=logtime
+infra-manager.jobs.solr_data_export.export_service_logs.query.sort_column[1]=id
+infra-manager.jobs.solr_data_export.export_audit_logs.zoo_keeper_connection_string=zookeeper:2181
+infra-manager.jobs.solr_data_export.export_audit_logs.read_block_size=100
+infra-manager.jobs.solr_data_export.export_audit_logs.write_block_size=150
+# TODO: logtime may not be enough: The same filename can be generated when more than write_block_size count docs has the same logtime value
+infra-manager.jobs.solr_data_export.export_audit_logs.file_name_suffix_column=logtime
+infra-manager.jobs.solr_data_export.export_audit_logs.destination_directory_path=/tmp/ambariInfraManager
+infra-manager.jobs.solr_data_export.export_audit_logs.query.collection=audit_logs
+infra-manager.jobs.solr_data_export.export_audit_logs.query.query_text=logtime:[${start} TO ${end}]
+infra-manager.jobs.solr_data_export.export_audit_logs.query.filter_query_text=(logtime:${logtime} AND id:{${id} TO *]) OR logtime:{${logtime} TO ${end}]
+infra-manager.jobs.solr_data_export.export_audit_logs.query.sort_column[0]=logtime
+infra-manager.jobs.solr_data_export.export_audit_logs.query.sort_column[1]=id
+# TODO: s3_access_key and s3_secret_key to separate file
+infra-manager.jobs.solr_data_export.export_audit_logs.s3_access_key=remote-identity
+infra-manager.jobs.solr_data_export.export_audit_logs.s3_secret_key=remote-credential
+infra-manager.jobs.solr_data_export.export_audit_logs.s3_key_prefix=solr_archive_
+infra-manager.jobs.solr_data_export.export_audit_logs.s3_bucket_name=testbucket
+infra-manager.jobs.solr_data_export.export_audit_logs.s3_endpoint=http://fakes3:4569
+# TODO: configure ranger audit logs
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.zoo_keeper_connection_string=zookeeper:2181
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.read_block_size=100
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.write_block_size=150
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.file_name_suffix_column=logtime
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.destination_directory_path=/tmp/ambariInfraManager
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.query.collection=hadoop_logs
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.query.query_text=logtime:[* TO "${end}"]
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.query.filter_query_text=(logtime:"${logtime}" AND id:{"${id}" TO *]) OR logtime:{"${logtime}" TO "${end}"]
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.query.sort_column[0]=logtime
+#infra-manager.jobs.solr_data_export.export_ranger_audit_logs.query.sort_column[1]=id

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentExportPropertiesTest.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentExportPropertiesTest.java b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentExportPropertiesTest.java
new file mode 100644
index 0000000..ae93710
--- /dev/null
+++ b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentExportPropertiesTest.java
@@ -0,0 +1,54 @@
+package org.apache.ambari.infra.job.archive;
+
+import org.junit.Test;
+
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertThat;
+
+/*
+ * 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.
+ */
+public class DocumentExportPropertiesTest {
+  @Test
+  public void testDeepCopy() throws Exception {
+    DocumentExportProperties documentExportProperties = new DocumentExportProperties();
+    documentExportProperties.setDestinationDirectoryPath("/tmp");
+    documentExportProperties.setFileNameSuffixColumn(".json");
+    documentExportProperties.setReadBlockSize(10);
+    documentExportProperties.setWriteBlockSize(20);
+    documentExportProperties.setZooKeeperConnectionString("localhost:2181");
+    SolrQueryProperties query = new SolrQueryProperties();
+    query.setFilterQueryText("id:1167");
+    query.setQueryText("name:'Joe'");
+    query.setCollection("Users");
+    query.setSortColumn(new String[] {"name"});
+    documentExportProperties.setQuery(query);
+
+    DocumentExportProperties parsed = documentExportProperties.deepCopy();
+
+    assertThat(parsed.getDestinationDirectoryPath(), is(documentExportProperties.getDestinationDirectoryPath()));
+    assertThat(parsed.getFileNameSuffixColumn(), is(documentExportProperties.getFileNameSuffixColumn()));
+    assertThat(parsed.getReadBlockSize(), is(documentExportProperties.getReadBlockSize()));
+    assertThat(parsed.getWriteBlockSize(), is(documentExportProperties.getWriteBlockSize()));
+    assertThat(parsed.getZooKeeperConnectionString(), is(documentExportProperties.getZooKeeperConnectionString()));
+    assertThat(parsed.getQuery().getQueryText(), is(query.getQueryText()));
+    assertThat(parsed.getQuery().getFilterQueryText(), is(query.getFilterQueryText()));
+    assertThat(parsed.getQuery().getCollection(), is(query.getCollection()));
+    assertThat(parsed.getQuery().getSortColumn(), is(query.getSortColumn()));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentItemReaderTest.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentItemReaderTest.java b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentItemReaderTest.java
index 942713f..0776c3c 100644
--- a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentItemReaderTest.java
+++ b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/DocumentItemReaderTest.java
@@ -19,6 +19,8 @@
 
 package org.apache.ambari.infra.job.archive;
 
+import org.apache.ambari.infra.job.CloseableIterator;
+import org.apache.ambari.infra.job.ObjectSource;
 import org.easymock.EasyMockRunner;
 import org.easymock.EasyMockSupport;
 import org.easymock.Mock;
@@ -45,11 +47,11 @@ public class DocumentItemReaderTest extends EasyMockSupport {
 
   private DocumentItemReader documentItemReader;
   @Mock
-  private DocumentSource documentSource;
+  private ObjectSource<Document> documentSource;
   @Mock
-  private DocumentIterator documentIterator;
+  private CloseableIterator<Document> documentIterator;
   @Mock
-  private DocumentIterator documentIterator2;
+  private CloseableIterator<Document> documentIterator2;
 
   @Before
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/LocalDocumentItemWriterTest.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/LocalDocumentItemWriterTest.java b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/LocalDocumentItemWriterTest.java
index 6411ff1..3af93bc 100644
--- a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/LocalDocumentItemWriterTest.java
+++ b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/LocalDocumentItemWriterTest.java
@@ -35,7 +35,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
-import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
@@ -49,12 +49,12 @@ public class LocalDocumentItemWriterTest extends EasyMockSupport {
   private LocalDocumentItemWriter localDocumentItemWriter;
   private File outFile;
   @Mock
-  private FileAction fileAction;
+  private ItemWriterListener itemWriterListener;
 
   @Before
   public void setUp() throws Exception {
     outFile = File.createTempFile("LocalDocumentItemWriterTest", "json.tmp");
-    localDocumentItemWriter = new LocalDocumentItemWriter(outFile, fileAction);
+    localDocumentItemWriter = new LocalDocumentItemWriter(outFile, itemWriterListener);
   }
 
   @After
@@ -65,7 +65,7 @@ public class LocalDocumentItemWriterTest extends EasyMockSupport {
 
   @Test
   public void testWrite() throws Exception {
-    expect(fileAction.perform(outFile)).andReturn(outFile);
+    itemWriterListener.onCompleted(outFile); expectLastCall();
     replayAll();
 
     localDocumentItemWriter.write(DOCUMENT);

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryBuilderTest.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryBuilderTest.java b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryBuilderTest.java
index 4d5ebf1..e9513dc 100644
--- a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryBuilderTest.java
+++ b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryBuilderTest.java
@@ -48,7 +48,7 @@ public class SolrQueryBuilderTest {
   @Test
   public void testSetQuery() throws Exception {
     SolrQuery solrQuery = new SolrQueryBuilder()
-            .setQueryText("logtime:[* TO \"${end}\"]")
+            .setQueryText("logtime:[* TO ${end}]")
             .setEndValue("2017-11-27'T'10:12:11.372Z")
             .build();
     assertThat(solrQuery.getQuery(), is("logtime:[* TO \"2017-11-27'T'10:12:11.372Z\"]"));
@@ -57,7 +57,7 @@ public class SolrQueryBuilderTest {
   @Test
   public void testSetFilterQuery() throws Exception {
     SolrQuery solrQuery = new SolrQueryBuilder()
-            .setFilterQueryText("(logtime:\"${logtime}\" AND id:{\"${id}\" TO *]) OR logtime:{\"${logtime}\" TO \"${end}\"]")
+            .setFilterQueryText("(logtime:${logtime} AND id:{${id} TO *]) OR logtime:{${logtime} TO ${end}]")
             .setDocument(DOCUMENT)
             .setEndValue("2017-11-27'T'10:12:11.372Z")
             .build();
@@ -76,7 +76,7 @@ public class SolrQueryBuilderTest {
   @Test
   public void testSetFilterQueryWhenEndValueIsNull() throws Exception {
     SolrQuery solrQuery = new SolrQueryBuilder()
-            .setFilterQueryText("logtime:\"${logtime}\" AND id:{\"${id}\" TO *]")
+            .setFilterQueryText("logtime:${logtime} AND id:{${id} TO *]")
             .setDocument(DOCUMENT)
             .build();
     assertThat(solrQuery.getFilterQueries()[0], is("logtime:\"2017-10-02'T'10:00:11.634Z\" AND id:{\"1\" TO *]"));
@@ -110,4 +110,16 @@ public class SolrQueryBuilderTest {
     assertThat(solrQuery.getSorts().get(0).getItem(), is("logtime"));
     assertThat(solrQuery.getSorts().get(1).getItem(), is("id"));
   }
+
+  @Test
+  public void test_start_and_end_values_are_given() throws Exception {
+    SolrQuery solrQuery = new SolrQueryBuilder().setQueryText("id:[${start} TO ${end}]").setInterval("10", "13").build();
+    assertThat(solrQuery.getQuery(), is("id:[\"10\" TO \"13\"]"));
+  }
+
+  @Test
+  public void test_start_and_end_values_are_null() throws Exception {
+    SolrQuery solrQuery = new SolrQueryBuilder().setQueryText("id:[${start} TO ${end}]").build();
+    assertThat(solrQuery.getQuery(), is("id:[* TO *]"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryPropertiesTest.java
----------------------------------------------------------------------
diff --git a/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryPropertiesTest.java b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryPropertiesTest.java
new file mode 100644
index 0000000..322775e
--- /dev/null
+++ b/ambari-infra/ambari-infra-manager/src/test/java/org/apache/ambari/infra/job/archive/SolrQueryPropertiesTest.java
@@ -0,0 +1,54 @@
+package org.apache.ambari.infra.job.archive;
+
+import org.junit.Test;
+import org.springframework.batch.core.JobParameters;
+import org.springframework.batch.core.JobParametersBuilder;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/*
+ * 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.
+ */
+public class SolrQueryPropertiesTest {
+  @Test
+  public void testApplySortColumns() throws Exception {
+    JobParameters jobParameters = new JobParametersBuilder()
+            .addString("sortColumn[0]", "logtime")
+            .addString("sortColumn[1]", "id")
+            .toJobParameters();
+
+    SolrQueryProperties solrQueryProperties = new SolrQueryProperties();
+    solrQueryProperties.setSortColumn(new String[] {"testColumn"});
+    solrQueryProperties.apply(jobParameters);
+    assertThat(solrQueryProperties.getSortColumn().length, is(2));
+    assertThat(solrQueryProperties.getSortColumn()[0], is("logtime"));
+    assertThat(solrQueryProperties.getSortColumn()[1], is("id"));
+  }
+
+  @Test
+  public void testApplyWhenNoSortIsDefined() throws Exception {
+    JobParameters jobParameters = new JobParametersBuilder()
+            .toJobParameters();
+
+    SolrQueryProperties solrQueryProperties = new SolrQueryProperties();
+    solrQueryProperties.setSortColumn(new String[] {"testColumn"});
+    solrQueryProperties.apply(jobParameters);
+    assertThat(solrQueryProperties.getSortColumn().length, is(1));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/36d0271f/ambari-infra/pom.xml
----------------------------------------------------------------------
diff --git a/ambari-infra/pom.xml b/ambari-infra/pom.xml
index e4b6a76..9e7a71b 100644
--- a/ambari-infra/pom.xml
+++ b/ambari-infra/pom.xml
@@ -23,7 +23,6 @@
     <version>2.0.0.0-SNAPSHOT</version>
     <relativePath>../ambari-project</relativePath>
   </parent>
-  <groupId>org.apache.ambari</groupId>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>ambari-infra</artifactId>
   <version>2.0.0.0-SNAPSHOT</version>
@@ -44,6 +43,7 @@
     <module>ambari-infra-solr-client</module>
     <module>ambari-infra-solr-plugin</module>
     <module>ambari-infra-manager</module>
+    <module>ambari-infra-manager-it</module>
   </modules>
 
   <build>
@@ -161,6 +161,9 @@
             <exclude>**/*.json</exclude>
             <exclude>**/*.log</exclude>
             <exclude>**/*.txt</exclude>
+            <exclude>**/docker/Profile</exclude>
+            <exclude>**/docker/.env</exclude>
+            <exclude>**/*.story</exclude>
           </excludes>
         </configuration>
         <executions>