add mapping metadata to cluster state metadata, will allow in the future to add specific mappings hints (since we don't have the parsed mapping on each node anymore)

This commit is contained in:
kimchy 2010-11-15 21:21:18 +02:00
parent 51c18cd9c1
commit b8b4cbbb46
11 changed files with 125 additions and 56 deletions

View File

@ -92,11 +92,11 @@ public class IndexMetaData {
private final Settings settings; private final Settings settings;
private final ImmutableMap<String, CompressedString> mappings; private final ImmutableMap<String, MappingMetaData> mappings;
private transient final int totalNumberOfShards; private transient final int totalNumberOfShards;
private IndexMetaData(String index, State state, Settings settings, ImmutableMap<String, CompressedString> mappings) { private IndexMetaData(String index, State state, Settings settings, ImmutableMap<String, MappingMetaData> mappings) {
Preconditions.checkArgument(settings.getAsInt(SETTING_NUMBER_OF_SHARDS, -1) != -1, "must specify numberOfShards for index [" + index + "]"); Preconditions.checkArgument(settings.getAsInt(SETTING_NUMBER_OF_SHARDS, -1) != -1, "must specify numberOfShards for index [" + index + "]");
Preconditions.checkArgument(settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, -1) != -1, "must specify numberOfReplicas for index [" + index + "]"); Preconditions.checkArgument(settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, -1) != -1, "must specify numberOfReplicas for index [" + index + "]");
this.index = index; this.index = index;
@ -164,15 +164,15 @@ public class IndexMetaData {
return aliases(); return aliases();
} }
public ImmutableMap<String, CompressedString> mappings() { public ImmutableMap<String, MappingMetaData> mappings() {
return mappings; return mappings;
} }
public ImmutableMap<String, CompressedString> getMappings() { public ImmutableMap<String, MappingMetaData> getMappings() {
return mappings(); return mappings();
} }
public CompressedString mapping(String mappingType) { public MappingMetaData mapping(String mappingType) {
return mappings.get(mappingType); return mappings.get(mappingType);
} }
@ -192,7 +192,7 @@ public class IndexMetaData {
private Settings settings = ImmutableSettings.Builder.EMPTY_SETTINGS; private Settings settings = ImmutableSettings.Builder.EMPTY_SETTINGS;
private MapBuilder<String, CompressedString> mappings = MapBuilder.newMapBuilder(); private MapBuilder<String, MappingMetaData> mappings = MapBuilder.newMapBuilder();
public Builder(String index) { public Builder(String index) {
this.index = index; this.index = index;
@ -242,14 +242,13 @@ public class IndexMetaData {
return this; return this;
} }
public Builder putMapping(String mappingType, CompressedString mappingSource) { public Builder putMapping(MappingMetaData mappingMd) {
mappings.put(mappingType, mappingSource); mappings.put(mappingMd.type(), mappingMd);
return this; return this;
} }
public Builder putMapping(String mappingType, String mappingSource) throws IOException { public Builder putMapping(String mappingType, String mappingSource) throws IOException {
mappings.put(mappingType, new CompressedString(mappingSource)); return putMapping(new MappingMetaData(mappingType, new CompressedString(mappingSource)));
return this;
} }
public Builder state(State state) { public Builder state(State state) {
@ -273,8 +272,8 @@ public class IndexMetaData {
builder.endObject(); builder.endObject();
builder.startArray("mappings"); builder.startArray("mappings");
for (Map.Entry<String, CompressedString> entry : indexMetaData.mappings().entrySet()) { for (Map.Entry<String, MappingMetaData> entry : indexMetaData.mappings().entrySet()) {
byte[] data = entry.getValue().uncompressed(); byte[] data = entry.getValue().source().uncompressed();
XContentParser parser = XContentFactory.xContent(data).createParser(data); XContentParser parser = XContentFactory.xContent(data).createParser(data);
Map<String, Object> mapping = parser.map(); Map<String, Object> mapping = parser.map();
parser.close(); parser.close();
@ -307,12 +306,13 @@ public class IndexMetaData {
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
Map<String, Object> mapping = parser.map(); Map<String, Object> mapping = parser.map();
if (mapping.size() == 1) { if (mapping.size() == 1) {
String mappingType = mapping.keySet().iterator().next();
String mappingSource = XContentFactory.jsonBuilder().map(mapping).string(); String mappingSource = XContentFactory.jsonBuilder().map(mapping).string();
if (mappingSource == null) { if (mappingSource == null) {
// crap, no mapping source, warn? // crap, no mapping source, warn?
} else { } else {
builder.putMapping(mapping.keySet().iterator().next(), new CompressedString(mappingSource)); builder.putMapping(new MappingMetaData(mappingType, new CompressedString(mappingSource)));
} }
} }
} }
@ -332,7 +332,8 @@ public class IndexMetaData {
builder.settings(readSettingsFromStream(in, globalSettings)); builder.settings(readSettingsFromStream(in, globalSettings));
int mappingsSize = in.readVInt(); int mappingsSize = in.readVInt();
for (int i = 0; i < mappingsSize; i++) { for (int i = 0; i < mappingsSize; i++) {
builder.putMapping(in.readUTF(), CompressedString.readCompressedString(in)); MappingMetaData mappingMd = MappingMetaData.readFrom(in);
builder.putMapping(mappingMd);
} }
return builder.build(); return builder.build();
} }
@ -342,9 +343,8 @@ public class IndexMetaData {
out.writeByte(indexMetaData.state().id()); out.writeByte(indexMetaData.state().id());
writeSettingsToStream(indexMetaData.settings(), out); writeSettingsToStream(indexMetaData.settings(), out);
out.writeVInt(indexMetaData.mappings().size()); out.writeVInt(indexMetaData.mappings().size());
for (Map.Entry<String, CompressedString> entry : indexMetaData.mappings().entrySet()) { for (MappingMetaData mappingMd : indexMetaData.mappings().values()) {
out.writeUTF(entry.getKey()); MappingMetaData.writeTo(mappingMd, out);
entry.getValue().writeTo(out);
} }
} }
} }

View File

@ -0,0 +1,64 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.mapper.DocumentMapper;
import java.io.IOException;
/**
* @author kimchy (shay.banon)
*/
public class MappingMetaData {
private final String type;
private final CompressedString source;
public MappingMetaData(DocumentMapper docMapper) {
this.type = docMapper.type();
this.source = docMapper.mappingSource();
}
public MappingMetaData(String type, CompressedString source) {
this.type = type;
this.source = source;
}
public String type() {
return this.type;
}
public CompressedString source() {
return this.source;
}
public static void writeTo(MappingMetaData mappingMd, StreamOutput out) throws IOException {
out.writeUTF(mappingMd.type());
mappingMd.source().writeTo(out);
}
public static MappingMetaData readFrom(StreamInput in) throws IOException {
return new MappingMetaData(in.readUTF(), CompressedString.readCompressedString(in));
}
}

View File

@ -186,14 +186,15 @@ public class MetaDataCreateIndexService extends AbstractComponent {
} }
} }
// now, update the mappings with the actual source // now, update the mappings with the actual source
mappings.clear(); Map<String, MappingMetaData> mappingsMetaData = Maps.newHashMap();
for (DocumentMapper mapper : mapperService) { for (DocumentMapper mapper : mapperService) {
mappings.put(mapper.type(), mapper.mappingSource()); MappingMetaData mappingMd = new MappingMetaData(mapper);
mappingsMetaData.put(mapper.type(), mappingMd);
} }
final IndexMetaData.Builder indexMetaDataBuilder = newIndexMetaDataBuilder(request.index).settings(actualIndexSettings); final IndexMetaData.Builder indexMetaDataBuilder = newIndexMetaDataBuilder(request.index).settings(actualIndexSettings);
for (Map.Entry<String, CompressedString> entry : mappings.entrySet()) { for (MappingMetaData mappingMd : mappingsMetaData.values()) {
indexMetaDataBuilder.putMapping(entry.getKey(), entry.getValue()); indexMetaDataBuilder.putMapping(mappingMd);
} }
indexMetaDataBuilder.state(request.state); indexMetaDataBuilder.state(request.state);
final IndexMetaData indexMetaData = indexMetaDataBuilder.build(); final IndexMetaData indexMetaData = indexMetaDataBuilder.build();
@ -300,6 +301,13 @@ public class MetaDataCreateIndexService extends AbstractComponent {
return this; return this;
} }
public Request mappingsMetaData(Map<String, MappingMetaData> mappings) throws IOException {
for (Map.Entry<String, MappingMetaData> entry : mappings.entrySet()) {
this.mappings.put(entry.getKey(), entry.getValue().source().string());
}
return this;
}
public Request mappingsCompressed(Map<String, CompressedString> mappings) throws IOException { public Request mappingsCompressed(Map<String, CompressedString> mappings) throws IOException {
for (Map.Entry<String, CompressedString> entry : mappings.entrySet()) { for (Map.Entry<String, CompressedString> entry : mappings.entrySet()) {
this.mappings.put(entry.getKey(), entry.getValue().string()); this.mappings.put(entry.getKey(), entry.getValue().string());

View File

@ -23,7 +23,6 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
@ -77,7 +76,7 @@ public class MetaDataMappingService extends AbstractComponent {
indexService = indicesService.createIndex(indexMetaData.index(), indexMetaData.settings(), currentState.nodes().localNode().id()); indexService = indicesService.createIndex(indexMetaData.index(), indexMetaData.settings(), currentState.nodes().localNode().id());
// only add the current relevant mapping (if exists) // only add the current relevant mapping (if exists)
if (indexMetaData.mappings().containsKey(type)) { if (indexMetaData.mappings().containsKey(type)) {
indexService.mapperService().add(type, indexMetaData.mappings().get(type).string()); indexService.mapperService().add(type, indexMetaData.mappings().get(type).source().string());
} }
} }
MapperService mapperService = indexService.mapperService(); MapperService mapperService = indexService.mapperService();
@ -104,7 +103,7 @@ public class MetaDataMappingService extends AbstractComponent {
MetaData.Builder builder = newMetaDataBuilder().metaData(currentState.metaData()); MetaData.Builder builder = newMetaDataBuilder().metaData(currentState.metaData());
IndexMetaData indexMetaData = currentState.metaData().index(index); IndexMetaData indexMetaData = currentState.metaData().index(index);
builder.put(newIndexMetaDataBuilder(indexMetaData).putMapping(type, existingMapper.mappingSource())); builder.put(newIndexMetaDataBuilder(indexMetaData).putMapping(new MappingMetaData(existingMapper)));
return newClusterStateBuilder().state(currentState).metaData(builder).build(); return newClusterStateBuilder().state(currentState).metaData(builder).build();
} catch (Exception e) { } catch (Exception e) {
logger.warn("failed to dynamically update the mapping in cluster_state from shard", e); logger.warn("failed to dynamically update the mapping in cluster_state from shard", e);
@ -160,7 +159,7 @@ public class MetaDataMappingService extends AbstractComponent {
IndexService indexService = indicesService.createIndex(indexMetaData.index(), indexMetaData.settings(), currentState.nodes().localNode().id()); IndexService indexService = indicesService.createIndex(indexMetaData.index(), indexMetaData.settings(), currentState.nodes().localNode().id());
// only add the current relevant mapping (if exists) // only add the current relevant mapping (if exists)
if (indexMetaData.mappings().containsKey(request.mappingType)) { if (indexMetaData.mappings().containsKey(request.mappingType)) {
indexService.mapperService().add(request.mappingType, indexMetaData.mappings().get(request.mappingType).string()); indexService.mapperService().add(request.mappingType, indexMetaData.mappings().get(request.mappingType).source().string());
} }
} }
@ -197,7 +196,7 @@ public class MetaDataMappingService extends AbstractComponent {
throw new InvalidTypeNameException("Document mapping type name can't start with '_'"); throw new InvalidTypeNameException("Document mapping type name can't start with '_'");
} }
final Map<String, Tuple<String, CompressedString>> mappings = newHashMap(); final Map<String, MappingMetaData> mappings = newHashMap();
for (Map.Entry<String, DocumentMapper> entry : newMappers.entrySet()) { for (Map.Entry<String, DocumentMapper> entry : newMappers.entrySet()) {
String index = entry.getKey(); String index = entry.getKey();
// do the actual merge here on the master, and update the mapping source // do the actual merge here on the master, and update the mapping source
@ -214,7 +213,7 @@ public class MetaDataMappingService extends AbstractComponent {
// same source, no changes, ignore it // same source, no changes, ignore it
} else { } else {
// use the merged mapping source // use the merged mapping source
mappings.put(index, new Tuple<String, CompressedString>(existingMapper.type(), updatedSource)); mappings.put(index, new MappingMetaData(existingMapper));
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("[{}] update_mapping [{}] with source [{}]", index, existingMapper.type(), updatedSource); logger.debug("[{}] update_mapping [{}] with source [{}]", index, existingMapper.type(), updatedSource);
} else if (logger.isInfoEnabled()) { } else if (logger.isInfoEnabled()) {
@ -223,7 +222,7 @@ public class MetaDataMappingService extends AbstractComponent {
} }
} else { } else {
CompressedString newSource = newMapper.mappingSource(); CompressedString newSource = newMapper.mappingSource();
mappings.put(index, new Tuple<String, CompressedString>(newMapper.type(), newSource)); mappings.put(index, new MappingMetaData(newMapper));
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("[{}] create_mapping [{}] with source [{}]", index, newMapper.type(), newSource); logger.debug("[{}] create_mapping [{}] with source [{}]", index, newMapper.type(), newSource);
} else if (logger.isInfoEnabled()) { } else if (logger.isInfoEnabled()) {
@ -244,9 +243,9 @@ public class MetaDataMappingService extends AbstractComponent {
if (indexMetaData == null) { if (indexMetaData == null) {
throw new IndexMissingException(new Index(indexName)); throw new IndexMissingException(new Index(indexName));
} }
Tuple<String, CompressedString> mapping = mappings.get(indexName); MappingMetaData mappingMd = mappings.get(indexName);
if (mapping != null) { if (mappingMd != null) {
builder.put(newIndexMetaDataBuilder(indexMetaData).putMapping(mapping.v1(), mapping.v2())); builder.put(newIndexMetaDataBuilder(indexMetaData).putMapping(mappingMd));
} }
} }

View File

@ -180,7 +180,7 @@ public class LocalGateway extends AbstractLifecycleComponent<Gateway> implements
try { try {
createIndexService.createIndex(new MetaDataCreateIndexService.Request("gateway", indexMetaData.index()) createIndexService.createIndex(new MetaDataCreateIndexService.Request("gateway", indexMetaData.index())
.settings(indexMetaData.settings()) .settings(indexMetaData.settings())
.mappingsCompressed(indexMetaData.mappings()) .mappingsMetaData(indexMetaData.mappings())
.state(indexMetaData.state()) .state(indexMetaData.state())
.blocks(ImmutableSet.of(GatewayService.INDEX_NOT_RECOVERED_BLOCK)) .blocks(ImmutableSet.of(GatewayService.INDEX_NOT_RECOVERED_BLOCK))
.timeout(timeValueSeconds(30)), .timeout(timeValueSeconds(30)),

View File

@ -149,7 +149,7 @@ public abstract class SharedStorageGateway extends AbstractLifecycleComponent<Ga
try { try {
createIndexService.createIndex(new MetaDataCreateIndexService.Request("gateway", indexMetaData.index()) createIndexService.createIndex(new MetaDataCreateIndexService.Request("gateway", indexMetaData.index())
.settings(indexMetaData.settings()) .settings(indexMetaData.settings())
.mappingsCompressed(indexMetaData.mappings()) .mappingsMetaData(indexMetaData.mappings())
.state(indexMetaData.state()) .state(indexMetaData.state())
.blocks(ImmutableSet.of(GatewayService.INDEX_NOT_RECOVERED_BLOCK)) .blocks(ImmutableSet.of(GatewayService.INDEX_NOT_RECOVERED_BLOCK))
.timeout(timeValueSeconds(30)), .timeout(timeValueSeconds(30)),

View File

@ -28,10 +28,10 @@ import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
import org.elasticsearch.cluster.action.index.NodeMappingCreatedAction; import org.elasticsearch.cluster.action.index.NodeMappingCreatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.*; import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.common.collect.ImmutableMap;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.compress.CompressedString;
@ -55,7 +55,6 @@ import org.elasticsearch.index.shard.service.InternalIndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
@ -219,11 +218,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
String index = indexMetaData.index(); String index = indexMetaData.index();
IndexService indexService = indicesService.indexServiceSafe(index); IndexService indexService = indicesService.indexServiceSafe(index);
MapperService mapperService = indexService.mapperService(); MapperService mapperService = indexService.mapperService();
ImmutableMap<String, CompressedString> mappings = indexMetaData.mappings();
// go over and add the relevant mappings (or update them) // go over and add the relevant mappings (or update them)
for (Map.Entry<String, CompressedString> entry : mappings.entrySet()) { for (MappingMetaData mappingMd : indexMetaData.mappings().values()) {
String mappingType = entry.getKey(); String mappingType = mappingMd.type();
CompressedString mappingSource = entry.getValue(); CompressedString mappingSource = mappingMd.source();
if (!seenMappings.containsKey(new Tuple<String, String>(index, mappingType))) { if (!seenMappings.containsKey(new Tuple<String, String>(index, mappingType))) {
seenMappings.put(new Tuple<String, String>(index, mappingType), true); seenMappings.put(new Tuple<String, String>(index, mappingType), true);
} }
@ -252,7 +250,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
} }
// go over and remove mappings // go over and remove mappings
for (DocumentMapper documentMapper : mapperService) { for (DocumentMapper documentMapper : mapperService) {
if (seenMappings.containsKey(new Tuple<String, String>(index, documentMapper.type())) && !mappings.containsKey(documentMapper.type())) { if (seenMappings.containsKey(new Tuple<String, String>(index, documentMapper.type())) && !indexMetaData.mappings().containsKey(documentMapper.type())) {
// we have it in our mappings, but not in the metadata, and we have seen it in the cluster state, remove it // we have it in our mappings, but not in the metadata, and we have seen it in the cluster state, remove it
mapperService.remove(documentMapper.type()); mapperService.remove(documentMapper.type());
seenMappings.remove(new Tuple<String, String>(index, documentMapper.type())); seenMappings.remove(new Tuple<String, String>(index, documentMapper.type()));

View File

@ -27,6 +27,7 @@ import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@ -34,7 +35,6 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.AllocationExplanation; import org.elasticsearch.cluster.routing.allocation.AllocationExplanation;
import org.elasticsearch.common.collect.ImmutableSet; import org.elasticsearch.common.collect.ImmutableSet;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsFilter;
@ -150,8 +150,8 @@ public class RestClusterStateAction extends BaseRestHandler {
builder.endObject(); builder.endObject();
builder.startObject("mappings"); builder.startObject("mappings");
for (Map.Entry<String, CompressedString> entry : indexMetaData.mappings().entrySet()) { for (Map.Entry<String, MappingMetaData> entry : indexMetaData.mappings().entrySet()) {
byte[] mappingSource = entry.getValue().uncompressed(); byte[] mappingSource = entry.getValue().source().uncompressed();
XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource); XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource);
Map<String, Object> mapping = parser.map(); Map<String, Object> mapping = parser.map();
if (mapping.size() == 1 && mapping.containsKey(entry.getKey())) { if (mapping.size() == 1 && mapping.containsKey(entry.getKey())) {

View File

@ -25,9 +25,9 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests; import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.collect.ImmutableSet; import org.elasticsearch.common.collect.ImmutableSet;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
@ -75,19 +75,19 @@ public class RestGetMappingAction extends BaseRestHandler {
for (IndexMetaData indexMetaData : metaData) { for (IndexMetaData indexMetaData : metaData) {
builder.startObject(indexMetaData.index()); builder.startObject(indexMetaData.index());
for (Map.Entry<String, CompressedString> entry : indexMetaData.mappings().entrySet()) { for (MappingMetaData mappingMd : indexMetaData.mappings().values()) {
if (!types.isEmpty() && !types.contains(entry.getKey())) { if (!types.isEmpty() && !types.contains(mappingMd.type())) {
// filter this type out... // filter this type out...
continue; continue;
} }
byte[] mappingSource = entry.getValue().uncompressed(); byte[] mappingSource = mappingMd.source().uncompressed();
XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource); XContentParser parser = XContentFactory.xContent(mappingSource).createParser(mappingSource);
Map<String, Object> mapping = parser.map(); Map<String, Object> mapping = parser.map();
if (mapping.size() == 1 && mapping.containsKey(entry.getKey())) { if (mapping.size() == 1 && mapping.containsKey(mappingMd.type())) {
// the type name is the root value, reduce it // the type name is the root value, reduce it
mapping = (Map<String, Object>) mapping.get(entry.getKey()); mapping = (Map<String, Object>) mapping.get(mappingMd.type());
} }
builder.field(entry.getKey()); builder.field(mappingMd.type());
builder.map(mapping); builder.map(mapping);
} }

View File

@ -27,11 +27,11 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.collect.Lists; import org.elasticsearch.common.collect.Lists;
import org.elasticsearch.common.collect.Maps; import org.elasticsearch.common.collect.Maps;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.common.xcontent.support.XContentMapValues;
@ -95,8 +95,8 @@ public class RiversRouter extends AbstractLifecycleComponent<RiversRouter> imple
IndexMetaData indexMetaData = event.state().metaData().index(riverIndexName); IndexMetaData indexMetaData = event.state().metaData().index(riverIndexName);
// go over and create new river routing (with no node) for new types (rivers names) // go over and create new river routing (with no node) for new types (rivers names)
for (Map.Entry<String, CompressedString> entry : indexMetaData.mappings().entrySet()) { for (MappingMetaData mappingMd : indexMetaData.mappings().values()) {
String mappingType = entry.getKey(); // mapping type is the name of the river String mappingType = mappingMd.type(); // mapping type is the name of the river
if (!currentState.routing().hasRiverByName(mappingType)) { if (!currentState.routing().hasRiverByName(mappingType)) {
// no river, we need to add it to the routing with no node allocation // no river, we need to add it to the routing with no node allocation
try { try {

View File

@ -83,7 +83,7 @@ public class ToAndFromJsonMetaDataTests {
assertThat(indexMetaData.numberOfReplicas(), equalTo(2)); assertThat(indexMetaData.numberOfReplicas(), equalTo(2));
assertThat(indexMetaData.settings().getAsMap().size(), equalTo(2)); assertThat(indexMetaData.settings().getAsMap().size(), equalTo(2));
assertThat(indexMetaData.mappings().size(), equalTo(1)); assertThat(indexMetaData.mappings().size(), equalTo(1));
assertThat(indexMetaData.mappings().get("mapping1").string(), equalTo(MAPPING_SOURCE1)); assertThat(indexMetaData.mappings().get("mapping1").source().string(), equalTo(MAPPING_SOURCE1));
indexMetaData = parsedMetaData.index("test4"); indexMetaData = parsedMetaData.index("test4");
assertThat(indexMetaData.numberOfShards(), equalTo(1)); assertThat(indexMetaData.numberOfShards(), equalTo(1));
@ -92,8 +92,8 @@ public class ToAndFromJsonMetaDataTests {
assertThat(indexMetaData.settings().get("setting1"), equalTo("value1")); assertThat(indexMetaData.settings().get("setting1"), equalTo("value1"));
assertThat(indexMetaData.settings().get("setting2"), equalTo("value2")); assertThat(indexMetaData.settings().get("setting2"), equalTo("value2"));
assertThat(indexMetaData.mappings().size(), equalTo(2)); assertThat(indexMetaData.mappings().size(), equalTo(2));
assertThat(indexMetaData.mappings().get("mapping1").string(), equalTo(MAPPING_SOURCE1)); assertThat(indexMetaData.mappings().get("mapping1").source().string(), equalTo(MAPPING_SOURCE1));
assertThat(indexMetaData.mappings().get("mapping2").string(), equalTo(MAPPING_SOURCE2)); assertThat(indexMetaData.mappings().get("mapping2").source().string(), equalTo(MAPPING_SOURCE2));
} }
private static final String MAPPING_SOURCE1 = "{\"mapping1\":{\"text1\":{\"type\":\"string\"}}}"; private static final String MAPPING_SOURCE1 = "{\"mapping1\":{\"text1\":{\"type\":\"string\"}}}";