This is an automated email from the ASF dual-hosted git repository.

jlli pushed a commit to branch add-virtual-key-to-segment-metadata
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit ac6e64709416f0422d90502c691a3d9e7b6c248f
Author: Jack Li(Analytics Engineering) <j...@jlli-mn1.linkedin.biz>
AuthorDate: Thu Aug 6 20:51:51 2020 -0700

    Identify whether the values are converted from map and emit server metric
---
 .../apache/pinot/common/metrics/ServerGauge.java   |  3 +-
 .../recordtransformer/DataTypeTransformer.java     | 22 ++++---
 .../creator/impl/SegmentColumnarIndexCreator.java  |  4 ++
 .../core/segment/creator/impl/V1Constants.java     |  1 +
 .../index/metadata/SegmentMetadataImpl.java        | 28 ++++-----
 .../recordtransformer/DataTypeTransformerTest.java | 70 +++++++++++-----------
 .../starter/helix/SegmentFetcherAndLoader.java     |  9 ++-
 .../apache/pinot/spi/data/readers/GenericRow.java  |  6 ++
 8 files changed, 83 insertions(+), 60 deletions(-)

diff --git 
a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java 
b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java
index d739c0f..de43278 100644
--- 
a/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java
+++ 
b/pinot-common/src/main/java/org/apache/pinot/common/metrics/ServerGauge.java
@@ -40,7 +40,8 @@ public enum ServerGauge implements AbstractMetrics.Gauge {
   LAST_REALTIME_SEGMENT_COMPLETION_DURATION_SECONDS("seconds", false),
   REALTIME_OFFHEAP_MEMORY_USED("bytes", false),
   REALTIME_SEGMENT_NUM_PARTITIONS("realtimeSegmentNumPartitions", false),
-  LLC_SIMULTANEOUS_SEGMENT_BUILDS("llcSimultaneousSegmentBuilds", true);
+  LLC_SIMULTANEOUS_SEGMENT_BUILDS("llcSimultaneousSegmentBuilds", true),
+  SEGMENT_VALUES_CONVERTED_FROM_MAP_TO_ARRAY("segments", false);
 
   private final String gaugeName;
   private final String unit;
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformer.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformer.java
index ccdda33..bc2d777 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformer.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformer.java
@@ -84,7 +84,7 @@ public class DataTypeTransformer implements RecordTransformer 
{
         continue;
       }
       PinotDataType dest = entry.getValue();
-      value = standardize(column, value, dest.isSingleValue());
+      value = standardize(record, column, value, dest.isSingleValue());
       // NOTE: The standardized value could be null for empty 
Collection/Map/Object[].
       if (value == null) {
         record.putValue(column, null);
@@ -126,15 +126,19 @@ public class DataTypeTransformer implements 
RecordTransformer {
    */
   @VisibleForTesting
   @Nullable
-  static Object standardize(String column, @Nullable Object value, boolean 
isSingleValue) {
+  static Object standardize(GenericRow record, String column, @Nullable Object 
value, boolean isSingleValue) {
     if (value == null) {
       return null;
     }
     if (value instanceof Collection) {
-      return standardizeCollection(column, (Collection) value, isSingleValue);
+      return standardizeCollection(record, column, (Collection) value, 
isSingleValue);
     }
     if (value instanceof Map) {
-      return standardizeCollection(column, ((Map) value).values(), 
isSingleValue);
+      Collection values = ((Map) value).values();
+      if (values.size() == 1) {
+        record.putValue(GenericRow.CONVERT_MAP_VALUE_TO_ARRAY_VALUE_KEY, 
Boolean.TRUE.toString());
+      }
+      return standardizeCollection(record, column, values, isSingleValue);
     }
     if (value instanceof Object[]) {
       Object[] values = (Object[]) value;
@@ -143,11 +147,11 @@ public class DataTypeTransformer implements 
RecordTransformer {
         return null;
       }
       if (numValues == 1) {
-        return standardize(column, values[0], isSingleValue);
+        return standardize(record, column, values[0], isSingleValue);
       }
       List<Object> standardizedValues = new ArrayList<>(numValues);
       for (Object singleValue : values) {
-        Object standardizedValue = standardize(column, singleValue, true);
+        Object standardizedValue = standardize(record, column, singleValue, 
true);
         if (standardizedValue != null) {
           standardizedValues.add(standardizedValue);
         }
@@ -166,17 +170,17 @@ public class DataTypeTransformer implements 
RecordTransformer {
     return value;
   }
 
-  private static Object standardizeCollection(String column, Collection 
collection, boolean isSingleValue) {
+  private static Object standardizeCollection(GenericRow record, String 
column, Collection collection, boolean isSingleValue) {
     int numValues = collection.size();
     if (numValues == 0) {
       return null;
     }
     if (numValues == 1) {
-      return standardize(column, collection.iterator().next(), isSingleValue);
+      return standardize(record, column, collection.iterator().next(), 
isSingleValue);
     }
     List<Object> standardizedValues = new ArrayList<>(numValues);
     for (Object singleValue : collection) {
-      Object standardizedValue = standardize(column, singleValue, true);
+      Object standardizedValue = standardize(record, column, singleValue, 
true);
       if (standardizedValue != null) {
         standardizedValues.add(standardizedValue);
       }
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
index 34131ab..21a37a6 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
@@ -91,6 +91,7 @@ public class SegmentColumnarIndexCreator implements 
SegmentCreator {
   private int totalDocs;
   private int docIdCounter;
   private boolean _nullHandlingEnabled;
+  private boolean _convertMapToArray;
 
   private final Set<String> _textIndexColumns = new HashSet<>();
 
@@ -217,6 +218,7 @@ public class SegmentColumnarIndexCreator implements 
SegmentCreator {
         // Initialize Null value vector map
         _nullValueVectorCreatorMap.put(columnName, new 
NullValueVectorCreator(_indexDir, columnName));
       }
+      _convertMapToArray = false;
     }
   }
 
@@ -304,6 +306,7 @@ public class SegmentColumnarIndexCreator implements 
SegmentCreator {
 
   @Override
   public void indexRow(GenericRow row) {
+    _convertMapToArray |= Boolean.parseBoolean((String) 
row.getValue(GenericRow.CONVERT_MAP_VALUE_TO_ARRAY_VALUE_KEY));
     for (Map.Entry<String, ForwardIndexCreator> entry : 
_forwardIndexCreatorMap.entrySet()) {
       String columnName = entry.getKey();
       ForwardIndexCreator forwardIndexCreator = entry.getValue();
@@ -414,6 +417,7 @@ public class SegmentColumnarIndexCreator implements 
SegmentCreator {
     String timeColumnName = config.getTimeColumnName();
     properties.setProperty(TIME_COLUMN_NAME, timeColumnName);
     properties.setProperty(SEGMENT_TOTAL_DOCS, String.valueOf(totalDocs));
+    properties.setProperty(SEGMENT_CONVERT_MAP_VALUE_TO_ARRAY_VALUE, 
_convertMapToArray);
 
     // Write time related metadata (start time, end time, time unit)
     if (timeColumnName != null) {
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/V1Constants.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/V1Constants.java
index 0498f8c..56cf320 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/V1Constants.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/V1Constants.java
@@ -62,6 +62,7 @@ public class V1Constants {
       public static final String DATETIME_COLUMNS = 
"segment.datetime.column.names";
       public static final String SEGMENT_TOTAL_DOCS = "segment.total.docs";
       public static final String SEGMENT_PADDING_CHARACTER = 
"segment.padding.character";
+      public static final String SEGMENT_CONVERT_MAP_VALUE_TO_ARRAY_VALUE = 
"segment.convert.map.value.to.array.value";
     }
 
     public static class Column {
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/metadata/SegmentMetadataImpl.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/metadata/SegmentMetadataImpl.java
index 1245034..a127652 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/segment/index/metadata/SegmentMetadataImpl.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/segment/index/metadata/SegmentMetadataImpl.java
@@ -18,20 +18,6 @@
  */
 package org.apache.pinot.core.segment.index.metadata;
 
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.DATETIME_COLUMNS;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.DIMENSIONS;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.METRICS;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_CREATOR_VERSION;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_END_TIME;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_NAME;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_PADDING_CHARACTER;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_START_TIME;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_TOTAL_DOCS;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_VERSION;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.TABLE_NAME;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.TIME_COLUMN_NAME;
-import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.TIME_UNIT;
-
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -75,6 +61,8 @@ import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
 
+import static 
org.apache.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.*;
+
 
 public class SegmentMetadataImpl implements SegmentMetadata {
   private static final Logger LOGGER = 
LoggerFactory.getLogger(SegmentMetadataImpl.class);
@@ -104,6 +92,7 @@ public class SegmentMetadataImpl implements SegmentMetadata {
   private int _totalDocs;
   private long _segmentStartTime;
   private long _segmentEndTime;
+  private boolean _valuesConvertedFromMapToArray;
 
   /**
    * For segments on disk.
@@ -165,7 +154,7 @@ public class SegmentMetadataImpl implements SegmentMetadata 
{
   public static PropertiesConfiguration getPropertiesConfiguration(File 
indexDir) {
     File metadataFile = SegmentDirectoryPaths.findMetadataFile(indexDir);
     Preconditions.checkNotNull(metadataFile, "Cannot find segment metadata 
file under directory: %s", indexDir);
-    
+
     return CommonsConfigurationUtils.fromFile(metadataFile);
   }
 
@@ -263,6 +252,11 @@ public class SegmentMetadataImpl implements 
SegmentMetadata {
             
segmentMetadataPropertiesConfiguration.subset(StarTreeV2Constants.MetadataKey.getStarTreePrefix(i))));
       }
     }
+
+    if 
(segmentMetadataPropertiesConfiguration.containsKey(SEGMENT_CONVERT_MAP_VALUE_TO_ARRAY_VALUE))
 {
+      _valuesConvertedFromMapToArray =
+          
segmentMetadataPropertiesConfiguration.getBoolean(SEGMENT_CONVERT_MAP_VALUE_TO_ARRAY_VALUE);
+    }
   }
 
   /**
@@ -489,6 +483,10 @@ public class SegmentMetadataImpl implements 
SegmentMetadata {
     return _paddingCharacter;
   }
 
+  public boolean areValuesConvertedFromMapToArray() {
+    return _valuesConvertedFromMapToArray;
+  }
+
   /**
    * Converts segment metadata to json
    * @param columnFilter list only  the columns in the set. Lists all the 
columns if
diff --git 
a/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformerTest.java
 
b/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformerTest.java
index a60c460..be4ad8d 100644
--- 
a/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformerTest.java
+++ 
b/pinot-core/src/test/java/org/apache/pinot/core/data/recordtransformer/DataTypeTransformerTest.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.pinot.spi.data.readers.GenericRow;
 import org.testng.annotations.Test;
 
 import static org.testng.Assert.assertEquals;
@@ -39,17 +40,18 @@ public class DataTypeTransformerTest {
     /**
      * Tests for Map
      */
+    GenericRow record = new GenericRow();
 
     // Empty Map
     Map<String, Object> map = Collections.emptyMap();
-    assertNull(DataTypeTransformer.standardize(COLUMN, map, true));
-    assertNull(DataTypeTransformer.standardize(COLUMN, map, false));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, map, true));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, map, false));
 
     // Map with single entry
     String expectedValue = "testValue";
     map = Collections.singletonMap("testKey", expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, true), 
expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, false), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, true), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, false), 
expectedValue);
 
     // Map with multiple entries
     Object[] expectedValues = new Object[]{"testValue1", "testValue2"};
@@ -58,12 +60,12 @@ public class DataTypeTransformerTest {
     map.put("testKey2", "testValue2");
     try {
       // Should fail because Map with multiple entries cannot be standardized 
as single value
-      DataTypeTransformer.standardize(COLUMN, map, true);
+      DataTypeTransformer.standardize(record, COLUMN, map, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(COLUMN, 
map, false), expectedValues);
+    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(record, 
COLUMN, map, false), expectedValues);
 
     /**
      * Tests for List
@@ -71,24 +73,24 @@ public class DataTypeTransformerTest {
 
     // Empty List
     List<Object> list = Collections.emptyList();
-    assertNull(DataTypeTransformer.standardize(COLUMN, list, true));
-    assertNull(DataTypeTransformer.standardize(COLUMN, list, false));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, list, true));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, list, false));
 
     // List with single entry
     list = Collections.singletonList(expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, list, true), 
expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, list, false), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, list, true), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, list, false), 
expectedValue);
 
     // List with multiple entries
     list = Arrays.asList(expectedValues);
     try {
       // Should fail because List with multiple entries cannot be standardized 
as single value
-      DataTypeTransformer.standardize(COLUMN, list, true);
+      DataTypeTransformer.standardize(record, COLUMN, list, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEquals((Object[]) DataTypeTransformer.standardize(COLUMN, list, 
false), expectedValues);
+    assertEquals((Object[]) DataTypeTransformer.standardize(record, COLUMN, 
list, false), expectedValues);
 
     /**
      * Tests for Object[]
@@ -96,24 +98,24 @@ public class DataTypeTransformerTest {
 
     // Empty Object[]
     Object[] values = new Object[0];
-    assertNull(DataTypeTransformer.standardize(COLUMN, values, true));
-    assertNull(DataTypeTransformer.standardize(COLUMN, values, false));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, values, true));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, values, false));
 
     // Object[] with single entry
     values = new Object[]{expectedValue};
-    assertEquals(DataTypeTransformer.standardize(COLUMN, values, true), 
expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, values, false), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, values, 
true), expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, values, 
false), expectedValue);
 
     // Object[] with multiple entries
     values = new Object[]{"testValue1", "testValue2"};
     try {
       // Should fail because Object[] with multiple entries cannot be 
standardized as single value
-      DataTypeTransformer.standardize(COLUMN, values, true);
+      DataTypeTransformer.standardize(record, COLUMN, values, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEquals((Object[]) DataTypeTransformer.standardize(COLUMN, values, 
false), expectedValues);
+    assertEquals((Object[]) DataTypeTransformer.standardize(record, COLUMN, 
values, false), expectedValues);
 
     /**
      * Tests for nested Map/List/Object[]
@@ -121,32 +123,32 @@ public class DataTypeTransformerTest {
 
     // Map with empty List
     map = Collections.singletonMap("testKey", Collections.emptyList());
-    assertNull(DataTypeTransformer.standardize(COLUMN, map, true));
-    assertNull(DataTypeTransformer.standardize(COLUMN, map, false));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, map, true));
+    assertNull(DataTypeTransformer.standardize(record, COLUMN, map, false));
 
     // Map with single-entry List
     map = Collections.singletonMap("testKey", 
Collections.singletonList(expectedValue));
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, true), 
expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, false), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, true), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, false), 
expectedValue);
 
     // Map with one empty Map and one single-entry Map
     map = new HashMap<>();
     map.put("testKey1", Collections.emptyMap());
     map.put("testKey2", Collections.singletonMap("testKey", expectedValue));
     // Can be standardized into single value because empty Map should be 
ignored
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, true), 
expectedValue);
-    assertEquals(DataTypeTransformer.standardize(COLUMN, map, false), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, true), 
expectedValue);
+    assertEquals(DataTypeTransformer.standardize(record, COLUMN, map, false), 
expectedValue);
 
     // Map with multi-entries List
     map = Collections.singletonMap("testKey", Arrays.asList(expectedValues));
     try {
       // Should fail because Map with multiple entries cannot be standardized 
as single value
-      DataTypeTransformer.standardize(COLUMN, map, true);
+      DataTypeTransformer.standardize(record, COLUMN, map, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(COLUMN, 
map, false), expectedValues);
+    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(record, 
COLUMN, map, false), expectedValues);
 
     // Map with one empty Map, one single-entry List and one single-entry 
Object[]
     map = new HashMap<>();
@@ -155,12 +157,12 @@ public class DataTypeTransformerTest {
     map.put("testKey3", new Object[]{"testValue2"});
     try {
       // Should fail because Map with multiple entries cannot be standardized 
as single value
-      DataTypeTransformer.standardize(COLUMN, map, true);
+      DataTypeTransformer.standardize(record, COLUMN, map, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(COLUMN, 
map, false), expectedValues);
+    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(record, 
COLUMN, map, false), expectedValues);
 
     // List with two single-entry Maps and one empty Map
     list = Arrays
@@ -168,35 +170,35 @@ public class DataTypeTransformerTest {
             Collections.emptyMap());
     try {
       // Should fail because List with multiple entries cannot be standardized 
as single value
-      DataTypeTransformer.standardize(COLUMN, list, true);
+      DataTypeTransformer.standardize(record, COLUMN, list, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEquals((Object[]) DataTypeTransformer.standardize(COLUMN, list, 
false), expectedValues);
+    assertEquals((Object[]) DataTypeTransformer.standardize(record, COLUMN, 
list, false), expectedValues);
 
     // Object[] with two single-entry Maps
     values = new Object[]{Collections.singletonMap("testKey", "testValue1"), 
Collections.singletonMap("testKey",
         "testValue2")};
     try {
       // Should fail because Object[] with multiple entries cannot be 
standardized as single value
-      DataTypeTransformer.standardize(COLUMN, values, true);
+      DataTypeTransformer.standardize(record, COLUMN, values, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(COLUMN, 
values, false), expectedValues);
+    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(record, 
COLUMN, values, false), expectedValues);
 
     // Object[] with one empty Object[], one multi-entries List of nested 
Map/List/Object[]
     values = new Object[]{new Object[0], Collections.singletonList(
         Collections.singletonMap("testKey", "testValue1")), 
Collections.singletonMap("testKey",
         Arrays.asList(new Object[]{"testValue2"}, Collections.emptyMap()))};
     try {
-      DataTypeTransformer.standardize(COLUMN, values, true);
+      DataTypeTransformer.standardize(record, COLUMN, values, true);
       fail();
     } catch (Exception e) {
       // Expected
     }
-    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(COLUMN, 
values, false), expectedValues);
+    assertEqualsNoOrder((Object[]) DataTypeTransformer.standardize(record, 
COLUMN, values, false), expectedValues);
   }
 }
diff --git 
a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentFetcherAndLoader.java
 
b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentFetcherAndLoader.java
index 65b6003..3710fd0 100644
--- 
a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentFetcherAndLoader.java
+++ 
b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/SegmentFetcherAndLoader.java
@@ -28,6 +28,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.Utils;
 import org.apache.pinot.common.metadata.ZKMetadataProvider;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
+import org.apache.pinot.common.metrics.ServerGauge;
 import org.apache.pinot.common.metrics.ServerMeter;
 import org.apache.pinot.common.metrics.ServerMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
@@ -154,10 +155,16 @@ public class SegmentFetcherAndLoader {
 
         // Retry will be done here.
         String localSegmentDir = downloadSegmentToLocal(uri, crypter, 
tableNameWithType, segmentName);
-        SegmentMetadata segmentMetadata = new SegmentMetadataImpl(new 
File(localSegmentDir));
+        SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(new 
File(localSegmentDir));
         _instanceDataManager.addOfflineSegment(tableNameWithType, segmentName, 
new File(localSegmentDir));
         LOGGER.info("Downloaded segment {} of table {} crc {} from 
controller", segmentName, tableNameWithType,
             segmentMetadata.getCrc());
+
+        // Emit server metric if the generated values of the segment are 
converted from map to array.
+        if (segmentMetadata.areValuesConvertedFromMapToArray()) {
+          _serverMetrics
+              .setValueOfTableGauge(tableNameWithType, 
ServerGauge.SEGMENT_VALUES_CONVERTED_FROM_MAP_TO_ARRAY, 1L);
+        }
       } else {
         LOGGER.info("Got already loaded segment {} of table {} crc {} again, 
will do nothing.", segmentName,
             tableNameWithType, localSegmentMetadata.getCrc());
diff --git 
a/pinot-spi/src/main/java/org/apache/pinot/spi/data/readers/GenericRow.java 
b/pinot-spi/src/main/java/org/apache/pinot/spi/data/readers/GenericRow.java
index 5c45d6b..8bd0e7c 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/readers/GenericRow.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/readers/GenericRow.java
@@ -61,6 +61,12 @@ public class GenericRow {
    */
   public static final String SKIP_RECORD_KEY = "$SKIP_RECORD_KEY$";
 
+  /**
+   * This key is used to identify whether a map with only 1 entry is converted 
to a value of a array for multi-value column.
+   * This is necessary for us to identify whether there is any existing use 
case that is leveraging this way to fetch values.
+   */
+  public static final String CONVERT_MAP_VALUE_TO_ARRAY_VALUE_KEY = 
"$CONVERT_MAP_VALUE_TO_ARRAY_VALUE_KEY$";
+
   private final Map<String, Object> _fieldToValueMap = new HashMap<>();
   private final Set<String> _nullValueFields = new HashSet<>();
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to