You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/02/19 10:31:58 UTC

[incubator-inlong] branch master updated: [INLONG-2554][Sort] Support array and map data structures in ORC writer (#2556)

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

dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 1cf276f  [INLONG-2554][Sort] Support array and map data structures in ORC writer (#2556)
1cf276f is described below

commit 1cf276f5a38cfc0c24061300586016ba829a317b
Author: Kevin Wen <ke...@gmail.com>
AuthorDate: Sat Feb 19 18:31:53 2022 +0800

    [INLONG-2554][Sort] Support array and map data structures in ORC writer (#2556)
---
 .../sort/flink/hive/formats/orc/RowVectorizer.java | 372 ++++++++++++++++++++-
 .../flink/hive/formats/orc/RowVectorizerTest.java  | 219 ++++++++++++
 2 files changed, 587 insertions(+), 4 deletions(-)

diff --git a/inlong-sort/sort-connectors/src/main/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizer.java b/inlong-sort/sort-connectors/src/main/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizer.java
index cff3bc4..710855c 100644
--- a/inlong-sort/sort-connectors/src/main/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizer.java
+++ b/inlong-sort/sort-connectors/src/main/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizer.java
@@ -24,7 +24,13 @@ import java.nio.charset.StandardCharsets;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Date;
+import java.util.Map;
+
+import org.apache.flink.shaded.guava18.com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
 import org.apache.flink.types.Row;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.storage.common.type.HiveDecimal;
@@ -32,7 +38,9 @@ import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
 import org.apache.orc.storage.ql.exec.vector.ColumnVector;
 import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
 import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
 import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
 import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
 import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
 
@@ -72,11 +80,11 @@ public class RowVectorizer implements Serializable {
     /**
      * Referenced from RowDataVector in flink-orc and ParquetRowWriter
      */
-    private static void setColumn(int rowId, ColumnVector column, LogicalType type, Row row, int columnId) {
+    @VisibleForTesting
+    static void setColumn(int rowId, ColumnVector column, LogicalType type, Row row, int columnId) {
         final Object field = row.getField(columnId);
         if (field == null) {
-            column.noNulls = false;
-            column.isNull[rowId] = true;
+            setNull(column, rowId);
             return;
         }
         switch (type.getTypeRoot()) {
@@ -150,9 +158,365 @@ public class RowVectorizer implements Serializable {
                 vector.set(rowId, timestamp);
                 break;
             }
-            // TODO: support MAP type
+            case ARRAY: {
+                ColumnVector innerVector =
+                        constructColumnVectorFromArray(field, ((ArrayType) type).getElementType());
+                if (innerVector == null) {
+                    setNull(column, rowId);
+                    break;
+                }
+
+                ListColumnVector tempListVector = new ListColumnVector(1, innerVector);
+                tempListVector.lengths[0] = innerVector.isNull.length;
+                ListColumnVector vector = (ListColumnVector) column;
+                vector.setElement(rowId, 0, tempListVector);
+                break;
+            }
+            case MAP: {
+                Map<?, ?> mapField = (Map<?, ?>) field;
+                int mapEleSize = mapField.size();
+                if (mapEleSize == 0) {
+                    setNull(column, rowId);
+                    break;
+                }
+
+                Object[] keys = new Object[mapEleSize];
+                Object[] values = new Object[mapEleSize];
+                int i = 0;
+                for (Map.Entry<?, ?> entry : mapField.entrySet()) {
+                    keys[i] = entry.getKey();
+                    values[i] = entry.getValue();
+                    ++i;
+                }
+
+                MapType mapType = (MapType) type;
+                ColumnVector keysVector = constructColumnVectorFromArray(keys, mapType.getKeyType());
+                ColumnVector valuesVector = constructColumnVectorFromArray(values, mapType.getValueType());
+                MapColumnVector tempMapVector = new MapColumnVector(1, keysVector, valuesVector);
+                tempMapVector.lengths[0] = mapEleSize;
+
+                MapColumnVector vector = (MapColumnVector) column;
+                vector.setElement(rowId, 0, tempMapVector);
+                break;
+            }
             default:
                 throw new UnsupportedOperationException("Unsupported type: " + type);
         }
     }
+
+    @VisibleForTesting
+    static ColumnVector constructColumnVectorFromArray(Object input, LogicalType elementType) {
+        switch (elementType.getTypeRoot()) {
+            case CHAR:
+            case VARCHAR: {
+                return createVectorFromStringArray(input);
+            }
+            case BOOLEAN: {
+                return createVectorFromBooleanArray(input);
+            }
+            case DECIMAL: {
+                DecimalType decimalType = (DecimalType) elementType;
+                return createVectorFromDecimalArray(input, decimalType.getPrecision(), decimalType.getScale());
+            }
+            case TINYINT: {
+                return createVectorFromTinyIntArray(input);
+            }
+            case SMALLINT: {
+                return createVectorFromShortArray(input);
+            }
+            case DATE: {
+                return createVectorFromDateArray(input);
+            }
+            case INTEGER: {
+                return createVectorFromIntArray(input);
+            }
+            case TIME_WITHOUT_TIME_ZONE: {
+                return createVectorFromTimeArray(input);
+            }
+            case BIGINT: {
+                return createVectorFromLongArray(input);
+            }
+            case FLOAT: {
+                return createVectorFromFloatArray(input);
+            }
+            case DOUBLE: {
+                return createVectorFromDoubleArray(input);
+            }
+            case TIMESTAMP_WITHOUT_TIME_ZONE: {
+                return createVectorFromTimestampArray(input);
+            }
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + elementType);
+        }
+    }
+
+    private static BytesColumnVector createVectorFromStringArray(Object input) {
+        Object[] inputArray = (Object[]) input;
+        int length = inputArray.length;
+        if (inputArray.length == 0) {
+            return null;
+        }
+
+        BytesColumnVector vector = new BytesColumnVector(length);
+        vector.initBuffer();
+        for (int i = 0; i < length; i++) {
+            vector.setVal(i, ((String) inputArray[i]).getBytes(StandardCharsets.UTF_8));
+        }
+
+        return vector;
+    }
+
+    private static LongColumnVector createVectorFromBooleanArray(Object input) {
+        if (input instanceof boolean[]) {
+            boolean[] inputArray = (boolean[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = inputArray[i] ? 1 : 0;
+            }
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = ((Boolean) inputArray[i]) ? 1 : 0;
+            }
+            return vector;
+        }
+    }
+
+    private static DecimalColumnVector createVectorFromDecimalArray(Object input, int precision, int scale) {
+        Object[] inputArray = (Object[]) input;
+        int length = inputArray.length;
+        if (length == 0) {
+            return null;
+        }
+
+        DecimalColumnVector vector = new DecimalColumnVector(length, precision, scale);
+        for (int i = 0; i < length; i++) {
+            vector.set(i, HiveDecimal.create((BigDecimal) inputArray[i]));
+        }
+        return vector;
+    }
+
+    private static LongColumnVector createVectorFromTinyIntArray(Object input) {
+        if (input instanceof byte[]) {
+            byte[] inputArray = (byte[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = inputArray[i];
+            }
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Byte) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static LongColumnVector createVectorFromShortArray(Object input) {
+        if (input instanceof short[]) {
+            short[] inputArray = (short[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = inputArray[i];
+            }
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Short) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static LongColumnVector createVectorFromIntArray(Object input) {
+        if (input instanceof int[]) {
+            int[] inputArray = (int[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = inputArray[i];
+            }
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Integer) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static LongColumnVector createVectorFromLongArray(Object input) {
+        if (input instanceof long[]) {
+            long[] inputArray = (long[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            System.arraycopy(inputArray, 0, vector.vector, 0, length);
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            LongColumnVector vector = new LongColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Long) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static DoubleColumnVector createVectorFromFloatArray(Object input) {
+        if (input instanceof float[]) {
+            float[] inputArray = (float[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            DoubleColumnVector vector = new DoubleColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = inputArray[i];
+            }
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            DoubleColumnVector vector = new DoubleColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Float) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static DoubleColumnVector createVectorFromDoubleArray(Object input) {
+        if (input instanceof double[]) {
+            double[] inputArray = (double[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            DoubleColumnVector vector = new DoubleColumnVector(length);
+            System.arraycopy(inputArray, 0, vector.vector, 0, length);
+            return vector;
+        } else {
+            Object[] inputArray = (Object[]) input;
+            int length = inputArray.length;
+            if (length == 0) {
+                return null;
+            }
+
+            DoubleColumnVector vector = new DoubleColumnVector(length);
+            for (int i = 0; i < length; i++) {
+                vector.vector[i] = (Double) inputArray[i];
+            }
+            return vector;
+        }
+    }
+
+    private static LongColumnVector createVectorFromDateArray(Object input) {
+        Object[] inputArray = (Object[]) input;
+        int length = inputArray.length;
+        if (length == 0) {
+            return null;
+        }
+
+        LongColumnVector vector = new LongColumnVector(length);
+        for (int i = 0; i < length; i++) {
+            vector.vector[i] = ((Date) inputArray[i]).getTime();
+        }
+        return vector;
+    }
+
+    private static LongColumnVector createVectorFromTimeArray(Object input) {
+        Object[] inputArray = (Object[]) input;
+        int length = inputArray.length;
+        if (length == 0) {
+            return null;
+        }
+
+        LongColumnVector vector = new LongColumnVector(length);
+        for (int i = 0; i < length; i++) {
+            vector.vector[i] = ((Time) inputArray[i]).getTime();
+        }
+        return vector;
+    }
+
+    private static TimestampColumnVector createVectorFromTimestampArray(Object input) {
+        Object[] inputArray = (Object[]) input;
+        int length = inputArray.length;
+        if (length == 0) {
+            return null;
+        }
+
+        TimestampColumnVector vector = new TimestampColumnVector(length);
+        for (int i = 0; i < length; i++) {
+            vector.set(i, (Timestamp) inputArray[i]);
+        }
+        return vector;
+    }
+
+    private static void setNull(ColumnVector column, int rowId) {
+        column.noNulls = false;
+        column.isNull[rowId] = true;
+    }
+
 }
diff --git a/inlong-sort/sort-connectors/src/test/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizerTest.java b/inlong-sort/sort-connectors/src/test/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizerTest.java
new file mode 100644
index 0000000..278fdf4
--- /dev/null
+++ b/inlong-sort/sort-connectors/src/test/java/org/apache/inlong/sort/flink/hive/formats/orc/RowVectorizerTest.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.sort.flink.hive.formats.orc;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.inlong.sort.flink.hive.formats.orc.RowVectorizer.constructColumnVectorFromArray;
+import static org.apache.inlong.sort.flink.hive.formats.orc.RowVectorizer.setColumn;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RowVectorizerTest {
+
+    @ClassRule
+    public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    @Test
+    public void testConstructColumnVectorFromArray() {
+        Integer[] testInput = new Integer[] {1, 2, 3, 4, 5, 6};
+        ColumnVector outputVector = constructColumnVectorFromArray(testInput, new IntType());
+        assertTrue(outputVector instanceof LongColumnVector);
+
+        LongColumnVector longColumnVector = (LongColumnVector) outputVector;
+        assertArrayEquals(new long[] {1, 2, 3, 4, 5, 6}, longColumnVector.vector);
+    }
+
+    @Test
+    public void testSetColumnForArray() {
+        LongColumnVector innerVector = new LongColumnVector(3);
+        ListColumnVector testVector = new ListColumnVector(1, innerVector);
+        Row testRow = Row.of("String", new Integer[] {1, 2, 3});
+        setColumn(0, testVector, new ArrayType(new IntType()), testRow, 1);
+        assertEquals(3, testVector.lengths[0]);
+
+        StringBuilder stringBuilder = new StringBuilder();
+        testVector.stringifyValue(stringBuilder, 0);
+        assertEquals("[1, 2, 3]", stringBuilder.toString());
+    }
+
+    @Test
+    public void testSetColumnForMap() {
+        Map<Integer, Double> testMap = new HashMap<>();
+        testMap.put(1, 10.0);
+        testMap.put(2, 20.0);
+        testMap.put(3, 30.0);
+        Row testRow = Row.of(testMap);
+        LongColumnVector keyVector = new LongColumnVector(3);
+        DoubleColumnVector valueVector = new DoubleColumnVector(3);
+        MapColumnVector testVector = new MapColumnVector(1, keyVector, valueVector);
+        setColumn(0, testVector, new MapType(new IntType(), new DoubleType()), testRow, 0);
+        assertEquals(3, testVector.lengths[0]);
+
+        StringBuilder stringBuilder = new StringBuilder();
+        testVector.stringifyValue(stringBuilder, 0);
+        assertEquals(
+                "[{\"key\": 1, \"value\": 10.0}, {\"key\": 2, \"value\": 20.0}, {\"key\": 3, \"value\": 30.0}]",
+                stringBuilder.toString()
+        );
+    }
+
+    @Test
+    public void testWriteFile() throws IOException {
+        String struct = "struct<test_array:array<int>,test_map:map<string,double>>";
+        String testFilePath = temporaryFolder.newFolder().toString() + "/test.orc";
+
+        TypeDescription schema = TypeDescription.fromString(struct);
+        Configuration hadoopConf = new Configuration();
+        try (Writer writer = OrcFile.createWriter(new Path(testFilePath),
+                OrcFile.writerOptions(hadoopConf).setSchema(schema))) {
+            VectorizedRowBatch batch = schema.createRowBatch();
+            List<int[]> intList = prepareListDataForOrcFileTest();
+            List<Map<String, Double>> mapList = prepareMapDataForOrcFileTest();
+            for (int i = 0; i < 5; i++) {
+                batch.size++;
+                setColumn(i, batch.cols[0], new ArrayType(new IntType()), Row.of("string", intList.get(i)), 1);
+                setColumn(i, batch.cols[1], new MapType(new CharType(), new DoubleType()),
+                        Row.of("String", mapList.get(i)), 1);
+            }
+
+            writer.addRowBatch(batch);
+        }
+
+        List<String> resultForList = new ArrayList<>();
+        List<String> resultForMap = new ArrayList<>();
+        try (Reader reader = OrcFile.createReader(new Path(testFilePath), OrcFile.readerOptions(hadoopConf))) {
+            try (RecordReader records = reader.rows(reader.options())) {
+                VectorizedRowBatch batch = reader.getSchema().createRowBatch(1024);
+                ListColumnVector listColumnVector = (ListColumnVector) batch.cols[0];
+                MapColumnVector mapColumnVector = (MapColumnVector) batch.cols[1];
+
+                while (records.nextBatch(batch)) {
+                    for (int rowNum = 0; rowNum < batch.size; rowNum++) {
+                        StringBuilder stringBuilderForList = new StringBuilder();
+                        listColumnVector.stringifyValue(stringBuilderForList, rowNum);
+                        resultForList.add(stringBuilderForList.toString());
+
+                        StringBuilder stringBuilderForMap = new StringBuilder();
+                        mapColumnVector.stringifyValue(stringBuilderForMap, rowNum);
+                        resultForMap.add(stringBuilderForMap.toString());
+                    }
+                }
+            }
+        }
+
+        List<String> expectedForList = new ArrayList<>();
+        expectedForList.add("[1, 2, 3]");
+        expectedForList.add("null");
+        expectedForList.add("[6, 5, 4]");
+        expectedForList.add("null");
+        expectedForList.add("[7, 8, 9, 10]");
+
+        List<String> expectedForMap = new ArrayList<>();
+        expectedForMap.add("[{\"key\": \"mary\", \"value\": 100.0}, {\"key\": \"lisa\", \"value\": 95.0}, {\"key\": "
+                + "\"anna\", \"value\": 99.0}]");
+        expectedForMap.add("null");
+        expectedForMap.add("null");
+        expectedForMap.add("[{\"key\": \"mary4\", \"value\": 100.4}, {\"key\": \"lisa4\", \"value\": 95.4}, {\"key\":"
+                + " \"anna4\", \"value\": 99.4}]");
+        expectedForMap.add("[{\"key\": \"luna5\", \"value\": 98.5}, {\"key\": \"mary5\", \"value\": 100.5}, "
+                + "{\"key\": \"lili5\", \"value\": 97.5}, {\"key\": \"lisa5\", \"value\": 95.5}, {\"key\": \"anna5\","
+                + " \"value\": 99.5}]");
+
+        assertEquals(expectedForList, resultForList);
+        assertEquals(expectedForMap, resultForMap);
+    }
+
+    private List<int[]> prepareListDataForOrcFileTest() {
+        List<int[]> data = new ArrayList<>();
+        int[] data1 = new int[] {1, 2, 3};
+        data.add(data1);
+
+        int[] data2 = new int[] {};
+        data.add(data2);
+
+        int[] data3 = new int[] {6, 5, 4};
+        data.add(data3);
+
+        data.add(null);
+
+        int[] data5 = new int[] {7, 8, 9, 10};
+        data.add(data5);
+
+        return data;
+    }
+
+    private List<Map<String, Double>> prepareMapDataForOrcFileTest() {
+
+        Map<String, Double> data1 = new HashMap<>();
+        data1.put("anna", 99.0);
+        data1.put("lisa", 95.0);
+        data1.put("mary", 100.0);
+        List<Map<String, Double>> data = new ArrayList<>();
+        data.add(data1);
+
+        Map<String, Double> data2 = new HashMap<>();
+        data.add(data2);
+
+        data.add(null);
+
+        Map<String, Double> data4 = new HashMap<>();
+        data4.put("anna4", 99.4);
+        data4.put("lisa4", 95.4);
+        data4.put("mary4", 100.4);
+        data.add(data4);
+
+        Map<String, Double> data5 = new HashMap<>();
+        data5.put("anna5", 99.5);
+        data5.put("lisa5", 95.5);
+        data5.put("mary5", 100.5);
+        data5.put("luna5", 98.5);
+        data5.put("lili5", 97.5);
+        data.add(data5);
+
+        return data;
+    }
+}