diff --git a/devops/aws/chart/values.yaml b/devops/aws/chart/values.yaml
index 1a880c4f40ddb9203cc0d30abc7bc10de5cfb3f3..b2f670ff10b949de94bcc3c1654e2b5c960ff9c1 100644
--- a/devops/aws/chart/values.yaml
+++ b/devops/aws/chart/values.yaml
@@ -64,6 +64,8 @@ environmentVariables:
     value: "true"
   - name: PARAMETER_MOUNT_PATH
     value: "/mnt/params"
+  - name: TMP_VOLUME_PATH
+    value: "/tmp"
 
 # Resource Config
 replicaCount: 1
@@ -85,7 +87,7 @@ serviceAccountRole: arn:aws:iam::{{ .Values.global.accountID }}:role/osdu-{{ .Va
 securityContext: 
   runAsUser: 10001
   runAsNonRoot: true
-  readOnlyRootFilesystem: false
+  readOnlyRootFilesystem: true
   allowPrivilegeEscalation: false
   capabilities:
     drop:
diff --git a/indexer-core/pom.xml b/indexer-core/pom.xml
index e229e7b11c1ac8b984b9d5b82e2486f363c5df85..3fd6fa05d20e9098b1dc904022513d8cb93ab994 100644
--- a/indexer-core/pom.xml
+++ b/indexer-core/pom.xml
@@ -4,12 +4,12 @@
 	<parent>
 		<groupId>org.opengroup.osdu.indexer</groupId>
 		<artifactId>indexer-service</artifactId>
-		<version>0.20.0-SNAPSHOT</version>
+		<version>0.21.0-SNAPSHOT</version>
 		<relativePath>../pom.xml</relativePath>
 	</parent>
 
 	<artifactId>indexer-core</artifactId>
-	<version>0.20.0-SNAPSHOT</version>
+	<version>0.21.0-SNAPSHOT</version>
 	<name>indexer-core</name>
 	<description>Indexer Service Core</description>
 	<packaging>jar</packaging>
diff --git a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IMappingService.java b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IMappingService.java
index f0009bd048fe18312c9cddefbb3b258fd591756f..00cad2baeaf19615ae764f68d1fa7b83cf803e06 100644
--- a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IMappingService.java
+++ b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IMappingService.java
@@ -19,7 +19,6 @@ import org.opengroup.osdu.core.common.model.indexer.IndexSchema;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.Set;
 
 public interface IMappingService {
 
@@ -31,7 +30,5 @@ public interface IMappingService {
 
     Map<String, Object> getIndexMappingFromRecordSchema(IndexSchema schema);
 
-    void updateIndexMappingForIndicesOfSameType(Set<String> indices, String fieldName) throws Exception;
-
     void syncIndexMappingIfRequired(RestHighLevelClient restClient, IndexSchema schema) throws Exception;
 }
\ No newline at end of file
diff --git a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyService.java b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyService.java
deleted file mode 100644
index 88e20720f510b3f7d0f31c31707b2501ddd1ba5c..0000000000000000000000000000000000000000
--- a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyService.java
+++ /dev/null
@@ -1,27 +0,0 @@
-// Copyright 2017-2019, Schlumberger
-//
-// Licensed 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.opengroup.osdu.indexer.service;
-
-
-import org.opengroup.osdu.core.common.model.http.AppException;
-
-import java.io.IOException;
-
-public interface IndexCopyService {
-
-    String fetchTaskStatus(String taskId) throws AppException;
-
-    String copyIndex(String kind) throws IOException;
-}
diff --git a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImpl.java b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImpl.java
deleted file mode 100644
index 23027e20654fcaae75e8aafd58cf51972565c4e7..0000000000000000000000000000000000000000
--- a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImpl.java
+++ /dev/null
@@ -1,215 +0,0 @@
-// Copyright 2017-2019, Schlumberger
-//
-// Licensed 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.opengroup.osdu.indexer.service;
-
-import com.google.common.collect.Lists;
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
-import org.apache.http.HttpEntity;
-import org.apache.http.HttpStatus;
-import org.apache.http.entity.ContentType;
-import org.apache.http.nio.entity.NStringEntity;
-import org.apache.http.util.EntityUtils;
-import org.elasticsearch.client.Request;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.RestHighLevelClient;
-import org.opengroup.osdu.core.common.model.search.ClusterSettings;
-import org.opengroup.osdu.core.common.model.http.DpsHeaders;
-import org.opengroup.osdu.core.common.model.tenant.TenantInfo;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.core.common.search.ElasticIndexNameResolver;
-import org.opengroup.osdu.core.common.search.Preconditions;
-import org.opengroup.osdu.indexer.logging.AuditLogger;
-import org.opengroup.osdu.core.common.model.indexer.IElasticSettingService;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
-import org.springframework.stereotype.Service;
-
-import javax.inject.Inject;
-import java.io.IOException;
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.Map;
-
-@Service
-public class IndexCopyServiceImpl implements IndexCopyService {
-
-    @Inject
-    private ElasticClientHandler elasticClientHandler;
-    @Inject
-    private ElasticIndexNameResolver elasticIndexNameResolver;
-    @Inject
-    private IndicesService indicesService;
-    @Inject
-    private IElasticSettingService elasticSettingService;
-    @Inject
-    private IMappingService mappingService;
-    @Inject
-    private DpsHeaders headersInfo;
-    @Inject
-    private AuditLogger auditLogger;
-
-    @Override
-    public String fetchTaskStatus(String taskId) {
-        try (RestHighLevelClient restClient = this.elasticClientHandler.createRestClient()) {
-            Request request = new Request("GET", String.format("/_tasks/%s", taskId));
-            Response response = restClient.getLowLevelClient().performRequest(request);
-            this.auditLogger.getTaskStatus(Lists.newArrayList(taskId));
-            return EntityUtils.toString(response.getEntity());
-        } catch (IOException e) {
-            throw new AppException(HttpStatus.SC_INTERNAL_SERVER_ERROR, "Unknown error", e.getMessage(), e);
-        }
-    }
-
-    /**
-     * This method is used to copy one index from common cluster to tenant cluster.
-     *
-     * @param kind request sent to ask the copying operation, includes kind and tenant
-     * @throws IOException if upstream server cannot process the request
-     */
-    @Override
-    public String copyIndex(String kind) throws IOException {
-        Preconditions.checkNotNull(kind, "kind can't be null");
-
-        String originalAccountId = this.headersInfo.getPartitionId();
-        String toBeCopiedIndex = this.elasticIndexNameResolver.getIndexNameFromKind(kind);
-
-        String typeOfToBeCopiedIndex = kind.split(":")[2];
-        if (typeOfToBeCopiedIndex == null) {
-            throw new AppException(HttpStatus.SC_NOT_FOUND, "Fail to find the type of the index", "Fail to find the type of the given index in common cluster.");
-        }
-
-        Map<String, Object> mappingsMap = this.getIndexMappingsFromCommonCluster(toBeCopiedIndex, typeOfToBeCopiedIndex);
-        String[] commonCluster = this.getCommonClusterInformation();
-
-        this.createIndexInTenantCluster(originalAccountId, toBeCopiedIndex, typeOfToBeCopiedIndex, mappingsMap);
-        String taskStatus = this.reindexInTenantCluster(originalAccountId, toBeCopiedIndex, commonCluster);
-        this.auditLogger.copyIndex(Lists.newArrayList(String.format("Kind:%s", kind), String.format("Task status: %s", taskStatus)));
-        return taskStatus;
-    }
-
-    /**
-     * This method is used to format the body of remote reindex request.
-     *
-     * @param host     host of the remote cluster
-     * @param username username of the remote cluster
-     * @param password password of the remote cluster
-     * @param index    the index to be copied
-     * @return request body in json string format
-     */
-    private String formatReindexRequestBody(String host, String username, String password, String index) {
-        Map<String, String> remoteMap = new HashMap<>();
-        remoteMap.put("host", host);
-        remoteMap.put("username", username);
-        remoteMap.put("password", password);
-
-        Map<String, Object> sourceMap = new HashMap<>();
-        sourceMap.put("index", index);
-        sourceMap.put("remote", remoteMap);
-        sourceMap.put("size", 10000);
-
-        Map<String, String> destMap = new HashMap<>();
-        destMap.put("index", index);
-
-        Map<String, Object> map = new HashMap<>();
-        map.put("source", sourceMap);
-        map.put("dest", destMap);
-
-        Gson gson = new Gson();
-        return gson.toJson(map);
-    }
-
-    /**
-     * This method is used to extract cluster information from cluster settings.
-     *
-     * @param setting setting of the cluster, including host, port, username and password
-     * @return the host which combined the hostname, scheme and port together, username, and password
-     */
-    private String[] extractInfoFromClusterSetting(ClusterSettings setting) {
-
-        String[] clusterInfo = new String[3];
-
-        StringBuilder host = new StringBuilder("https://");
-        host.append(setting.getHost())
-                .append(":")
-                .append(setting.getPort());
-        clusterInfo[0] = host.toString();
-
-        String userAndPwd = setting.getUserNameAndPassword();
-        int indexOfColon = userAndPwd.indexOf(':');
-        clusterInfo[1] = userAndPwd.substring(0, indexOfColon);
-        clusterInfo[2] = userAndPwd.substring(indexOfColon + 1);
-
-        return clusterInfo;
-    }
-
-    /**
-     * This method is used to cast mapping to fit the parameter requirement.
-     *
-     * @param mapping input map in json string format
-     * @return output map in Map<String, Object> format
-     */
-    private Map<String, Object> castMappingsMap(String mapping, String indexType, String index) {
-        Type type = new TypeToken<Map<String, Map<String, Map<String, Map<String, Object>>>>>() {}.getType();
-        Map<String, Map<String, Map<String, Map<String, Object>>>> indexMap = new Gson().fromJson(mapping, type);
-        Map<String, Map<String, Map<String, Object>>> mappingMap = indexMap.get(index);
-        Map<String, Map<String, Object>> mappingWithoutMappings = mappingMap.get("mappings");
-
-        return mappingWithoutMappings.get(indexType);
-    }
-
-    public Map<String, Object> getIndexMappingsFromCommonCluster(String toBeCopiedIndex, String typeOfToBeCopiedIndex) throws IOException {
-        this.headersInfo.getHeaders().put(DpsHeaders.ACCOUNT_ID, TenantInfo.COMMON);
-        this.headersInfo.getHeaders().put(DpsHeaders.DATA_PARTITION_ID, TenantInfo.COMMON);
-        try (RestHighLevelClient commonClient = this.elasticClientHandler.createRestClient()) {
-            String indexMapping = this.mappingService.getIndexMapping(commonClient, toBeCopiedIndex);
-            return castMappingsMap(indexMapping, typeOfToBeCopiedIndex, toBeCopiedIndex);
-        } catch (Exception e) {
-            throw new IOException("Fail to get mapping for the given index from common cluster.");
-        }
-    }
-
-    public String[] getCommonClusterInformation() throws IOException {
-        this.headersInfo.getHeaders().put(DpsHeaders.ACCOUNT_ID, TenantInfo.COMMON);
-        this.headersInfo.getHeaders().put(DpsHeaders.DATA_PARTITION_ID, TenantInfo.COMMON);
-        String[] commonCluster = extractInfoFromClusterSetting(this.elasticSettingService.getElasticClusterInformation());
-        if (commonCluster.length != 3) {
-            throw new IOException("fail to get the information of common cluster.");
-        }
-        return commonCluster;
-    }
-
-    public void createIndexInTenantCluster(String originalAccountId, String toBeCopiedIndex, String typeOfToBeCopiedIndex, Map<String, Object> mappingsMap) throws IOException {
-        this.headersInfo.getHeaders().put(DpsHeaders.ACCOUNT_ID, originalAccountId);
-        this.headersInfo.getHeaders().put(DpsHeaders.DATA_PARTITION_ID, originalAccountId);
-        try (RestHighLevelClient tenantClient = this.elasticClientHandler.createRestClient()) {
-            if (!this.indicesService.createIndex(tenantClient, toBeCopiedIndex, null, typeOfToBeCopiedIndex, mappingsMap)) {
-                throw new AppException(HttpStatus.SC_NOT_FOUND, "Fail to create new index", "Fail to create new corresponding new index in tenant cluster.");
-            }
-        }
-    }
-
-    public String reindexInTenantCluster(String originalAccountId, String toBeCopiedIndex, String[] commonCluster) throws IOException {
-        this.headersInfo.getHeaders().put(DpsHeaders.ACCOUNT_ID, originalAccountId);
-        this.headersInfo.getHeaders().put(DpsHeaders.DATA_PARTITION_ID, originalAccountId);
-        try (RestHighLevelClient tenantClient = this.elasticClientHandler.createRestClient()) {
-            String json = formatReindexRequestBody(commonCluster[0], commonCluster[1], commonCluster[2], toBeCopiedIndex);
-            HttpEntity requestBody = new NStringEntity(json, ContentType.APPLICATION_JSON);
-            Request request = new Request("POST", "/_reindex?wait_for_completion=false");
-            request.setEntity(requestBody);
-            Response response = tenantClient.getLowLevelClient().performRequest(request);
-            return EntityUtils.toString(response.getEntity());
-        }
-    }
-}
diff --git a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceImpl.java b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceImpl.java
index c77f4af7802cf34a347a306cda283d5e97eba15f..a18cbd3fb4db5f5384e1ffb3585d08b436eb0f8e 100644
--- a/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceImpl.java
+++ b/indexer-core/src/main/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceImpl.java
@@ -17,20 +17,15 @@ package org.opengroup.osdu.indexer.service;
 import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
 import com.lambdaworks.redis.RedisException;
-import org.apache.http.HttpStatus;
 import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
 import org.elasticsearch.client.Request;
 import org.elasticsearch.client.RequestOptions;
 import org.elasticsearch.client.Response;
 import org.elasticsearch.client.RestHighLevelClient;
-import org.elasticsearch.client.indices.GetFieldMappingsRequest;
-import org.elasticsearch.client.indices.GetFieldMappingsResponse;
 import org.elasticsearch.client.indices.PutMappingRequest;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.XContentType;
-import org.elasticsearch.index.reindex.BulkByScrollResponse;
-import org.elasticsearch.index.reindex.UpdateByQueryRequest;
 import org.opengroup.osdu.core.common.Constants;
 import org.opengroup.osdu.core.common.logging.JaxRsDpsLog;
 import org.opengroup.osdu.core.common.model.http.AppException;
@@ -40,7 +35,6 @@ import org.opengroup.osdu.core.common.search.ElasticIndexNameResolver;
 import org.opengroup.osdu.core.common.search.Preconditions;
 import org.opengroup.osdu.indexer.cache.PartitionSafeIndexCache;
 import org.opengroup.osdu.indexer.model.Kind;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
 import org.opengroup.osdu.indexer.util.TypeMapper;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
@@ -48,7 +42,10 @@ import org.springframework.stereotype.Service;
 import javax.inject.Inject;
 import java.io.IOException;
 import java.lang.reflect.Type;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 @Service
 public class IndexerMappingServiceImpl extends MappingServiceImpl implements IMappingService {
@@ -57,15 +54,12 @@ public class IndexerMappingServiceImpl extends MappingServiceImpl implements IMa
 
     @Inject
     private JaxRsDpsLog log;
-    @Inject
-    private ElasticClientHandler elasticClientHandler;
     @Autowired
     private PartitionSafeIndexCache indexCache;
     @Autowired
     private ElasticIndexNameResolver elasticIndexNameResolver;
 
 
-
     /**
      * Create a new type in Elasticsearch
      *
@@ -164,15 +158,6 @@ public class IndexerMappingServiceImpl extends MappingServiceImpl implements IMa
         return dataMapping;
     }
 
-    @Override
-    public void updateIndexMappingForIndicesOfSameType(Set<String> indices, String fieldName) throws Exception {
-        try (RestHighLevelClient restClient = this.elasticClientHandler.createRestClient()) {
-            if(!updateMappingToEnableKeywordIndexingForField(restClient,indices,fieldName)){
-                throw new AppException(HttpStatus.SC_INTERNAL_SERVER_ERROR, "Elastic error", "Error updating index mapping.", String.format("Failed to get confirmation from elastic server mapping update for indices: %s", indices));
-            }
-        }
-    }
-
     @Override
     public void syncIndexMappingIfRequired(RestHighLevelClient restClient, IndexSchema schema) throws Exception {
         String index = this.elasticIndexNameResolver.getIndexNameFromKind(schema.getKind());
@@ -187,7 +172,8 @@ public class IndexerMappingServiceImpl extends MappingServiceImpl implements IMa
         }
 
         String jsonResponse = this.getIndexMapping(restClient, index);
-        Type type = new TypeToken<Map<String, Object>>() {}.getType();
+        Type type = new TypeToken<Map<String, Object>>() {
+        }.getType();
         Map<String, Object> mappings = new Gson().fromJson(jsonResponse, type);
 
         if (mappings == null || mappings.isEmpty()) return;
@@ -228,108 +214,6 @@ public class IndexerMappingServiceImpl extends MappingServiceImpl implements IMa
         this.indexCache.put(cacheKey, true);
     }
 
-    private boolean updateMappingToEnableKeywordIndexingForField(RestHighLevelClient client, Set<String> indicesSet, String fieldName) throws IOException {
-        String[] indices = indicesSet.toArray(new String[indicesSet.size()]);
-        Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> indexMappingMap = getIndexFieldMap(new String[]{"data."+fieldName}, client, indices);
-        boolean failure = false;
-        for (String index : indicesSet) {
-            if (indexMappingMap.get(index)!=null && updateMappingForAllIndicesOfSameTypeToEnableKeywordIndexingForField(client, index, indexMappingMap.get(index), fieldName)) {
-                log.info(String.format("Updated field: %s | index:  %s", fieldName, index));
-            } else {
-                failure=true;
-                log.warning(String.format("Failed to update field: %s | index  %s", fieldName, index));
-            }
-        }
-        return !failure;
-    }
-
-    private Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> getIndexFieldMap(String[] fieldNames, RestHighLevelClient client, String[] indices) throws IOException  {
-        Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> indexMappingMap = new HashMap<>();
-        GetFieldMappingsRequest request = new GetFieldMappingsRequest();
-        request.indices(indices);
-        request.fields(fieldNames);
-        try {
-            GetFieldMappingsResponse response = client.indices().getFieldMapping(request, RequestOptions.DEFAULT);
-            if (response != null && !response.mappings().isEmpty()) {
-                final Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappings = response.mappings();
-                for (String index : indices) {
-                    if (mappings != null && !mappings.isEmpty()) {
-                        indexMappingMap.put(index, mappings);
-                    }
-                }
-            }
-
-            return indexMappingMap;
-        } catch (ElasticsearchException e) {
-            log.error(String.format("Failed to get indices: %s. | Error: %s", Arrays.toString(indices), e));
-            throw new AppException(HttpStatus.SC_INTERNAL_SERVER_ERROR, "Elastic error", "Error getting indices.", String.format("Failed to get indices error: %s", Arrays.toString(indices)));
-        }
-    }
-
-    private boolean updateMappingForAllIndicesOfSameTypeToEnableKeywordIndexingForField(
-            RestHighLevelClient client, String index, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> indexMapping, String fieldName) throws IOException {
-
-        PutMappingRequest request = new PutMappingRequest(index);
-        String type = indexMapping.keySet().iterator().next();
-        if(type.isEmpty()) {
-        	log.error(String.format("Could not find type of the mappings for index: %s.", index));
-            return false;
-        }
-
-        request.setTimeout(REQUEST_TIMEOUT);
-        Map<String, GetFieldMappingsResponse.FieldMappingMetadata> metaData = indexMapping.get(type);
-        if(metaData==null || metaData.get("data." + fieldName)==null) {
-            log.error(String.format("Could not find field: %s in the mapping of index: %s.", fieldName, index));
-            return false;
-        }
-
-        GetFieldMappingsResponse.FieldMappingMetadata fieldMetaData = metaData.get("data." + fieldName);
-        Map<String, Object> source = fieldMetaData.sourceAsMap();
-        if(!source.containsKey(fieldName)){
-            log.error(String.format("Could not find field: %s in the mapping of index: %s.", fieldName, index));
-            return false;
-        }
-
-        //Index the field with additional keyword type
-        Map<String, Object> keywordMap = new HashMap<>();
-        keywordMap.put(Constants.TYPE, "keyword");
-        Map<String, Object> fieldIndexTypeMap = new HashMap<>();
-        fieldIndexTypeMap.put("keyword", keywordMap);
-        Map<String, Object> dataFieldMap = (Map<String, Object>) source.get(fieldName);
-        dataFieldMap.put("fields", fieldIndexTypeMap);
-        Map<String, Object> dataProperties = new HashMap<>();
-        dataProperties.put(fieldName, dataFieldMap);
-        Map<String, Object> mapping = new HashMap<>();
-        mapping.put(Constants.PROPERTIES, dataProperties);
-        Map<String, Object> data = new HashMap<>();
-        data.put(Constants.DATA,mapping);
-        Map<String, Object> properties = new HashMap<>();
-        properties.put(Constants.PROPERTIES, data);
-
-        request.source(new Gson().toJson(properties), XContentType.JSON);
-
-        try {
-            AcknowledgedResponse response = client.indices().putMapping(request, RequestOptions.DEFAULT);
-            boolean isIndicesUpdated = updateIndices(client, index);
-            return response.isAcknowledged() && isIndicesUpdated;
-
-        } catch (Exception e) {
-            log.error(String.format("Could not update mapping of index: %s. | Error: %s", index, e));
-            return false;
-        }
-    }
-
-    private boolean updateIndices(RestHighLevelClient client, String index) throws IOException {
-        UpdateByQueryRequest request = new UpdateByQueryRequest(index);
-        request.setConflicts("proceed");
-        BulkByScrollResponse response = client.updateByQuery(request, RequestOptions.DEFAULT);
-        if(!response.getBulkFailures().isEmpty()) {
-        	log.error(String.format("Could not update index: %s.",index));
-        	return false;
-        }
-		return true;
-    }
-
     /**
      * Create a new type in Elasticsearch
      *
diff --git a/pom.xml b/pom.xml
index 53d258a33b3da93c4468665721bfe74c7b5f1fa8..317711bed092e226c1f04270335d0dbf2d71c396 100644
--- a/pom.xml
+++ b/pom.xml
@@ -5,7 +5,7 @@
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-service</artifactId>
     <packaging>pom</packaging>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <description>Indexer Service</description>
 
     <properties>
diff --git a/provider/indexer-aws/build-aws/Dockerfile b/provider/indexer-aws/build-aws/Dockerfile
index 615dde332e1de01c951a32c0b17677bfe72ea7d2..2885b3f2ed9c1ac30a89b0c99d351c57b5fdedbf 100644
--- a/provider/indexer-aws/build-aws/Dockerfile
+++ b/provider/indexer-aws/build-aws/Dockerfile
@@ -17,12 +17,9 @@ FROM amazoncorretto:8
 
 ARG JAR_FILE=provider/indexer-aws/target/*spring-boot.jar
 
-#Default to using self signed generated TLS cert
-ENV USE_SELF_SIGNED_SSL_CERT true
 
 WORKDIR /
 COPY ${JAR_FILE} app.jar
-COPY /provider/indexer-aws/build-aws/ssl.sh /ssl.sh
 COPY /provider/indexer-aws/build-aws/entrypoint.sh /entrypoint.sh
 EXPOSE 8080
 
diff --git a/provider/indexer-aws/build-aws/entrypoint.sh b/provider/indexer-aws/build-aws/entrypoint.sh
index 9bd3ec69d01fba69f4bece2162e7faba5cc0f0cb..412f71afd92e9c2ed3e50a2863cb5675c8a69b43 100755
--- a/provider/indexer-aws/build-aws/entrypoint.sh
+++ b/provider/indexer-aws/build-aws/entrypoint.sh
@@ -1,15 +1,4 @@
 
 
-if [ -n $USE_SELF_SIGNED_SSL_CERT ];
-then    
-    export SSL_KEY_PASSWORD=$RANDOM$RANDOM$RANDOM;
-    export SSL_KEY_STORE_PASSWORD=$SSL_KEY_PASSWORD;
-    export SSL_KEY_STORE_DIR=/tmp/certs;
-    export SSL_KEY_STORE_NAME=osduonaws.p12;
-    export SSL_KEY_STORE_PATH=$SSL_KEY_STORE_DIR/$SSL_KEY_STORE_NAME;
-    export SSL_KEY_ALIAS=osduonaws;
-    
-    ./ssl.sh;
-fi
 
 java $JAVA_OPTS -jar /app.jar
\ No newline at end of file
diff --git a/provider/indexer-aws/build-aws/ssl.sh b/provider/indexer-aws/build-aws/ssl.sh
deleted file mode 100755
index 9ede565684bdd46cb09e56fce721ced55206ca07..0000000000000000000000000000000000000000
--- a/provider/indexer-aws/build-aws/ssl.sh
+++ /dev/null
@@ -1,34 +0,0 @@
-# Copyright © 2021 Amazon Web Services
-#
-# Licensed 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.
-
-#!/usr/bin/env bash
-
-#Future: Support for using Amazon Cert Manager
-# if [ "$1" == "webserver" ] && [ -n $ACM_CERTIFICATE_ARN ];
-# then
-
-#   aws acm export-certificate --certificate-arn $ACM_CERTIFICATE_ARN --passphrase $(echo -n 'aws123' | openssl base64 -e) | jq -r '"\(.PrivateKey)"' > ${SSL_KEY_PATH}.enc
-#   openssl rsa -in ${SSL_KEY_PATH}.enc -out $SSL_KEY_PATH -passin pass:aws123
-#   aws acm get-certificate --certificate-arn $ACM_CERTIFICATE_ARN | jq -r '"\(.CertificateChain)"' > $SSL_CERT_PATH
-#   aws acm get-certificate --certificate-arn $ACM_CERTIFICATE_ARN | jq -r '"\(.Certificate)"' >> $SSL_CERT_PATH
-
-# fi
-
-if [ -n $USE_SELF_SIGNED_SSL_CERT ];
-then
-    mkdir -p $SSL_KEY_STORE_DIR
-    pushd $SSL_KEY_STORE_DIR
-    keytool -genkeypair -alias $SSL_KEY_ALIAS -keyalg RSA -keysize 2048 -storetype PKCS12 -keystore $SSL_KEY_STORE_NAME -validity 3650 -keypass $SSL_KEY_PASSWORD -storepass $SSL_KEY_PASSWORD -dname "CN=localhost, OU=AWS, O=Energy, L=Houston, ST=TX, C=US"
-    popd
-fi
diff --git a/provider/indexer-aws/pom.xml b/provider/indexer-aws/pom.xml
index 63a5c3a8817f4026cbf62c21130ac8dc85756704..bccac739c9de3f8cb2d153d8f309af319253ba91 100644
--- a/provider/indexer-aws/pom.xml
+++ b/provider/indexer-aws/pom.xml
@@ -18,7 +18,7 @@
   <parent>
       <groupId>org.opengroup.osdu.indexer</groupId>
       <artifactId>indexer-service</artifactId>
-      <version>0.20.0-SNAPSHOT</version>
+      <version>0.21.0-SNAPSHOT</version>
       <relativePath>../../pom.xml</relativePath>
   </parent>
 
@@ -26,7 +26,7 @@
   <artifactId>indexer-aws</artifactId>
   <description>Indexer service on AWS</description>
   <packaging>jar</packaging>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
 
   <properties>
       <aws.version>1.11.1018</aws.version>
@@ -66,12 +66,12 @@
     <dependency>
         <groupId>org.opengroup.osdu.indexer</groupId>
         <artifactId>indexer-core</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
     </dependency>
     <dependency>
         <groupId>org.opengroup.osdu.core.aws</groupId>
         <artifactId>os-core-lib-aws</artifactId>
-        <version>0.19.0-rc3</version>
+        <version>0.21.0-rc1</version>
     </dependency>
 
     <!-- AWS managed packages -->
diff --git a/provider/indexer-azure/pom.xml b/provider/indexer-azure/pom.xml
index 3bd861b7b2895955b711bb2ea809934427ecf82f..ec47402e269a35dd1599831de2fcc3e79e7a605a 100644
--- a/provider/indexer-azure/pom.xml
+++ b/provider/indexer-azure/pom.xml
@@ -21,12 +21,12 @@
     <parent>
         <groupId>org.opengroup.osdu.indexer</groupId>
         <artifactId>indexer-service</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
     <artifactId>indexer-azure</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <name>indexer-azure</name>
     <description>Indexer Service Azure</description>
     <packaging>jar</packaging>
@@ -39,7 +39,7 @@
         <azure.appservice.subscription />
         <log4j.version>2.17.1</log4j.version>
         <nimbus-jose-jwt.version>8.2</nimbus-jose-jwt.version>
-        <indexer-core.version>0.20.0-SNAPSHOT</indexer-core.version>
+        <indexer-core.version>0.21.0-SNAPSHOT</indexer-core.version>
         <spring-security-jwt.version>1.1.1.RELEASE</spring-security-jwt.version>
         <osdu.corelibazure.version>0.20.0-rc5</osdu.corelibazure.version>
         <os-core-common.version>0.19.0-rc6</os-core-common.version>
diff --git a/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexCopyServiceImplTest.java b/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexCopyServiceImplTest.java
deleted file mode 100644
index 6fa376a3f6d5a14fe151f2c881202809c220f700..0000000000000000000000000000000000000000
--- a/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexCopyServiceImplTest.java
+++ /dev/null
@@ -1,191 +0,0 @@
-// Copyright 2017-2019, Schlumberger
-//
-// Licensed 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.opengroup.osdu.indexer.azure.service;
-
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
-import org.apache.http.HttpEntity;
-import org.apache.http.util.EntityUtils;
-import org.elasticsearch.client.Request;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.RestClient;
-import org.elasticsearch.client.RestHighLevelClient;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.ArgumentMatchers;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.opengroup.osdu.core.common.model.http.DpsHeaders;
-import org.opengroup.osdu.indexer.logging.AuditLogger;
-import org.opengroup.osdu.indexer.service.IMappingService;
-import org.opengroup.osdu.indexer.service.IndexCopyServiceImpl;
-import org.opengroup.osdu.core.common.model.search.ClusterSettings;
-import org.opengroup.osdu.core.common.provider.interfaces.IRequestInfo;
-import org.opengroup.osdu.core.common.model.indexer.IElasticSettingService;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.service.IndicesService;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
-import org.opengroup.osdu.core.common.search.ElasticIndexNameResolver;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.springframework.test.context.junit4.SpringRunner;
-
-import java.io.IOException;
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-@RunWith(SpringRunner.class)
-@PrepareForTest({RestHighLevelClient.class, Response.class, RestClient.class, HttpEntity.class, EntityUtils.class})
-public class IndexCopyServiceImplTest {
-    private final String correlationId = UUID.randomUUID().toString();
-
-    @Mock
-    private HttpEntity httpEntity;
-    @Mock
-    private HttpEntity httpEntityRequest;
-    @Mock
-    private IRequestInfo requestInfo;
-    @Mock
-    private DpsHeaders dpsHeaders;
-    @Mock
-    private RestClient restClient;
-    @Mock
-    private RestHighLevelClient restHighLevelClient;
-    @Mock
-    private IndicesService indicesService;
-    @Mock
-    private IMappingService mappingService;
-    @Mock
-    private ElasticClientHandler elasticClientHandler;
-    @Mock
-    private ElasticIndexNameResolver elasticIndexNameResolver;
-    @Mock
-    private Response response;
-    @Mock
-    private IElasticSettingService elasticSettingService;
-    @Mock
-    private AuditLogger auditLogger;
-    @Mock
-    private Map<String, String> httpHeaders;
-    @InjectMocks
-    private IndexCopyServiceImpl sut;
-
-    private ClusterSettings commonCluster;
-
-    private Map<String, Object> correctMap;
-
-    @Before
-    public void setup() {
-
-        commonCluster = ClusterSettings.builder().host("commonhost").port(8080).userNameAndPassword("username:pwd").build();
-
-        httpHeaders = new HashMap<>();
-        httpHeaders.put(DpsHeaders.AUTHORIZATION, "testAuth");
-        httpHeaders.put(DpsHeaders.CORRELATION_ID, correlationId);
-        when(requestInfo.getHeadersMapWithDwdAuthZ()).thenReturn(httpHeaders);
-        when(response.getEntity()).thenReturn(httpEntity);
-
-        Type mapType = new TypeToken<Map<String, Object>>() {}.getType();
-        String afterFormat = "{\"properties\":{\"id\":{\"type\":\"keyword\"}}}";
-        correctMap = new Gson().fromJson(afterFormat, mapType);
-
-        restHighLevelClient = mock(RestHighLevelClient.class);
-
-    }
-
-    @Test(expected = IOException.class)
-    public void should_throwIOException_when_indexMappingNotFound() throws Exception {
-        IOException exception = new IOException("Fail to get mapping for the given index from common cluster.");
-
-        when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenThrow(exception);
-
-        this.sut.copyIndex("common:metadata:entity:1.0.0");
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void should_throwIllegalArgExceptionCopyIndexRequest_copyIndexTest() {
-        try {
-            this.sut.copyIndex(null);
-        } catch (IOException e) {
-            fail("Should not throw IOException but illegalArgumentException.");
-        }
-    }
-
-    @Test
-    public void should_returnIndexMapping_getIndexMappingFromCommonClustertest() {
-        String mappingJson = "{\"common-metadata-entity-1.0.0\":{\"mappings\":{\"entity\":{\"properties\":{\"id\":{\"type\":\"keyword\"}}}}}}";
-        when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-        try {
-            when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(mappingJson);
-            Map<String, Object> resultMap = this.sut.getIndexMappingsFromCommonCluster("test", "test");
-            Assert.assertEquals(resultMap, correctMap);
-        } catch (Exception ignored) {
-        }
-    }
-
-    @Test
-    public void should_returnClusterInfo_getCommonClusterInformationtest() {
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-            when(elasticSettingService.getElasticClusterInformation()).thenReturn(commonCluster);
-
-            String[] resultCommonCluster = this.sut.getCommonClusterInformation();
-            Assert.assertEquals(correctCommonCluster[0], resultCommonCluster[0]);
-            Assert.assertEquals(correctCommonCluster[1], resultCommonCluster[1]);
-            Assert.assertEquals(correctCommonCluster[2], resultCommonCluster[2]);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Test(expected = AppException.class)
-    public void should_throwException_failToCreateIndexInTenantCluster_createIndexInTenantClustertest() {
-        try {
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            this.sut.createIndexInTenantCluster("test", "test", "test", correctMap);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Ignore
-    public void should_returnTaskIdResponse_reindexRequestSucceed_reindexInTenantClustertest() {
-        //TODO: fix the null Response from restHighLevelClient.getLowLevelClient().performRequest().
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-            Request request = new Request("POST", "/_reindex?wait_for_completion=false");
-            request.setEntity(httpEntityRequest);
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            when(restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-            when(restClient.performRequest(request)).thenReturn(response);
-            when(response.getEntity()).thenReturn(httpEntity);
-            Assert.assertEquals(httpEntity, this.sut.reindexInTenantCluster("test", "test", correctCommonCluster));
-        } catch (IOException ignored) {
-        }
-    }
-}
diff --git a/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexerMappingServiceTest.java b/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexerMappingServiceTest.java
index 01b50be0be12488791a03f54b8c46dce65c4bbd6..faf5b53af92eb0b65bf15667c577307a09964c3e 100644
--- a/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexerMappingServiceTest.java
+++ b/provider/indexer-azure/src/test/java/org/opengroup/osdu/indexer/azure/service/IndexerMappingServiceTest.java
@@ -15,17 +15,13 @@
 package org.opengroup.osdu.indexer.azure.service;
 
 import org.apache.http.StatusLine;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
-import org.elasticsearch.client.*;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.index.reindex.BulkByScrollResponse;
+import org.elasticsearch.client.IndicesClient;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestHighLevelClient;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -34,284 +30,105 @@ import org.mockito.ArgumentMatchers;
 import org.mockito.InjectMocks;
 import org.mockito.Mock;
 import org.opengroup.osdu.core.common.model.indexer.IndexSchema;
-import org.opengroup.osdu.indexer.service.IndexerMappingServiceImpl;
 import org.opengroup.osdu.core.common.model.search.RecordMetaAttribute;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
+import org.opengroup.osdu.indexer.service.IndexerMappingServiceImpl;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.springframework.test.context.junit4.SpringRunner;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
 import static org.powermock.api.mockito.PowerMockito.when;
 
 @Ignore
 @RunWith(SpringRunner.class)
-@PrepareForTest({ RestHighLevelClient.class, IndicesClient.class })
+@PrepareForTest({RestHighLevelClient.class, IndicesClient.class})
 public class IndexerMappingServiceTest {
 
-	private final String kind = "tenant:test:test:1.0.0";
-	private final String index = "tenant-test-test-1.0.0";
-	private final String type = "test";
-	private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
-
-	@Mock
-	private RestClient restClient;
-	@Mock
-	private Response response;
-	@Mock
-	private StatusLine statusLine;
-
-	@InjectMocks
-	private IndexerMappingServiceImpl sut;
-
-	@Mock
-	private ElasticClientHandler elasticClientHandler;
-
-	@InjectMocks
-	private RestHighLevelClient restHighLevelClient;
-
-	@InjectMocks
-	private IndexSchema indexSchema;
-	@InjectMocks
-	private IndicesClient indicesClient;
-
-	@InjectMocks
-	private AcknowledgedResponse mappingResponse;
-
-	@Before
-	public void setup() throws IOException {
-		Map<String, Object> dataMapping = new HashMap<>();
-		dataMapping.put("Location", "geo_point");
-		Map<String, Object> metaMapping = new HashMap<>();
-		metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
-		this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
-				.build();
-
-		this.indicesClient = PowerMockito.mock(IndicesClient.class);
-		this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
-
-		when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-		when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
-		when(this.response.getStatusLine()).thenReturn(statusLine);
-		when(this.statusLine.getStatusCode()).thenReturn(200);
-	}
-
-	@Test
-	public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
-		try {
-			String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
-			assertEquals(mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenExistType_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-
-			IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
-			doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_update_indices_field_with_keyword_when_valid_indices() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType( indices,"any field");
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_someIndex_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("invalid 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_type_of_index_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field invalid");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_elastic_search_failedToFetch_andWeIndexfield_with_keyword() throws Exception {
-		try {
-
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenThrow(new ElasticsearchException(""));
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_when_elastic_failedToIndex_indices_field_with_keyword() {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(this.indicesClient.putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class))).thenThrow(new ElasticsearchException(""));
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
+    private final String kind = "tenant:test:test:1.0.0";
+    private final String index = "tenant-test-test-1.0.0";
+    private final String type = "test";
+    private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
+
+    @Mock
+    private RestClient restClient;
+    @Mock
+    private Response response;
+    @Mock
+    private StatusLine statusLine;
+
+    @InjectMocks
+    private IndexerMappingServiceImpl sut;
+
+    @InjectMocks
+    private RestHighLevelClient restHighLevelClient;
+
+    @InjectMocks
+    private IndexSchema indexSchema;
+    @InjectMocks
+    private IndicesClient indicesClient;
+
+    @InjectMocks
+    private AcknowledgedResponse mappingResponse;
+
+    @Before
+    public void setup() throws IOException {
+        Map<String, Object> dataMapping = new HashMap<>();
+        dataMapping.put("Location", "geo_point");
+        Map<String, Object> metaMapping = new HashMap<>();
+        metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
+        this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
+                .build();
+
+        this.indicesClient = PowerMockito.mock(IndicesClient.class);
+        this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
+
+        when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
+        when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
+        when(this.response.getStatusLine()).thenReturn(statusLine);
+        when(this.statusLine.getStatusCode()).thenReturn(200);
+    }
+
+    @Test
+    public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
+        try {
+            String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
+            assertEquals(mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenExistType_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+
+            IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
+            doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
 }
diff --git a/provider/indexer-gc/pom.xml b/provider/indexer-gc/pom.xml
index 090598722f2c84438664cf8f2a982276e342988b..94b23a7ca4e7ae5331c0a15875c42b43e02489bd 100644
--- a/provider/indexer-gc/pom.xml
+++ b/provider/indexer-gc/pom.xml
@@ -5,12 +5,12 @@
     <parent>
         <groupId>org.opengroup.osdu.indexer</groupId>
         <artifactId>indexer-service</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
     <artifactId>indexer-gc</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <name>indexer-gc</name>
     <description>Indexer Service Google Cloud</description>
     <packaging>jar</packaging>
@@ -41,7 +41,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
             <!-- excluded due to runtime conflict with latest core-lib-gcp transient dependencies -->
             <exclusions>
                 <exclusion>
diff --git a/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImplTest.java b/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImplTest.java
deleted file mode 100644
index 7b26ad8a3a283adb917ffc00c41dd27d58f40b1a..0000000000000000000000000000000000000000
--- a/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexCopyServiceImplTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-// Copyright 2017-2019, Schlumberger
-//
-// Licensed 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.opengroup.osdu.indexer.service;
-
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
-
-import org.apache.http.HttpEntity;
-import org.apache.http.util.EntityUtils;
-import org.elasticsearch.client.Request;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.RestClient;
-import org.elasticsearch.client.RestHighLevelClient;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.ArgumentMatchers;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.opengroup.osdu.core.common.model.search.ClusterSettings;
-import org.opengroup.osdu.core.common.model.http.DpsHeaders;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.logging.AuditLogger;
-import org.opengroup.osdu.core.common.provider.interfaces.IRequestInfo;
-import org.opengroup.osdu.core.common.model.indexer.IElasticSettingService;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
-import org.opengroup.osdu.core.common.search.ElasticIndexNameResolver;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.springframework.test.context.junit4.SpringRunner;
-
-import java.io.IOException;
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-@RunWith(SpringRunner.class)
-@PrepareForTest({RestHighLevelClient.class, Response.class, RestClient.class, HttpEntity.class, EntityUtils.class})
-public class IndexCopyServiceImplTest {
-    private final String correlationId = UUID.randomUUID().toString();
-
-    @Mock
-    private HttpEntity httpEntity;
-    @Mock
-    private HttpEntity httpEntityRequest;
-    @Mock
-    private IRequestInfo requestInfo;
-    @Mock
-    private DpsHeaders headersInfo;
-    @Mock
-    private RestClient restClient;
-    @Mock
-    private RestHighLevelClient restHighLevelClient;
-    @Mock
-    private IndicesService indicesService;
-    @Mock
-    private IMappingService mappingService;
-    @Mock
-    private ElasticClientHandler elasticClientHandler;
-    @Mock
-    private ElasticIndexNameResolver elasticIndexNameResolver;
-    @Mock
-    private Response response;
-    @Mock
-    private IElasticSettingService elasticSettingService;
-    @Mock
-    private AuditLogger auditLogger;
-    @Mock
-    private Map<String, String> httpHeaders;
-    @InjectMocks
-    private IndexCopyServiceImpl sut;
-
-    private ClusterSettings commonCluster;
-
-    private Map<String, Object> correctMap;
-
-    @Before
-    public void setup() {
-
-        commonCluster = ClusterSettings.builder().host("commonhost").port(8080).userNameAndPassword("username:pwd").build();
-
-        httpHeaders = new HashMap<>();
-        httpHeaders.put(DpsHeaders.AUTHORIZATION, "testAuth");
-        httpHeaders.put(DpsHeaders.CORRELATION_ID, correlationId);
-        when(requestInfo.getHeadersMapWithDwdAuthZ()).thenReturn(httpHeaders);
-        when(response.getEntity()).thenReturn(httpEntity);
-
-        Type mapType = new TypeToken<Map<String, Object>>() {}.getType();
-        String afterFormat = "{\"properties\":{\"id\":{\"type\":\"keyword\"}}}";
-        correctMap = new Gson().fromJson(afterFormat, mapType);
-
-        restHighLevelClient = mock(RestHighLevelClient.class);
-
-    }
-
-    @Test(expected = IOException.class)
-    public void should_throwIOException_when_indexMappingNotFound() throws Exception {
-        IOException exception = new IOException("Fail to get mapping for the given index from common cluster.");
-
-        when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenThrow(exception);
-
-        this.sut.copyIndex("common:metadata:entity:1.0.0");
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void should_throwIllegalArgExceptionCopyIndexRequest_copyIndexTest() {
-        try {
-            this.sut.copyIndex(null);
-        } catch (IOException e) {
-            fail("Should not throw IOException but illegalArgumentException.");
-        }
-    }
-
-    @Test
-    public void should_returnIndexMapping_getIndexMappingFromCommonClustertest() {
-        String mappingJson = "{\"common-metadata-entity-1.0.0\":{\"mappings\":{\"entity\":{\"properties\":{\"id\":{\"type\":\"keyword\"}}}}}}";
-        when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-        try {
-            when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(mappingJson);
-            Map<String, Object> resultMap = this.sut.getIndexMappingsFromCommonCluster("test", "test");
-            Assert.assertEquals(resultMap, correctMap);
-        } catch (Exception ignored) {
-        }
-    }
-
-    @Test
-    public void should_returnClusterInfo_getCommonClusterInformationtest() {
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-            when(elasticSettingService.getElasticClusterInformation()).thenReturn(commonCluster);
-
-            String[] resultCommonCluster = this.sut.getCommonClusterInformation();
-            Assert.assertEquals(correctCommonCluster[0], resultCommonCluster[0]);
-            Assert.assertEquals(correctCommonCluster[1], resultCommonCluster[1]);
-            Assert.assertEquals(correctCommonCluster[2], resultCommonCluster[2]);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Test(expected = AppException.class)
-    public void should_throwException_failToCreateIndexInTenantCluster_createIndexInTenantClustertest() {
-        try {
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            this.sut.createIndexInTenantCluster("test", "test", "test", correctMap);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Ignore
-    public void should_returnTaskIdResponse_reindexRequestSucceed_reindexInTenantClustertest() {
-        //TODO: fix the null Response from restHighLevelClient.getLowLevelClient().performRequest().
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-            Request request = new Request("POST", "/_reindex?wait_for_completion=false");
-            request.setEntity(httpEntityRequest);
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            when(restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-            when(restClient.performRequest(request)).thenReturn(response);
-            when(response.getEntity()).thenReturn(httpEntity);
-            Assert.assertEquals(httpEntity, this.sut.reindexInTenantCluster("test", "test", correctCommonCluster));
-        } catch (IOException ignored) {
-        }
-    }
-}
diff --git a/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceTest.java b/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceTest.java
index 87dea8ec4ca8573a2b5e588fd13c978816a5898f..4d77b8d4ce9bf63cdca9c36ec0b5cd77a8c0f304 100644
--- a/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceTest.java
+++ b/provider/indexer-gc/src/test/java/org/opengroup/osdu/indexer/service/IndexerMappingServiceTest.java
@@ -1,17 +1,13 @@
 package org.opengroup.osdu.indexer.service;
 
 import org.apache.http.StatusLine;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
-import org.elasticsearch.client.*;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.index.reindex.BulkByScrollResponse;
+import org.elasticsearch.client.IndicesClient;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestHighLevelClient;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -20,284 +16,104 @@ import org.mockito.ArgumentMatchers;
 import org.mockito.InjectMocks;
 import org.mockito.Mock;
 import org.opengroup.osdu.core.common.model.indexer.IndexSchema;
-import org.opengroup.osdu.indexer.service.IndexerMappingServiceImpl;
 import org.opengroup.osdu.core.common.model.search.RecordMetaAttribute;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.springframework.test.context.junit4.SpringRunner;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
 import static org.powermock.api.mockito.PowerMockito.when;
 
 @Ignore
 @RunWith(SpringRunner.class)
-@PrepareForTest({ RestHighLevelClient.class, IndicesClient.class })
+@PrepareForTest({RestHighLevelClient.class, IndicesClient.class})
 public class IndexerMappingServiceTest {
 
-	private final String kind = "tenant:test:test:1.0.0";
-	private final String index = "tenant-test-test-1.0.0";
-	private final String type = "test";
-	private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
-
-	@Mock
-	private RestClient restClient;
-	@Mock
-	private Response response;
-	@Mock
-	private StatusLine statusLine;
-
-	@InjectMocks
-	private IndexerMappingServiceImpl sut;
-
-	@Mock
-	private ElasticClientHandler elasticClientHandler;
-
-	@InjectMocks
-	private RestHighLevelClient restHighLevelClient;
-
-	@InjectMocks
-	private IndexSchema indexSchema;
-	@InjectMocks
-	private IndicesClient indicesClient;
-
-	@InjectMocks
-	private AcknowledgedResponse mappingResponse;
-
-	@Before
-	public void setup() throws IOException {
-		Map<String, Object> dataMapping = new HashMap<>();
-		dataMapping.put("Location", "geo_point");
-		Map<String, Object> metaMapping = new HashMap<>();
-		metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
-		this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
-				.build();
-
-		this.indicesClient = PowerMockito.mock(IndicesClient.class);
-		this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
-
-		when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-		when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
-		when(this.response.getStatusLine()).thenReturn(statusLine);
-		when(this.statusLine.getStatusCode()).thenReturn(200);
-	}
-
-	@Test
-	public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
-		try {
-			String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
-			assertEquals(mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenExistType_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-
-			IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
-			doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_update_indices_field_with_keyword_when_valid_indices() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType( indices,"any field");
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_someIndex_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("invalid 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_type_of_index_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field invalid");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_elastic_search_failedToFetch_andWeIndexfield_with_keyword() throws Exception {
-		try {
-
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenThrow(new ElasticsearchException(""));
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_when_elastic_failedToIndex_indices_field_with_keyword() {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(this.indicesClient.putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class))).thenThrow(new ElasticsearchException(""));
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
+    private final String kind = "tenant:test:test:1.0.0";
+    private final String index = "tenant-test-test-1.0.0";
+    private final String type = "test";
+    private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
+
+    @Mock
+    private RestClient restClient;
+    @Mock
+    private Response response;
+    @Mock
+    private StatusLine statusLine;
+
+    @InjectMocks
+    private IndexerMappingServiceImpl sut;
+
+    @InjectMocks
+    private RestHighLevelClient restHighLevelClient;
+
+    @InjectMocks
+    private IndexSchema indexSchema;
+    @InjectMocks
+    private IndicesClient indicesClient;
+
+    @InjectMocks
+    private AcknowledgedResponse mappingResponse;
+
+    @Before
+    public void setup() throws IOException {
+        Map<String, Object> dataMapping = new HashMap<>();
+        dataMapping.put("Location", "geo_point");
+        Map<String, Object> metaMapping = new HashMap<>();
+        metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
+        this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
+                .build();
+
+        this.indicesClient = PowerMockito.mock(IndicesClient.class);
+        this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
+
+        when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
+        when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
+        when(this.response.getStatusLine()).thenReturn(statusLine);
+        when(this.statusLine.getStatusCode()).thenReturn(200);
+    }
+
+    @Test
+    public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
+        try {
+            String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
+            assertEquals(mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenExistType_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+
+            IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
+            doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
 }
diff --git a/provider/indexer-ibm/pom.xml b/provider/indexer-ibm/pom.xml
index c550b2a8642e6352f4b85e926d491434d52b33d1..82d58bd58e8364de9c639bb2ae4b24433ae9b1d1 100644
--- a/provider/indexer-ibm/pom.xml
+++ b/provider/indexer-ibm/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <groupId>org.opengroup.osdu.indexer</groupId>
         <artifactId>indexer-service</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 
@@ -57,7 +57,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
             <exclusions>
             	<exclusion>
             		<groupId>io.netty</groupId>
diff --git a/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexCopyServiceImplTest.java b/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexCopyServiceImplTest.java
deleted file mode 100644
index a12e905f8d6602f52701806e51c58cffecf66688..0000000000000000000000000000000000000000
--- a/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexCopyServiceImplTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/* Licensed Materials - Property of IBM              */		
-/* (c) Copyright IBM Corp. 2020. All Rights Reserved.*/
-
-package org.opengroup.osdu.indexer.ibm.service;
-
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
-import org.apache.http.HttpEntity;
-import org.apache.http.util.EntityUtils;
-import org.elasticsearch.client.Request;
-import org.elasticsearch.client.Response;
-import org.elasticsearch.client.RestClient;
-import org.elasticsearch.client.RestHighLevelClient;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.ArgumentMatchers;
-import org.mockito.InjectMocks;
-import org.mockito.Mock;
-import org.opengroup.osdu.core.common.model.http.DpsHeaders;
-import org.opengroup.osdu.indexer.logging.AuditLogger;
-import org.opengroup.osdu.indexer.service.IMappingService;
-import org.opengroup.osdu.indexer.service.IndexCopyServiceImpl;
-import org.opengroup.osdu.core.common.model.search.ClusterSettings;
-import org.opengroup.osdu.core.common.provider.interfaces.IRequestInfo;
-import org.opengroup.osdu.core.common.model.indexer.IElasticSettingService;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.service.IndicesService;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
-import org.opengroup.osdu.core.common.search.ElasticIndexNameResolver;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.springframework.test.context.junit4.SpringRunner;
-
-import java.io.IOException;
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.powermock.api.mockito.PowerMockito.when;
-
-@RunWith(SpringRunner.class)
-@PrepareForTest({RestHighLevelClient.class, Response.class, RestClient.class, HttpEntity.class, EntityUtils.class})
-public class IndexCopyServiceImplTest {
-    private final String correlationId = UUID.randomUUID().toString();
-
-    @Mock
-    private HttpEntity httpEntity;
-    @Mock
-    private HttpEntity httpEntityRequest;
-    @Mock
-    private IRequestInfo requestInfo;
-    @Mock
-    private DpsHeaders dpsHeaders;
-    @Mock
-    private RestClient restClient;
-    @Mock
-    private RestHighLevelClient restHighLevelClient;
-    @Mock
-    private IndicesService indicesService;
-    @Mock
-    private IMappingService mappingService;
-    @Mock
-    private ElasticClientHandler elasticClientHandler;
-    @Mock
-    private ElasticIndexNameResolver elasticIndexNameResolver;
-    @Mock
-    private Response response;
-    @Mock
-    private IElasticSettingService elasticSettingService;
-    @Mock
-    private AuditLogger auditLogger;
-    @Mock
-    private Map<String, String> httpHeaders;
-    @InjectMocks
-    private IndexCopyServiceImpl sut;
-
-    private ClusterSettings commonCluster;
-
-    private Map<String, Object> correctMap;
-
-    @Before
-    public void setup() {
-
-        commonCluster = ClusterSettings.builder().host("commonhost").port(8080).userNameAndPassword("username:pwd").build();
-
-        httpHeaders = new HashMap<>();
-        httpHeaders.put(DpsHeaders.AUTHORIZATION, "testAuth");
-        httpHeaders.put(DpsHeaders.CORRELATION_ID, correlationId);
-        when(requestInfo.getHeadersMapWithDwdAuthZ()).thenReturn(httpHeaders);
-        when(response.getEntity()).thenReturn(httpEntity);
-
-        Type mapType = new TypeToken<Map<String, Object>>() {}.getType();
-        String afterFormat = "{\"properties\":{\"id\":{\"type\":\"keyword\"}}}";
-        correctMap = new Gson().fromJson(afterFormat, mapType);
-
-        restHighLevelClient = mock(RestHighLevelClient.class);
-
-    }
-
-    @Test(expected = IOException.class)
-    public void should_throwIOException_when_indexMappingNotFound() throws Exception {
-        IOException exception = new IOException("Fail to get mapping for the given index from common cluster.");
-
-        when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenThrow(exception);
-
-        this.sut.copyIndex("common:metadata:entity:1.0.0");
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void should_throwIllegalArgExceptionCopyIndexRequest_copyIndexTest() {
-        try {
-            this.sut.copyIndex(null);
-        } catch (IOException e) {
-            fail("Should not throw IOException but illegalArgumentException.");
-        }
-    }
-
-    @Test
-    public void should_returnIndexMapping_getIndexMappingFromCommonClustertest() {
-        String mappingJson = "{\"common-metadata-entity-1.0.0\":{\"mappings\":{\"entity\":{\"properties\":{\"id\":{\"type\":\"keyword\"}}}}}}";
-        when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-        try {
-            when(this.mappingService.getIndexMapping(ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(mappingJson);
-            Map<String, Object> resultMap = this.sut.getIndexMappingsFromCommonCluster("test", "test");
-            Assert.assertEquals(resultMap, correctMap);
-        } catch (Exception ignored) {
-        }
-    }
-
-    @Test
-    public void should_returnClusterInfo_getCommonClusterInformationtest() {
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-            when(elasticSettingService.getElasticClusterInformation()).thenReturn(commonCluster);
-
-            String[] resultCommonCluster = this.sut.getCommonClusterInformation();
-            Assert.assertEquals(correctCommonCluster[0], resultCommonCluster[0]);
-            Assert.assertEquals(correctCommonCluster[1], resultCommonCluster[1]);
-            Assert.assertEquals(correctCommonCluster[2], resultCommonCluster[2]);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Test(expected = AppException.class)
-    public void should_throwException_failToCreateIndexInTenantCluster_createIndexInTenantClustertest() {
-        try {
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            this.sut.createIndexInTenantCluster("test", "test", "test", correctMap);
-        } catch (IOException ignored) {
-            fail("Should not throw this exception " + ignored.getMessage());
-        }
-    }
-
-    @Ignore
-    public void should_returnTaskIdResponse_reindexRequestSucceed_reindexInTenantClustertest() {
-        //TODO: fix the null Response from restHighLevelClient.getLowLevelClient().performRequest().
-        try {
-            String[] correctCommonCluster = {"https://commonhost:8080", "username", "pwd"};
-            Request request = new Request("POST", "/_reindex?wait_for_completion=false");
-            request.setEntity(httpEntityRequest);
-            when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-            when(indicesService.createIndex(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any())).thenReturn(false);
-            when(restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-            when(restClient.performRequest(request)).thenReturn(response);
-            when(response.getEntity()).thenReturn(httpEntity);
-            Assert.assertEquals(httpEntity, this.sut.reindexInTenantCluster("test", "test", correctCommonCluster));
-        } catch (IOException ignored) {
-        }
-    }
-}
diff --git a/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexerMappingServiceTest.java b/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexerMappingServiceTest.java
index 9e22a80fce0c02ff55d9bd940c07279dfcd39472..7f1f5a7d587538f3b376e1002224831b20f42871 100644
--- a/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexerMappingServiceTest.java
+++ b/provider/indexer-ibm/src/test/java/org/opengroup/osdu/indexer/ibm/service/IndexerMappingServiceTest.java
@@ -1,19 +1,15 @@
-/* Licensed Materials - Property of IBM              */		
+/* Licensed Materials - Property of IBM              */
 /* (c) Copyright IBM Corp. 2020. All Rights Reserved.*/
 package org.opengroup.osdu.indexer.ibm.service;
 
 import org.apache.http.StatusLine;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest;
-import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
-import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
 import org.elasticsearch.action.support.master.AcknowledgedResponse;
-import org.elasticsearch.client.*;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.index.reindex.BulkByScrollResponse;
+import org.elasticsearch.client.IndicesClient;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestHighLevelClient;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -22,283 +18,104 @@ import org.mockito.ArgumentMatchers;
 import org.mockito.InjectMocks;
 import org.mockito.Mock;
 import org.opengroup.osdu.core.common.model.indexer.IndexSchema;
-import org.opengroup.osdu.indexer.service.IndexerMappingServiceImpl;
 import org.opengroup.osdu.core.common.model.search.RecordMetaAttribute;
-import org.opengroup.osdu.core.common.model.http.AppException;
-import org.opengroup.osdu.indexer.util.ElasticClientHandler;
+import org.opengroup.osdu.indexer.service.IndexerMappingServiceImpl;
 import org.powermock.api.mockito.PowerMockito;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.springframework.test.context.junit4.SpringRunner;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
 import static org.powermock.api.mockito.PowerMockito.when;
 
 @Ignore
 @RunWith(SpringRunner.class)
-@PrepareForTest({ RestHighLevelClient.class, IndicesClient.class })
+@PrepareForTest({RestHighLevelClient.class, IndicesClient.class})
 public class IndexerMappingServiceTest {
 
-	private final String kind = "tenant:test:test:1.0.0";
-	private final String index = "tenant-test-test-1.0.0";
-	private final String type = "test";
-	private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
-
-	@Mock
-	private RestClient restClient;
-	@Mock
-	private Response response;
-	@Mock
-	private StatusLine statusLine;
-
-	@InjectMocks
-	private IndexerMappingServiceImpl sut;
-
-	@Mock
-    private ElasticClientHandler elasticClientHandler;
-
-	@InjectMocks
-	private RestHighLevelClient restHighLevelClient;
-
-	@InjectMocks
-	private IndexSchema indexSchema;
-	@InjectMocks
-	private IndicesClient indicesClient;
-
-	@InjectMocks
-	private AcknowledgedResponse mappingResponse;
-
-	@Before
-	public void setup() throws IOException {
-		Map<String, Object> dataMapping = new HashMap<>();
-		dataMapping.put("Location", "geo_point");
-		Map<String, Object> metaMapping = new HashMap<>();
-		metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
-		this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
-				.build();
-
-		this.indicesClient = PowerMockito.mock(IndicesClient.class);
-		this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
-
-		when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
-		when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
-		when(this.response.getStatusLine()).thenReturn(statusLine);
-		when(this.statusLine.getStatusCode()).thenReturn(200);
-	}
-
-	@Test
-	public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
-		try {
-			String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
-			assertEquals(mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-
-	@Test
-	public void should_returnValidMapping_givenExistType_createMappingTest() {
-		try {
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
-			doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
-			String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
-			assertEquals(this.mappingValid, mapping);
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void should_update_indices_field_with_keyword_when_valid_indices() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType( indices,"any field");
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
-	
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_someIndex_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("invalid 1");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_type_of_index_is_invalid_andWeIndexfield_with_keyword() throws Exception {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");			
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field invalid");
-		} catch (Exception e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_if_elastic_search_failedToFetch_andWeIndexfield_with_keyword() throws Exception {
-		try {
-
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenThrow(new ElasticsearchException(""));
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		}
-	}
-
-	@Test(expected = AppException.class)
-	public void should_throw_exception_when_elastic_failedToIndex_indices_field_with_keyword() {
-		try {
-			Set<String> indices = new HashSet<String>();
-			indices.add("indices 1");
-			indices.add("indices Invalid");
-			GetFieldMappingsResponse getFieldMappingsResponse = mock(GetFieldMappingsResponse.class);
-			doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
-			when(this.indicesClient.getFieldMapping(ArgumentMatchers.any(GetFieldMappingsRequest.class), ArgumentMatchers.any())).thenReturn(getFieldMappingsResponse);
-			XContentBuilder builder = XContentFactory.jsonBuilder();
-			builder.startObject();
-			builder.field("any field", new HashMap());
-			builder.endObject();
-			BytesReference bytesReference = BytesReference.bytes(builder);
-			GetFieldMappingsResponse.FieldMappingMetadata mappingMetaData = new GetFieldMappingsResponse.FieldMappingMetadata(index, bytesReference);
-			Map<String, GetFieldMappingsResponse.FieldMappingMetadata> mapBuilder = new HashMap<>();
-			mapBuilder.put("data.any field", mappingMetaData);
-			Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>> mappingBuilder = new HashMap<>();
-			mappingBuilder.put("any index 1", mapBuilder);
-			mappingBuilder.put("any index 2", mapBuilder);
-			Map<String, Map<String, Map<String, GetFieldMappingsResponse.FieldMappingMetadata>>> mapping = new HashMap<>();
-			mapping.put("indices 1", mappingBuilder);
-			when(getFieldMappingsResponse.mappings()).thenReturn(mapping);
-			doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
-			BulkByScrollResponse response = mock(BulkByScrollResponse.class);
-			doReturn(response).when(this.restHighLevelClient).updateByQuery(ArgumentMatchers.any(), ArgumentMatchers.any(RequestOptions.class));
-			when(response.getBulkFailures()).thenReturn(new ArrayList<Failure>());
-			when(this.indicesClient.putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class))).thenThrow(new ElasticsearchException(""));
-			when(elasticClientHandler.createRestClient()).thenReturn(restHighLevelClient);
-			this.sut.updateIndexMappingForIndicesOfSameType(indices,"any field");
-		} catch (AppException e) {
-			throw e;
-		} catch (Exception e) {
-			fail("Should not throw this exception" + e.getMessage());
-		}
-	}
+    private final String kind = "tenant:test:test:1.0.0";
+    private final String index = "tenant-test-test-1.0.0";
+    private final String type = "test";
+    private final String mappingValid = "{\"dynamic\":false,\"properties\":{\"data\":{\"properties\":{\"Location\":{\"type\":\"geo_point\"}}},\"id\":{\"type\":\"keyword\"}}}";
+
+    @Mock
+    private RestClient restClient;
+    @Mock
+    private Response response;
+    @Mock
+    private StatusLine statusLine;
+
+    @InjectMocks
+    private IndexerMappingServiceImpl sut;
+
+    @InjectMocks
+    private RestHighLevelClient restHighLevelClient;
+
+    @InjectMocks
+    private IndexSchema indexSchema;
+    @InjectMocks
+    private IndicesClient indicesClient;
+
+    @InjectMocks
+    private AcknowledgedResponse mappingResponse;
+
+    @Before
+    public void setup() throws IOException {
+        Map<String, Object> dataMapping = new HashMap<>();
+        dataMapping.put("Location", "geo_point");
+        Map<String, Object> metaMapping = new HashMap<>();
+        metaMapping.put(RecordMetaAttribute.ID.getValue(), "keyword");
+        this.indexSchema = IndexSchema.builder().kind(kind).type(type).dataSchema(dataMapping).metaSchema(metaMapping)
+                .build();
+
+        this.indicesClient = PowerMockito.mock(IndicesClient.class);
+        this.restHighLevelClient = PowerMockito.mock(RestHighLevelClient.class);
+
+        when(this.restHighLevelClient.getLowLevelClient()).thenReturn(restClient);
+        when(this.restClient.performRequest(ArgumentMatchers.any())).thenReturn(response);
+        when(this.response.getStatusLine()).thenReturn(statusLine);
+        when(this.statusLine.getStatusCode()).thenReturn(200);
+    }
+
+    @Test
+    public void should_returnValidMapping_givenFalseMerge_createMappingTest() {
+        try {
+            String mapping = this.sut.createMapping(restHighLevelClient, indexSchema, index, false);
+            assertEquals(mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenTrueMerge_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
+
+    @Test
+    public void should_returnValidMapping_givenExistType_createMappingTest() {
+        try {
+            doReturn(this.indicesClient).when(this.restHighLevelClient).indices();
+            doReturn(mappingResponse).when(this.indicesClient).putMapping(ArgumentMatchers.any(PutMappingRequest.class), ArgumentMatchers.any(RequestOptions.class));
+            IndexerMappingServiceImpl indexerMappingServiceLocal = PowerMockito.spy(new IndexerMappingServiceImpl());
+            doReturn(false).when(indexerMappingServiceLocal).isTypeExist(ArgumentMatchers.any(), ArgumentMatchers.any(), ArgumentMatchers.any());
+            String mapping = this.sut.createMapping(this.restHighLevelClient, this.indexSchema, this.index, true);
+            assertEquals(this.mappingValid, mapping);
+        } catch (Exception e) {
+            fail("Should not throw this exception" + e.getMessage());
+        }
+    }
 }
 
diff --git a/testing/indexer-test-anthos/pom.xml b/testing/indexer-test-anthos/pom.xml
index 1a521499d7790f7ea1e1df1a51cc8eebe957ff45..c1c1191aa79fadaff4e64c4c9deb6e6a91fbce22 100644
--- a/testing/indexer-test-anthos/pom.xml
+++ b/testing/indexer-test-anthos/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>indexer-test</artifactId>
         <groupId>org.opengroup.osdu</groupId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
@@ -36,7 +36,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-test-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>com.nimbusds</groupId>
diff --git a/testing/indexer-test-aws/pom.xml b/testing/indexer-test-aws/pom.xml
index 19bdc68759d3923f93fed0ad09912d37f1af928a..4873e4dd208a357a1edd026e1c49b299ca5dded0 100644
--- a/testing/indexer-test-aws/pom.xml
+++ b/testing/indexer-test-aws/pom.xml
@@ -21,13 +21,13 @@
     <parent>
         <groupId>org.opengroup.osdu</groupId>
         <artifactId>indexer-test</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
 
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-test-aws</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -44,7 +44,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-test-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
         </dependency>
 
         <!-- AWS specific packages -->
diff --git a/testing/indexer-test-azure/pom.xml b/testing/indexer-test-azure/pom.xml
index 1f441ec059f5a65b4a6cc6764190e068b1cc20c9..c636dad6ad94d6143ef73dbef89a7bcc0e753fb2 100644
--- a/testing/indexer-test-azure/pom.xml
+++ b/testing/indexer-test-azure/pom.xml
@@ -21,13 +21,13 @@
     <parent>
         <groupId>org.opengroup.osdu</groupId>
         <artifactId>indexer-test</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
 
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-test-azure</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -46,7 +46,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-test-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
             <exclusions>
                 <exclusion>
                     <groupId>org.slf4j</groupId>
diff --git a/testing/indexer-test-core/pom.xml b/testing/indexer-test-core/pom.xml
index b1a4bb19f0985c899b10a85a162cd8b1ce0f6934..c5e856334d3d9cb01b22885714b3841db7668a3e 100644
--- a/testing/indexer-test-core/pom.xml
+++ b/testing/indexer-test-core/pom.xml
@@ -5,13 +5,13 @@
     <parent>
         <groupId>org.opengroup.osdu</groupId>
         <artifactId>indexer-test</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
 
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-test-core</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
 
     <properties>
         <maven.compiler.target>1.8</maven.compiler.target>
diff --git a/testing/indexer-test-gc/pom.xml b/testing/indexer-test-gc/pom.xml
index 5873bff29b58b84629d995b20d44ecdaa86eda90..db3c4f41e71c5524572f9f433a4d7288532370e1 100644
--- a/testing/indexer-test-gc/pom.xml
+++ b/testing/indexer-test-gc/pom.xml
@@ -6,13 +6,13 @@
     <parent>
         <groupId>org.opengroup.osdu</groupId>
         <artifactId>indexer-test</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
 
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-test-gc</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -37,7 +37,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-test-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
         </dependency>
 
         <!-- Cucumber -->
diff --git a/testing/indexer-test-ibm/pom.xml b/testing/indexer-test-ibm/pom.xml
index a749aeda28ce6bea5d50fdc801e80cccef7936a4..6d27a4904bff803ef284949fd99b9730e0bd6f28 100644
--- a/testing/indexer-test-ibm/pom.xml
+++ b/testing/indexer-test-ibm/pom.xml
@@ -6,13 +6,13 @@
     <parent>
         <groupId>org.opengroup.osdu</groupId>
         <artifactId>indexer-test</artifactId>
-        <version>0.20.0-SNAPSHOT</version>
+        <version>0.21.0-SNAPSHOT</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
 
     <groupId>org.opengroup.osdu.indexer</groupId>
     <artifactId>indexer-test-ibm</artifactId>
-    <version>0.20.0-SNAPSHOT</version>
+    <version>0.21.0-SNAPSHOT</version>
     <packaging>jar</packaging>
 
     <properties>
@@ -38,7 +38,7 @@
         <dependency>
             <groupId>org.opengroup.osdu.indexer</groupId>
             <artifactId>indexer-test-core</artifactId>
-            <version>0.20.0-SNAPSHOT</version>
+            <version>0.21.0-SNAPSHOT</version>
         </dependency>
 
         <dependency>
diff --git a/testing/pom.xml b/testing/pom.xml
index ddaf9284a1e76083eddc4c097f9768cbef2080b6..0af155f036c2d6b01b7720d850d3555e49beef36 100644
--- a/testing/pom.xml
+++ b/testing/pom.xml
@@ -18,7 +18,7 @@
 	<modelVersion>4.0.0</modelVersion>
 	<groupId>org.opengroup.osdu</groupId>
 	<artifactId>indexer-test</artifactId>
-	<version>0.20.0-SNAPSHOT</version>
+	<version>0.21.0-SNAPSHOT</version>
 	<description>Indexer Service Integration Test Root Project</description>
 	<properties>
 		<spring.version>5.3.23</spring.version>