]> git.proxmox.com Git - ceph.git/blame - ceph/src/arrow/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Schema.java
import quincy 17.2.0
[ceph.git] / ceph / src / arrow / java / vector / src / main / java / org / apache / arrow / vector / types / pojo / Schema.java
CommitLineData
1d09f67e
TL
1/*
2 * Licensed to the Apache Software Foundation (ASF) under one or more
3 * contributor license agreements. See the NOTICE file distributed with
4 * this work for additional information regarding copyright ownership.
5 * The ASF licenses this file to You under the Apache License, Version 2.0
6 * (the "License"); you may not use this file except in compliance with
7 * the License. You may obtain a copy of the License at
8 *
9 * http://www.apache.org/licenses/LICENSE-2.0
10 *
11 * Unless required by applicable law or agreed to in writing, software
12 * distributed under the License is distributed on an "AS IS" BASIS,
13 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 * See the License for the specific language governing permissions and
15 * limitations under the License.
16 */
17
18package org.apache.arrow.vector.types.pojo;
19
20
21import static org.apache.arrow.vector.types.pojo.Field.convertField;
22
23import java.io.IOException;
24import java.nio.ByteBuffer;
25import java.util.AbstractMap;
26import java.util.ArrayList;
27import java.util.Collections;
28import java.util.HashMap;
29import java.util.List;
30import java.util.Map;
31import java.util.Objects;
32import java.util.stream.Collectors;
33
34import org.apache.arrow.flatbuf.KeyValue;
35import org.apache.arrow.util.Collections2;
36import org.apache.arrow.util.Preconditions;
37import org.apache.arrow.vector.ipc.message.FBSerializables;
38
39import com.fasterxml.jackson.annotation.JsonCreator;
40import com.fasterxml.jackson.annotation.JsonIgnore;
41import com.fasterxml.jackson.annotation.JsonInclude;
42import com.fasterxml.jackson.annotation.JsonInclude.Include;
43import com.fasterxml.jackson.annotation.JsonProperty;
44import com.fasterxml.jackson.core.JsonProcessingException;
45import com.fasterxml.jackson.databind.ObjectMapper;
46import com.fasterxml.jackson.databind.ObjectReader;
47import com.fasterxml.jackson.databind.ObjectWriter;
48import com.google.flatbuffers.FlatBufferBuilder;
49
50/**
51 * An Arrow Schema.
52 */
53public class Schema {
54
55 /**
56 * Search for a field by name in given the list of fields.
57 *
58 * @param fields the list of the fields
59 * @param name the name of the field to return
60 * @return the corresponding field
61 * @throws IllegalArgumentException if the field was not found
62 */
63 public static Field findField(List<Field> fields, String name) {
64 for (Field field : fields) {
65 if (field.getName().equals(name)) {
66 return field;
67 }
68 }
69 throw new IllegalArgumentException(String.format("field %s not found in %s", name, fields));
70 }
71
72 static final String METADATA_KEY = "key";
73 static final String METADATA_VALUE = "value";
74
75 private static final ObjectMapper mapper = new ObjectMapper();
76 private static final ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
77 private static final ObjectReader reader = mapper.readerFor(Schema.class);
78
79 public static Schema fromJSON(String json) throws IOException {
80 return reader.readValue(Preconditions.checkNotNull(json));
81 }
82
83 public static Schema deserialize(ByteBuffer buffer) {
84 return convertSchema(org.apache.arrow.flatbuf.Schema.getRootAsSchema(buffer));
85 }
86
87 /** Converts a flatbuffer schema to its POJO representation. */
88 public static Schema convertSchema(org.apache.arrow.flatbuf.Schema schema) {
89 List<Field> fields = new ArrayList<>();
90 for (int i = 0; i < schema.fieldsLength(); i++) {
91 fields.add(convertField(schema.fields(i)));
92 }
93 Map<String, String> metadata = new HashMap<>();
94 for (int i = 0; i < schema.customMetadataLength(); i++) {
95 KeyValue kv = schema.customMetadata(i);
96 String key = kv.key();
97 String value = kv.value();
98 metadata.put(key == null ? "" : key, value == null ? "" : value);
99 }
100 return new Schema(true, Collections.unmodifiableList(fields), Collections.unmodifiableMap(metadata));
101 }
102
103 private final List<Field> fields;
104 private final Map<String, String> metadata;
105
106 public Schema(Iterable<Field> fields) {
107 this(fields, (Map<String, String>) null);
108 }
109
110 /**
111 * Constructor with metadata.
112 */
113 public Schema(Iterable<Field> fields,
114 Map<String, String> metadata) {
115 this(true,
116 Collections2.toImmutableList(fields),
117 metadata == null ? Collections.emptyMap() : Collections2.immutableMapCopy(metadata));
118 }
119
120
121 /**
122 * Constructor used for JSON deserialization.
123 */
124 @JsonCreator
125 private Schema(@JsonProperty("fields") Iterable<Field> fields,
126 @JsonProperty("metadata") List<Map<String, String>> metadata) {
127 this(fields, convertMetadata(metadata));
128 }
129
130
131 /**
132 * Private constructor to bypass automatic collection copy.
133 * @param unsafe a ignored argument. Its only purpose is to prevent using the constructor
134 * by accident because of type collisions (List vs Iterable).
135 */
136 private Schema(boolean unsafe, List<Field> fields, Map<String, String> metadata) {
137 this.fields = fields;
138 this.metadata = metadata;
139 }
140
141 static Map<String, String> convertMetadata(List<Map<String, String>> metadata) {
142 return (metadata == null) ? null : metadata.stream()
143 .map(e -> new AbstractMap.SimpleImmutableEntry<>(e.get(METADATA_KEY), e.get(METADATA_VALUE)))
144 .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
145 }
146
147 static List<Map<String, String>> convertMetadata(Map<String, String> metadata) {
148 return (metadata == null) ? null : metadata.entrySet()
149 .stream()
150 .map(Schema::convertEntryToKeyValueMap)
151 .collect(Collectors.toList());
152 }
153
154 private static Map<String, String> convertEntryToKeyValueMap(Map.Entry<String, String> entry) {
155 Map<String, String> map = new HashMap<>(2);
156 map.put(METADATA_KEY, entry.getKey());
157 map.put(METADATA_VALUE, entry.getValue());
158 return Collections.unmodifiableMap(map);
159 }
160
161 public List<Field> getFields() {
162 return fields;
163 }
164
165 @JsonIgnore
166 public Map<String, String> getCustomMetadata() {
167 return metadata;
168 }
169
170 @JsonProperty("metadata")
171 @JsonInclude(Include.NON_EMPTY)
172 List<Map<String, String>> getCustomMetadataForJson() {
173 return convertMetadata(getCustomMetadata());
174 }
175
176 /**
177 * Search for a field by name in this Schema.
178 *
179 * @param name the name of the field to return
180 * @return the corresponding field
181 * @throws IllegalArgumentException if the field was not found
182 */
183 public Field findField(String name) {
184 return findField(getFields(), name);
185 }
186
187 /**
188 * Returns the JSON string representation of this schema.
189 */
190 public String toJson() {
191 try {
192 return writer.writeValueAsString(this);
193 } catch (JsonProcessingException e) {
194 // this should not happen
195 throw new RuntimeException(e);
196 }
197 }
198
199 /**
200 * Adds this schema to the builder returning the size of the builder after adding.
201 */
202 public int getSchema(FlatBufferBuilder builder) {
203 int[] fieldOffsets = new int[fields.size()];
204 for (int i = 0; i < fields.size(); i++) {
205 fieldOffsets[i] = fields.get(i).getField(builder);
206 }
207 int fieldsOffset = org.apache.arrow.flatbuf.Schema.createFieldsVector(builder, fieldOffsets);
208 int metadataOffset = FBSerializables.writeKeyValues(builder, metadata);
209 org.apache.arrow.flatbuf.Schema.startSchema(builder);
210 org.apache.arrow.flatbuf.Schema.addFields(builder, fieldsOffset);
211 org.apache.arrow.flatbuf.Schema.addCustomMetadata(builder, metadataOffset);
212 return org.apache.arrow.flatbuf.Schema.endSchema(builder);
213 }
214
215 /**
216 * Returns the serialized flatbuffer representation of this schema.
217 */
218 public byte[] toByteArray() {
219 FlatBufferBuilder builder = new FlatBufferBuilder();
220 int schemaOffset = this.getSchema(builder);
221 builder.finish(schemaOffset);
222 ByteBuffer bb = builder.dataBuffer();
223 byte[] bytes = new byte[bb.remaining()];
224 bb.get(bytes);
225 return bytes;
226 }
227
228 @Override
229 public int hashCode() {
230 return Objects.hash(fields, metadata);
231 }
232
233 @Override
234 public boolean equals(Object obj) {
235 if (!(obj instanceof Schema)) {
236 return false;
237 }
238 return Objects.equals(this.fields, ((Schema) obj).fields) &&
239 Objects.equals(this.metadata, ((Schema) obj).metadata);
240 }
241
242 @Override
243 public String toString() {
244 String meta = metadata.isEmpty() ? "" : "(metadata: " + metadata.toString() + ")";
245 return "Schema<" + fields.stream().map(t -> t.toString()).collect(Collectors.joining(", ")) + ">" + meta;
246 }
247}