You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jl...@apache.org on 2017/05/17 20:51:47 UTC

[13/25] ambari git commit: AMBARI-20881 Add Log Level Filter to the Log Search config API (mgergely)

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/configurer/LogfeederFilterConfigurer.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/configurer/LogfeederFilterConfigurer.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/configurer/LogfeederFilterConfigurer.java
deleted file mode 100644
index c2d27f9..0000000
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/configurer/LogfeederFilterConfigurer.java
+++ /dev/null
@@ -1,66 +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.apache.ambari.logsearch.configurer;
-
-import org.apache.ambari.logsearch.conf.SolrPropsConfig;
-import org.apache.ambari.logsearch.conf.global.SolrCollectionState;
-import org.apache.ambari.logsearch.dao.UserConfigSolrDao;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LogfeederFilterConfigurer implements Configurer {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LogfeederFilterConfigurer.class);
-
-  private static final int SETUP_RETRY_SECOND = 10;
-
-  private final UserConfigSolrDao userConfigSolrDao;
-
-  public LogfeederFilterConfigurer(final UserConfigSolrDao userConfigSolrDao) {
-    this.userConfigSolrDao = userConfigSolrDao;
-  }
-
-  @Override
-  public void start() {
-    final SolrPropsConfig solrPropsConfig = userConfigSolrDao.getSolrPropsConfig();
-    final SolrCollectionState state = userConfigSolrDao.getSolrCollectionState();
-    Thread setupFiltersThread = new Thread("logfeeder_filter_setup") {
-      @Override
-      public void run() {
-        LOG.info("logfeeder_filter_setup thread started (to upload logfeeder config)");
-        while (true) {
-          int retryCount = 0;
-          try {
-            retryCount++;
-            Thread.sleep(SETUP_RETRY_SECOND * 1000);
-            if (state.isSolrCollectionReady()) {
-              LOG.info("Tries to initialize logfeeder filters in '{}' collection", solrPropsConfig.getCollection());
-              userConfigSolrDao.getUserFilter();
-              break;
-            }
-          } catch (Exception e) {
-            LOG.error("Not able to save logfeeder filter while initialization, retryCount=" + retryCount, e);
-          }
-        }
-      }
-    };
-    setupFiltersThread.setDaemon(true);
-    setupFiltersThread.start();
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/dao/UserConfigSolrDao.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/dao/UserConfigSolrDao.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/dao/UserConfigSolrDao.java
index 8fb27a7..a0e01a3 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/dao/UserConfigSolrDao.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/dao/UserConfigSolrDao.java
@@ -20,45 +20,25 @@
 package org.apache.ambari.logsearch.dao;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeMap;
 
 import javax.annotation.PostConstruct;
 import javax.inject.Inject;
 import javax.inject.Named;
 
-import org.apache.ambari.logsearch.common.HadoopServiceConfigHelper;
-import org.apache.ambari.logsearch.common.LogSearchConstants;
 import org.apache.ambari.logsearch.common.LogSearchContext;
 import org.apache.ambari.logsearch.common.LogType;
 import org.apache.ambari.logsearch.conf.SolrPropsConfig;
 import org.apache.ambari.logsearch.conf.SolrUserPropsConfig;
 import org.apache.ambari.logsearch.conf.global.SolrCollectionState;
-import org.apache.ambari.logsearch.configurer.LogfeederFilterConfigurer;
 import org.apache.ambari.logsearch.configurer.SolrCollectionConfigurer;
-import org.apache.ambari.logsearch.model.common.LogFeederDataMap;
-import org.apache.ambari.logsearch.model.common.LogfeederFilterData;
-import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 
-import org.apache.ambari.logsearch.util.JSONUtil;
-import org.apache.commons.collections.CollectionUtils;
 import org.apache.log4j.Logger;
 import org.springframework.data.solr.core.SolrTemplate;
 
-import static org.apache.ambari.logsearch.solr.SolrConstants.UserConfigConstants.ID;
-import static org.apache.ambari.logsearch.solr.SolrConstants.UserConfigConstants.USER_NAME;
-import static org.apache.ambari.logsearch.solr.SolrConstants.UserConfigConstants.VALUES;
-import static org.apache.ambari.logsearch.solr.SolrConstants.UserConfigConstants.FILTER_NAME;
-import static org.apache.ambari.logsearch.solr.SolrConstants.UserConfigConstants.ROW_TYPE;
-
 @Named
 public class UserConfigSolrDao extends SolrDaoBase {
 
@@ -99,25 +79,12 @@ public class UserConfigSolrDao extends SolrDaoBase {
 
     try {
       new SolrCollectionConfigurer(this, false).start();
-      new LogfeederFilterConfigurer(this).start();
     } catch (Exception e) {
       LOG.error("error while connecting to Solr for history logs : solrUrl=" + solrUrl + ", zkConnectString=" + zkConnectString +
           ", collection=" + collection, e);
     }
   }
 
-  public void saveUserFilter(LogFeederDataMap logfeederFilterWrapper) throws SolrException, SolrServerException, IOException {
-    String filterName = LogSearchConstants.LOGFEEDER_FILTER_NAME;
-    String json = JSONUtil.objToJson(logfeederFilterWrapper);
-    SolrInputDocument configDocument = new SolrInputDocument();
-    configDocument.addField(ID, logfeederFilterWrapper.getId());
-    configDocument.addField(ROW_TYPE, filterName);
-    configDocument.addField(VALUES, json);
-    configDocument.addField(USER_NAME, filterName);
-    configDocument.addField(FILTER_NAME, filterName);
-    addDocs(configDocument);
-  }
-
   public void deleteUserConfig(String id) throws SolrException, SolrServerException, IOException {
     removeDoc("id:" + id);
   }
@@ -138,52 +105,6 @@ public class UserConfigSolrDao extends SolrDaoBase {
     return updateResoponse;
   }
 
-  public LogFeederDataMap getUserFilter() throws SolrServerException, IOException {
-    SolrQuery solrQuery = new SolrQuery();
-    solrQuery.setQuery("*:*");
-    solrQuery.setFilterQueries(ROW_TYPE + ":" + LogSearchConstants.LOGFEEDER_FILTER_NAME);
-
-    QueryResponse response = process(solrQuery);
-    SolrDocumentList documentList = response.getResults();
-    LogFeederDataMap logfeederDataMap = null;
-    if (CollectionUtils.isNotEmpty(documentList)) {
-      SolrDocument configDoc = documentList.get(0);
-      String json = (String) configDoc.get(VALUES);
-      logfeederDataMap = (LogFeederDataMap) JSONUtil.jsonToObj(json, LogFeederDataMap.class);
-      logfeederDataMap.setId("" + configDoc.get(ID));
-    } else {
-      logfeederDataMap = new LogFeederDataMap();
-      logfeederDataMap.setFilter(new TreeMap<String, LogfeederFilterData>());
-      logfeederDataMap.setId(Long.toString(System.currentTimeMillis()));
-    }
-    
-    addMissingFilters(logfeederDataMap);
-    
-    return logfeederDataMap;
-  }
-
-  private void addMissingFilters(LogFeederDataMap logfeederDataMap) throws SolrServerException, IOException {
-    Set<String> logIds = HadoopServiceConfigHelper.getAllLogIds();
-    if (logIds != null) {
-      List<String> logfeederDefaultLevels = solrUserConfig.getLogLevels();
-      
-      boolean modified = false;
-      for (String logId : logIds) {
-        if (!logfeederDataMap.getFilter().containsKey(logId)) {
-          LogfeederFilterData logfeederFilterData = new LogfeederFilterData();
-          logfeederFilterData.setLabel(logId);
-          logfeederFilterData.setDefaultLevels(logfeederDefaultLevels);
-          logfeederDataMap.getFilter().put(logId, logfeederFilterData);
-          modified = true;
-        }
-      }
-      
-      if (modified) {
-        saveUserFilter(logfeederDataMap);
-      }
-    }
-  }
-
   @Override
   public SolrCollectionState getSolrCollectionState() {
     return solrUserConfigState;

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/doc/DocConstants.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/doc/DocConstants.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/doc/DocConstants.java
index 00adb67..885771d 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/doc/DocConstants.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/doc/DocConstants.java
@@ -113,15 +113,15 @@ public class DocConstants {
     public static final String SAVE_USER_CONFIG_OD = "Save user config";
     public static final String DELETE_USER_CONFIG_OD = "Delete user config";
     public static final String GET_USER_CONFIG_OD = "Get user config";
-    public static final String GET_USER_FILTER_OD = "Get user filter";
-    public static final String UPDATE_USER_FILTER_OD = "Update user filter";
     public static final String GET_ALL_USER_NAMES_OD = "Get all user names";
   }
 
   public class ShipperConfigOperationDescriptions {
-    public static final String GET_SERVICE_NAMES = "Get service names";
-    public static final String GET_SHIPPER_CONFIG = "Get shipper config";
-    public static final String SET_SHIPPER_CONFIG = "Set shipper config";
+    public static final String GET_SERVICE_NAMES_OD = "Get service names";
+    public static final String GET_SHIPPER_CONFIG_OD = "Get shipper config";
+    public static final String SET_SHIPPER_CONFIG_OD = "Set shipper config";
+    public static final String GET_LOG_LEVEL_FILTER_OD = "Get log level filter";
+    public static final String UPDATE_LOG_LEVEL_FILTER_OD = "Update log level filter";
   }
 
   public class StatusOperationDescriptions {

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/ShipperConfigManager.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/ShipperConfigManager.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/ShipperConfigManager.java
index c0c1167..1118233 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/ShipperConfigManager.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/ShipperConfigManager.java
@@ -22,11 +22,15 @@ package org.apache.ambari.logsearch.manager;
 import java.util.List;
 
 import org.apache.ambari.logsearch.configurer.LogSearchConfigConfigurer;
+import org.apache.ambari.logsearch.model.common.LSServerLogLevelFilterMap;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.ImmutableMap;
+
 import javax.annotation.PostConstruct;
 import javax.inject.Inject;
 import javax.inject.Named;
+import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
 @Named
@@ -50,12 +54,51 @@ public class ShipperConfigManager extends JsonManagerBase {
     return LogSearchConfigConfigurer.getConfig().getInputConfig(clusterName, serviceName);
   }
 
+  public Response createInputConfig(String clusterName, String serviceName, String inputConfig) {
+    
+    try {
+      if (LogSearchConfigConfigurer.getConfig().inputConfigExists(clusterName, serviceName)) {
+        return Response.serverError()
+            .type(MediaType.APPLICATION_JSON)
+            .entity(ImmutableMap.of("errorMessage", "Input config already exists for service " + serviceName))
+            .build();
+      }
+      
+      LogSearchConfigConfigurer.getConfig().createInputConfig(clusterName, serviceName, inputConfig);
+      return Response.ok().build();
+    } catch (Exception e) {
+      logger.warn("Could not create input config", e);
+      return Response.serverError().build();
+    }
+  }
+
   public Response setInputConfig(String clusterName, String serviceName, String inputConfig) {
     try {
+      if (!LogSearchConfigConfigurer.getConfig().inputConfigExists(clusterName, serviceName)) {
+        return Response.serverError()
+            .type(MediaType.APPLICATION_JSON)
+            .entity(ImmutableMap.of("errorMessage", "Input config doesn't exist for service " + serviceName))
+            .build();
+      }
+      
       LogSearchConfigConfigurer.getConfig().setInputConfig(clusterName, serviceName, inputConfig);
       return Response.ok().build();
     } catch (Exception e) {
-      logger.warn("Could not write input config", e);
+      logger.warn("Could not update input config", e);
+      return Response.serverError().build();
+    }
+  }
+
+  public LSServerLogLevelFilterMap getLogLevelFilters(String clusterName) {
+    return new LSServerLogLevelFilterMap(LogSearchConfigConfigurer.getConfig().getLogLevelFilters(clusterName));
+  }
+
+  public Response setLogLevelFilters(String clusterName, LSServerLogLevelFilterMap request) {
+    try {
+      LogSearchConfigConfigurer.getConfig().setLogLevelFilters(clusterName, request.convertToApi());
+      return Response.ok().build();
+    } catch (Exception e) {
+      logger.warn("Could not update log level filters", e);
       return Response.serverError().build();
     }
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/UserConfigManager.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/UserConfigManager.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/UserConfigManager.java
index a60fc5c..1df9f5a 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/UserConfigManager.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/manager/UserConfigManager.java
@@ -27,7 +27,6 @@ import java.util.List;
 import org.apache.ambari.logsearch.common.LogSearchContext;
 import org.apache.ambari.logsearch.common.MessageEnums;
 import org.apache.ambari.logsearch.dao.UserConfigSolrDao;
-import org.apache.ambari.logsearch.model.common.LogFeederDataMap;
 import org.apache.ambari.logsearch.model.request.impl.UserConfigRequest;
 import org.apache.ambari.logsearch.model.response.UserConfigData;
 import org.apache.ambari.logsearch.model.response.UserConfigDataListResponse;
@@ -176,29 +175,6 @@ public class UserConfigManager extends JsonManagerBase {
 
   }
 
-  // ////////////////////////////LEVEL FILTER/////////////////////////////////////
-
-  public LogFeederDataMap getUserFilter() {
-    LogFeederDataMap userFilter;
-    try {
-      userFilter = userConfigSolrDao.getUserFilter();
-    } catch (SolrServerException | IOException e) {
-      logger.error(e);
-      throw RESTErrorUtil.createRESTException(MessageEnums.SOLR_ERROR.getMessage().getMessage(), MessageEnums.ERROR_SYSTEM);
-    }
-    return userFilter;
-  }
-
-  public LogFeederDataMap saveUserFiter(LogFeederDataMap logfeederFilters) {
-    try {
-      userConfigSolrDao.saveUserFilter(logfeederFilters);
-    } catch (SolrException | SolrServerException | IOException e) {
-      logger.error("user config not able to save", e);
-      throw RESTErrorUtil.createRESTException(MessageEnums.SOLR_ERROR.getMessage().getMessage(), MessageEnums.ERROR_SYSTEM);
-    }
-    return getUserFilter();
-  }
-
   public List<String> getAllUserName() {
     List<String> userList = new ArrayList<String>();
     try {

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilter.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilter.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilter.java
new file mode 100644
index 0000000..2a00802
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilter.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 org.apache.ambari.logsearch.model.common;
+
+import java.util.Date;
+import java.util.List;
+
+import org.apache.ambari.logsearch.config.api.model.loglevelfilter.LogLevelFilter;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+@ApiModel
+public class LSServerLogLevelFilter {
+
+  @ApiModelProperty private String label;
+  @ApiModelProperty private List<String> hosts;
+  @ApiModelProperty private List<String> defaultLevels;
+  @ApiModelProperty private List<String> overrideLevels;
+  @ApiModelProperty private Date expiryTime;
+
+  public LSServerLogLevelFilter() {}
+
+  public LSServerLogLevelFilter(LogLevelFilter logLevelFilter) {
+    label = logLevelFilter.getLabel();
+    hosts = logLevelFilter.getHosts();
+    defaultLevels = logLevelFilter.getDefaultLevels();
+    overrideLevels = logLevelFilter.getOverrideLevels();
+    expiryTime = logLevelFilter.getExpiryTime();
+  }
+
+  public String getLabel() {
+    return label;
+  }
+
+  public void setLabel(String label) {
+    this.label = label;
+  }
+
+  public List<String> getHosts() {
+    return hosts;
+  }
+
+  public void setHosts(List<String> hosts) {
+    this.hosts = hosts;
+  }
+
+  public List<String> getDefaultLevels() {
+    return defaultLevels;
+  }
+
+  public void setDefaultLevels(List<String> defaultLevels) {
+    this.defaultLevels = defaultLevels;
+  }
+
+  public List<String> getOverrideLevels() {
+    return overrideLevels;
+  }
+
+  public void setOverrideLevels(List<String> overrideLevels) {
+    this.overrideLevels = overrideLevels;
+  }
+
+  public Date getExpiryTime() {
+    return expiryTime;
+  }
+
+  public void setExpiryTime(Date expiryTime) {
+    this.expiryTime = expiryTime;
+  }
+
+  public LogLevelFilter convertToApi() {
+    LogLevelFilter apiFilter = new LogLevelFilter();
+    
+    apiFilter.setLabel(label);
+    apiFilter.setHosts(hosts);
+    apiFilter.setDefaultLevels(defaultLevels);
+    apiFilter.setOverrideLevels(overrideLevels);
+    apiFilter.setExpiryTime(expiryTime);
+    
+    return apiFilter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilterMap.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilterMap.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilterMap.java
new file mode 100644
index 0000000..3088db1
--- /dev/null
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LSServerLogLevelFilterMap.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.ambari.logsearch.model.common;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.ambari.logsearch.config.api.model.loglevelfilter.LogLevelFilter;
+import org.apache.ambari.logsearch.config.api.model.loglevelfilter.LogLevelFilterMap;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+@ApiModel
+public class LSServerLogLevelFilterMap {
+
+  @ApiModelProperty
+  private TreeMap<String, LSServerLogLevelFilter> filter;
+
+  public LSServerLogLevelFilterMap() {}
+
+  public LSServerLogLevelFilterMap(LogLevelFilterMap logLevelFilterMap) {
+    filter = new TreeMap<>();
+    for (Map.Entry<String, LogLevelFilter> e : logLevelFilterMap.getFilter().entrySet()) {
+      filter.put(e.getKey(), new LSServerLogLevelFilter(e.getValue()));
+    }
+  }
+
+  public TreeMap<String, LSServerLogLevelFilter> getFilter() {
+    return filter;
+  }
+
+  public void setFilter(TreeMap<String, LSServerLogLevelFilter> filter) {
+    this.filter = filter;
+  }
+
+  public LogLevelFilterMap convertToApi() {
+    LogLevelFilterMap logLevelFilterMap = new LogLevelFilterMap();
+
+    TreeMap<String, LogLevelFilter> apiFilter = new TreeMap<>();
+    for (Map.Entry<String, LSServerLogLevelFilter> e : filter.entrySet()) {
+      apiFilter.put(e.getKey(), e.getValue().convertToApi());
+    }
+    logLevelFilterMap.setFilter(apiFilter);
+
+    return logLevelFilterMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogFeederDataMap.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogFeederDataMap.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogFeederDataMap.java
deleted file mode 100644
index cc7d53d..0000000
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogFeederDataMap.java
+++ /dev/null
@@ -1,50 +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.apache.ambari.logsearch.model.common;
-
-import io.swagger.annotations.ApiModel;
-import io.swagger.annotations.ApiModelProperty;
-
-import java.util.TreeMap;
-
-@ApiModel
-public class LogFeederDataMap {
-
-  @ApiModelProperty
-  private String id;
-
-  @ApiModelProperty
-  private TreeMap<String, LogfeederFilterData> filter;
-
-  public TreeMap<String, LogfeederFilterData> getFilter() {
-    return filter;
-  }
-
-  public void setFilter(TreeMap<String, LogfeederFilterData> filter) {
-    this.filter = filter;
-  }
-
-  public String getId() {
-    return id;
-  }
-
-  public void setId(String id) {
-    this.id = id;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogfeederFilterData.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogfeederFilterData.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogfeederFilterData.java
deleted file mode 100644
index e0f8013..0000000
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/model/common/LogfeederFilterData.java
+++ /dev/null
@@ -1,87 +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.apache.ambari.logsearch.model.common;
-
-import io.swagger.annotations.ApiModel;
-import io.swagger.annotations.ApiModelProperty;
-
-import java.util.ArrayList;
-import java.util.List;
-
-@ApiModel
-public class LogfeederFilterData {
-
-  @ApiModelProperty
-  private String label;
-
-  @ApiModelProperty
-  private List<String> hosts = new ArrayList<>();
-
-  @ApiModelProperty
-  private List<String> defaultLevels = new ArrayList<>();
-
-  @ApiModelProperty
-  private List<String> overrideLevels = new ArrayList<>();
-
-  @ApiModelProperty
-  private String expiryTime;
-
-  public LogfeederFilterData() {
-  }
-
-  public String getLabel() {
-    return label;
-  }
-
-  public void setLabel(String label) {
-    this.label = label;
-  }
-
-  public List<String> getHosts() {
-    return hosts;
-  }
-
-  public void setHosts(List<String> hosts) {
-    this.hosts = hosts;
-  }
-
-  public List<String> getDefaultLevels() {
-    return defaultLevels;
-  }
-
-  public void setDefaultLevels(List<String> defaultLevels) {
-    this.defaultLevels = defaultLevels;
-  }
-
-  public List<String> getOverrideLevels() {
-    return overrideLevels;
-  }
-
-  public void setOverrideLevels(List<String> overrideLevels) {
-    this.overrideLevels = overrideLevels;
-  }
-
-  public String getExpiryTime() {
-    return expiryTime;
-  }
-
-  public void setExpiryTime(String expiryTime) {
-    this.expiryTime = expiryTime;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/ShipperConfigResource.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/ShipperConfigResource.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/ShipperConfigResource.java
index 47e6ba2..342d1cf 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/ShipperConfigResource.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/ShipperConfigResource.java
@@ -22,6 +22,7 @@ package org.apache.ambari.logsearch.rest;
 import javax.inject.Inject;
 import javax.inject.Named;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
@@ -30,12 +31,18 @@ import javax.ws.rs.core.Response;
 
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
+
 import org.apache.ambari.logsearch.manager.ShipperConfigManager;
+import org.apache.ambari.logsearch.model.common.LSServerLogLevelFilterMap;
 import org.springframework.context.annotation.Scope;
 
 import java.util.List;
 
-import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.*;
+import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.GET_LOG_LEVEL_FILTER_OD;
+import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.GET_SERVICE_NAMES_OD;
+import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.GET_SHIPPER_CONFIG_OD;
+import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.SET_SHIPPER_CONFIG_OD;
+import static org.apache.ambari.logsearch.doc.DocConstants.ShipperConfigOperationDescriptions.UPDATE_LOG_LEVEL_FILTER_OD;
 
 @Api(value = "shipper", description = "Shipper config operations")
 @Path("shipper")
@@ -49,7 +56,7 @@ public class ShipperConfigResource {
   @GET
   @Path("/input/{clusterName}/services")
   @Produces({"application/json"})
-  @ApiOperation(GET_SERVICE_NAMES)
+  @ApiOperation(GET_SERVICE_NAMES_OD)
   public List<String> getServices(@PathParam("clusterName") String clusterName) {
     return shipperConfigManager.getServices(clusterName);
   }
@@ -57,17 +64,43 @@ public class ShipperConfigResource {
   @GET
   @Path("/input/{clusterName}/services/{serviceName}")
   @Produces({"application/json"})
-  @ApiOperation(GET_SHIPPER_CONFIG)
+  @ApiOperation(GET_SHIPPER_CONFIG_OD)
   public String getShipperConfig(@PathParam("clusterName") String clusterName, @PathParam("serviceName") String serviceName) {
     return shipperConfigManager.getInputConfig(clusterName, serviceName);
   }
 
+  @POST
+  @Path("/input/{clusterName}/services/{serviceName}")
+  @Produces({"application/json"})
+  @ApiOperation(SET_SHIPPER_CONFIG_OD)
+  public Response createShipperConfig(String body, @PathParam("clusterName") String clusterName, @PathParam("serviceName")
+    String serviceName) {
+    return shipperConfigManager.createInputConfig(clusterName, serviceName, body);
+  }
+
   @PUT
   @Path("/input/{clusterName}/services/{serviceName}")
-  @Produces("text/plain")
-  @ApiOperation(SET_SHIPPER_CONFIG)
+  @Produces({"application/json"})
+  @ApiOperation(SET_SHIPPER_CONFIG_OD)
   public Response setShipperConfig(String body, @PathParam("clusterName") String clusterName, @PathParam("serviceName")
     String serviceName) {
     return shipperConfigManager.setInputConfig(clusterName, serviceName, body);
   }
+
+  @GET
+  @Path("/filters/{clusterName}/level")
+  @Produces({"application/json"})
+  @ApiOperation(GET_LOG_LEVEL_FILTER_OD)
+  public LSServerLogLevelFilterMap getLogLevelFilters(@PathParam("clusterName") String clusterName) {
+    return shipperConfigManager.getLogLevelFilters(clusterName);
+  }
+
+  @PUT
+  @Path("/filters/{clusterName}/level")
+  @Produces({"application/json"})
+  @ApiOperation(UPDATE_LOG_LEVEL_FILTER_OD)
+  public Response setogLevelFilter(LSServerLogLevelFilterMap request, @PathParam("clusterName") String clusterName) {
+    return shipperConfigManager.setLogLevelFilters(clusterName, request);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/UserConfigResource.java
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/UserConfigResource.java b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/UserConfigResource.java
index 41dda05..00b971a 100644
--- a/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/UserConfigResource.java
+++ b/ambari-logsearch/ambari-logsearch-server/src/main/java/org/apache/ambari/logsearch/rest/UserConfigResource.java
@@ -25,7 +25,6 @@ import javax.ws.rs.BeanParam;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
@@ -33,7 +32,6 @@ import javax.ws.rs.Produces;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import org.apache.ambari.logsearch.manager.UserConfigManager;
-import org.apache.ambari.logsearch.model.common.LogFeederDataMap;
 import org.apache.ambari.logsearch.model.request.impl.UserConfigRequest;
 import org.apache.ambari.logsearch.model.response.UserConfigData;
 import org.apache.ambari.logsearch.model.response.UserConfigDataListResponse;
@@ -74,22 +72,6 @@ public class UserConfigResource {
   }
 
   @GET
-  @Path("/filters")
-  @Produces({"application/json"})
-  @ApiOperation(GET_USER_FILTER_OD)
-  public LogFeederDataMap getUserFilter() {
-    return userConfigManager.getUserFilter();
-  }
-
-  @PUT
-  @Path("/filters")
-  @Produces({"application/json"})
-  @ApiOperation(UPDATE_USER_FILTER_OD)
-  public LogFeederDataMap updateUserFilter(LogFeederDataMap request) {
-    return userConfigManager.saveUserFiter(request);
-  }
-
-  @GET
   @Path("/names")
   @Produces({"application/json"})
   @ApiOperation(GET_ALL_USER_NAMES_OD)

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-logsearch/ambari-logsearch-web/src/main/webapp/templates/common/Header_tmpl.html
----------------------------------------------------------------------
diff --git a/ambari-logsearch/ambari-logsearch-web/src/main/webapp/templates/common/Header_tmpl.html b/ambari-logsearch/ambari-logsearch-web/src/main/webapp/templates/common/Header_tmpl.html
index 5f1bbdb..24cc392 100644
--- a/ambari-logsearch/ambari-logsearch-web/src/main/webapp/templates/common/Header_tmpl.html
+++ b/ambari-logsearch/ambari-logsearch-web/src/main/webapp/templates/common/Header_tmpl.html
@@ -34,11 +34,12 @@
                         <i class="fa fa-filter  pull-right"></i>
                     </a>
                 </li> -->
-                 <li class="dropdown" data-id="createFilters" title="Logfeeder Filters">
+<!-- TODO: update filters to support multiple clusters
+                <li class="dropdown" data-id="createFilters" title="Logfeeder Filters">
                     <a href="#" class="account excludeStatus" data-toggle="modal">
                         <i class="fa fa-filter"></i>
                     </a>
-                </li>
+                </li>-->
                 <li class="dropdown" title="Menu">
                     <a href="#" class="dropdown-toggle account" data-toggle="dropdown">
                         <!-- <div class="avatar">

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog300.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog300.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog300.java
index dc6037e..44fbd4d 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog300.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog300.java
@@ -23,6 +23,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -52,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.jdbc.support.JdbcUtils;
 
+import com.google.common.collect.Sets;
 import com.google.inject.Inject;
 import com.google.inject.Injector;
 
@@ -333,6 +335,19 @@ public class UpgradeCatalog300 extends AbstractUpgradeCatalog {
 
             removeConfigurationPropertiesFromCluster(cluster, configType, removeProperties);
           }
+          
+          Config logSearchProperties = cluster.getDesiredConfigByType("logsearch-properties");
+          Config logFeederProperties = cluster.getDesiredConfigByType("logfeeder-properties");
+          if (logSearchProperties != null && logFeederProperties != null) {
+            String defaultLogLevels = logSearchProperties.getProperties().get("logsearch.logfeeder.include.default.level");
+            
+            Set<String> removeProperties = Sets.newHashSet("logsearch.logfeeder.include.default.level");
+            removeConfigurationPropertiesFromCluster(cluster, "logsearch-properties", removeProperties);
+            
+            Map<String, String> newProperties = new HashMap<>();
+            newProperties.put("logfeeder.include.default.level", defaultLogLevels);
+            updateConfigurationPropertiesForCluster(cluster, "logfeeder-properties", newProperties, true, true);
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logfeeder-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logfeeder-properties.xml b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logfeeder-properties.xml
index 1ff8ad3..a38f961 100644
--- a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logfeeder-properties.xml
+++ b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logfeeder-properties.xml
@@ -120,4 +120,14 @@
     <display-name>Input cache key field</display-name>
     <on-ambari-upgrade add="true"/>
   </property>
+  <property>
+    <name>logfeeder.include.default.level</name>
+    <value>FATAL,ERROR,WARN</value>
+    <description>Include default Log Feeder Log Levels for Log Search. Used for bootstrapping the configuration only. (levels: FATAL,ERROR,WARN,INFO,DEBUG,TRACE)</description>
+    <display-name>Log Feeder Log Levels</display-name>
+    <value-attributes>
+      <editable-only-at-install>true</editable-only-at-install>
+    </value-attributes>
+    <on-ambari-upgrade add="true"/>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logsearch-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logsearch-properties.xml b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logsearch-properties.xml
index a18c5c5..be586fd 100644
--- a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logsearch-properties.xml
+++ b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/configuration/logsearch-properties.xml
@@ -130,16 +130,6 @@
     <on-ambari-upgrade add="true"/>
   </property>
   <property>
-    <name>logsearch.logfeeder.include.default.level</name>
-    <value>FATAL,ERROR,WARN</value>
-    <description>Include default Log Feeder Log Levels for Log Search. Used for bootstrapping the configuration only. (levels: FATAL,ERROR,WARN,INFO,DEBUG,TRACE)</description>
-    <display-name>Log Feeder Log Levels</display-name>
-    <value-attributes>
-      <editable-only-at-install>true</editable-only-at-install>
-    </value-attributes>
-    <on-ambari-upgrade add="true"/>
-  </property>
-  <property>
     <name>logsearch.solr.metrics.collector.hosts</name>
     <value>{metrics_collector_hosts}</value>
     <description>Metrics collector hosts for pushing metrics by Log Search Solr</description>

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/themes/theme.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/themes/theme.json b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/themes/theme.json
index 0adcbde..d36d89c 100644
--- a/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/themes/theme.json
+++ b/ambari-server/src/main/resources/common-services/LOGSEARCH/0.5.0/themes/theme.json
@@ -87,7 +87,7 @@
       "configuration-layout": "default",
       "configs": [
         {
-          "config" : "logsearch-properties/logsearch.logfeeder.include.default.level",
+          "config" : "logfeeder-properties/logfeeder.include.default.level",
           "subsection-name": "subsection-logsearch-server-col1"
         },
         {
@@ -353,7 +353,7 @@
         }
       },
       {
-        "config": "logsearch-properties/logsearch.logfeeder.include.default.level",
+        "config": "logfeeder-properties/logfeeder.include.default.level",
         "widget": {
           "type": "text-field"
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/3b94d3cf/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog300Test.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog300Test.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog300Test.java
index 8f1510b..c949ca2 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog300Test.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog300Test.java
@@ -302,9 +302,32 @@ public class UpgradeCatalog300Test {
     expect(controller.createConfig(anyObject(Cluster.class), anyString(), capture(logSearchConfCapture), anyString(),
         EasyMock.<Map<String, Map<String, String>>>anyObject())).andReturn(config).times(2);
 
+    Map<String, String> oldLogSearchProperties = ImmutableMap.of(
+        "logsearch.logfeeder.include.default.level", "FATAL,ERROR,WARN"
+    );
+
+    Map<String, String> expectedLogFeederProperties = ImmutableMap.of(
+        "logfeeder.include.default.level", "FATAL,ERROR,WARN"
+    );
+
+    Config logFeederPropertiesConf = easyMockSupport.createNiceMock(Config.class);
+    expect(cluster.getDesiredConfigByType("logfeeder-properties")).andReturn(logFeederPropertiesConf).times(2);
+    expect(logFeederPropertiesConf.getProperties()).andReturn(Collections.<String, String> emptyMap()).once();
+    Capture<Map<String, String>> logFeederPropertiesCapture = EasyMock.newCapture();
+    expect(controller.createConfig(anyObject(Cluster.class), eq("logfeeder-properties"), capture(logFeederPropertiesCapture),
+        anyString(), EasyMock.<Map<String, Map<String, String>>>anyObject())).andReturn(config).once();
+
+    Config logSearchPropertiesConf = easyMockSupport.createNiceMock(Config.class);
+    expect(cluster.getDesiredConfigByType("logsearch-properties")).andReturn(logSearchPropertiesConf).times(2);
+    expect(logSearchPropertiesConf.getProperties()).andReturn(oldLogSearchProperties).times(2);
+    Capture<Map<String, String>> logSearchPropertiesCapture = EasyMock.newCapture();
+    expect(controller.createConfig(anyObject(Cluster.class), eq("logsearch-properties"), capture(logSearchPropertiesCapture),
+        anyString(), EasyMock.<Map<String, Map<String, String>>>anyObject())).andReturn(config).once();
+
     replay(clusters, cluster);
     replay(controller, injector2);
     replay(confSomethingElse1, confSomethingElse2, confLogSearchConf1, confLogSearchConf2);
+    replay(logSearchPropertiesConf, logFeederPropertiesConf);
     new UpgradeCatalog300(injector2).updateLogSearchConfigs();
     easyMockSupport.verifyAll();
 
@@ -313,5 +336,11 @@ public class UpgradeCatalog300Test {
     for (Map<String, String> updatedLogSearchConf : updatedLogSearchConfs) {
       assertTrue(Maps.difference(Collections.<String, String> emptyMap(), updatedLogSearchConf).areEqual());
     }
+    
+    Map<String,String> newLogFeederProperties = logFeederPropertiesCapture.getValue();
+    assertTrue(Maps.difference(expectedLogFeederProperties, newLogFeederProperties).areEqual());
+    
+    Map<String,String> newLogSearchProperties = logSearchPropertiesCapture.getValue();
+    assertTrue(Maps.difference(Collections.<String, String> emptyMap(), newLogSearchProperties).areEqual());
   }
 }