github-advanced-security[bot] commented on code in PR #15817:
URL: https://github.com/apache/druid/pull/15817#discussion_r1484042170


##########
server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java:
##########
@@ -2060,9 +2107,11 @@
 
     final Set<Interval> replaceIntervals = 
intervalToNumCorePartitions.keySet();
 
-    final Set<DataSegment> upgradedSegments = new HashSet<>();
-    for (DataSegment oldSegment : segmentsToUpgrade) {
+    final Set<DataSegmentPlus> upgradedSegments = new HashSet<>();
+    final Map<SegmentId, Pair<Long, Long>> upgradeSegmentSchemaMetadata = new 
HashMap<>();

Review Comment:
   ## Unread local variable
   
   Variable 'Map<SegmentId,Pair<Long,Long>> upgradeSegmentSchemaMetadata' is 
never read.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/6553)



##########
server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java:
##########
@@ -2154,8 +2229,27 @@
     for (List<DataSegment> partition : partitionedSegments) {
       for (DataSegment segment : partition) {
         final String now = DateTimes.nowUtc().toString();
+        String segmentId = segment.getId().toString();
+        Long schemaId = null;
+        Long numRows = null;
+
+        if (schemaPresent && 
minimalSegmentSchemas.getSegmentStatsMap().containsKey(segmentId)) {
+          MinimalSegmentSchemas.SegmentStats segmentStats = 
minimalSegmentSchemas.getSegmentStatsMap().get(segmentId);
+          if (segmentStats != null) {
+            numRows = segmentStats.getNumRows();
+            String fingerprint = segmentStats.getFingerprint();
+            if (fingerprintSchemaIdMap != null && 
fingerprintSchemaIdMap.containsKey(fingerprint)) {
+              schemaId = fingerprintSchemaIdMap.get(fingerprint);
+            }
+          }
+        } else if (upgradeSegmentMetadata.containsKey(segment.getId())) {
+          Pair<Long, Long> metadata = 
upgradeSegmentMetadata.get(segment.getId());
+          schemaId = metadata.lhs;
+          numRows = metadata.rhs;
+        }
+
         batch.add()
-             .bind("id", segment.getId().toString())
+             .bind("id", segmentId.toString())

Review Comment:
   ## Useless toString on String
   
   Redundant call to 'toString' on a String object.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/6552)



##########
server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java:
##########
@@ -769,6 +775,8 @@
         persistAll(committer),
         (Function<Object, SegmentsAndCommitMetadata>) commitMetadata -> {
           final List<DataSegment> dataSegments = new ArrayList<>();
+          final Map<String, SegmentSchemaMetadata> schemaMetadataMap = new 
HashMap<>();

Review Comment:
   ## Unread local variable
   
   Variable 'Map<String,SegmentSchemaMetadata> schemaMetadataMap' is never read.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/6554)



##########
server/src/main/java/org/apache/druid/segment/metadata/SegmentSchemaCache.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.druid.segment.metadata;
+
+import org.apache.druid.guice.LazySingleton;
+import org.apache.druid.segment.column.RowSignature;
+import org.apache.druid.segment.column.SchemaPayload;
+import org.apache.druid.segment.column.SegmentSchemaMetadata;
+import org.apache.druid.timeline.SegmentId;
+
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+
+// schema cache for published segments only
+//
+@LazySingleton
+public class SegmentSchemaCache
+{
+
+  private CountDownLatch initialized = new CountDownLatch(1);
+
+  // Mapping from segmentId to segmentStats, reference is updated on each 
database poll.
+  // edge case what happens if first this map is build from db
+  // next new schema is fetch
+  // in between
+  private volatile ConcurrentMap<SegmentId, SegmentStats> 
finalizedSegmentStats = new ConcurrentHashMap<>();
+
+  // Mapping from schemaId to schema fingerprint & schema fingerprint to 
schema payload
+  // On each database poll, fetch newly added schema since last poll.
+  // Coordinator schema cleanup duty, removes orphan schema in the same 
transaction also remove schema from here
+  // if transaction failed revert stuff by storing in temporary structure
+  private volatile ConcurrentMap<Long, SchemaPayload> finalizedSegmentSchema = 
new ConcurrentHashMap<>();
+
+  private ConcurrentMap<SegmentId, SegmentSchemaMetadata> 
realtimeSegmentSchemaMap = new ConcurrentHashMap<>();
+
+  private ConcurrentMap<SegmentId, SegmentSchemaMetadata> inTransitSMQResults 
= new ConcurrentHashMap<>();
+  private volatile ConcurrentMap<SegmentId, SegmentSchemaMetadata> 
inTransitSMQPublishedResults = new ConcurrentHashMap<>();
+
+  public void setInitialized()
+  {
+    initialized.countDown();
+  }
+
+  public void awaitInitialization() throws InterruptedException
+  {
+    initialized.await();
+  }
+
+
+  public void updateFinalizedSegmentStatsReference(ConcurrentMap<SegmentId, 
SegmentStats> segmentStatsMap)
+  {
+    this.finalizedSegmentStats = segmentStatsMap;
+  }
+
+  public void updateFinalizedSegmentSchemaReference(
+      ConcurrentMap<Long, SchemaPayload> schemaPayloadMap
+  )
+  {
+    this.finalizedSegmentSchema = schemaPayloadMap;
+  }
+
+  public void addFinalizedSegmentSchema(long schemaId, SchemaPayload 
schemaPayload)
+  {
+    finalizedSegmentSchema.put(schemaId, schemaPayload);
+  }
+
+  public void addRealtimeSegmentSchema(SegmentId segmentId, RowSignature 
rowSignature, long numRows)
+  {
+    realtimeSegmentSchemaMap.put(segmentId, new SegmentSchemaMetadata(new 
SchemaPayload(rowSignature), numRows));
+  }
+
+  public void addInTransitSMQResult(SegmentId segmentId, RowSignature 
rowSignature, long numRows)
+  {
+    inTransitSMQResults.put(segmentId, new SegmentSchemaMetadata(new 
SchemaPayload(rowSignature), numRows));
+  }
+
+  public void markInTransitSMQResultPublished(SegmentId segmentId)
+  {
+    if (!inTransitSMQResults.containsKey(segmentId)) {
+      // error how come it is not present?
+    }
+
+    inTransitSMQPublishedResults.put(segmentId, 
inTransitSMQResults.get(segmentId));
+  }
+
+  public void resetInTransitSMQResultPublishedOnDBPoll()
+  {
+    inTransitSMQPublishedResults = new ConcurrentHashMap<>();
+  }
+
+  public Optional<SegmentSchemaMetadata> getSchemaForSegment(SegmentId 
segmentId)
+  {
+    // check realtime
+    if (!finalizedSegmentStats.containsKey(segmentId)) {
+      return Optional.empty();
+    }
+
+    SegmentStats segmentStats = finalizedSegmentStats.get(segmentId);
+    if (!finalizedSegmentSchema.containsKey(segmentStats.getSchemaId())) {
+      return Optional.empty();
+    }
+
+    long schemaId = segmentStats.getSchemaId();
+
+    return Optional.of(new SegmentSchemaMetadata(
+        finalizedSegmentSchema.get(schemaId),
+        segmentStats.getNumRows()
+    ));
+  }
+
+  public boolean isSchemaCached(SegmentId segmentId)
+  {
+    return finalizedSegmentStats.containsKey(segmentId);
+  }
+
+  public boolean segmentRemoved(SegmentId segmentId)

Review Comment:
   ## Useless parameter
   
   The parameter 'segmentId' is never used.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/6556)



##########
server/src/main/java/org/apache/druid/segment/metadata/SchemaManager.java:
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.druid.segment.metadata;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import com.google.inject.Inject;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.metadata.MetadataStorageTablesConfig;
+import org.apache.druid.metadata.SQLMetadataConnector;
+import org.apache.druid.segment.column.SchemaPayload;
+import org.apache.druid.segment.column.SegmentSchemaMetadata;
+import org.apache.druid.timeline.SegmentId;
+import org.skife.jdbi.v2.Handle;
+import org.skife.jdbi.v2.PreparedBatch;
+import org.skife.jdbi.v2.TransactionCallback;
+import org.skife.jdbi.v2.Update;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+public class SchemaManager
+{
+  private static final EmittingLogger log = new 
EmittingLogger(SchemaManager.class);
+  private static final int DB_ACTION_PARTITION_SIZE = 100;
+  private final MetadataStorageTablesConfig dbTables;
+  private final ObjectMapper jsonMapper;
+  private final SQLMetadataConnector connector;
+
+  @Inject
+  public SchemaManager(
+      MetadataStorageTablesConfig dbTables,
+      ObjectMapper jsonMapper,
+      SQLMetadataConnector connector
+  )
+  {
+    this.dbTables = dbTables;
+    this.jsonMapper = jsonMapper;
+    this.connector = connector;
+  }
+
+  public void cleanUpUnreferencedSchema()
+  {
+    connector.retryTransaction(
+        (handle, transactionStatus) -> {
+          Update deleteStatement = handle.createStatement(
+              StringUtils.format("DELETE FROM %1$s WHERE schema_id NOT IN 
(SELECT schema_id FROM %2$s)",
+                                 dbTables.getSegmentSchemaTable(), 
dbTables.getSegmentsTable()
+              ));
+          return deleteStatement.execute();
+        }, 1, 3
+    );
+  }
+
+  public void 
persistSchemaAndUpdateSegmentsTable(List<SegmentSchemaMetadataPlus> 
segmentSchemas)
+  {
+    connector.retryTransaction((TransactionCallback<Void>) (handle, status) -> 
{
+      Map<String, SchemaPayload> schemaPayloadMap = new HashMap<>();
+
+      for (SegmentSchemaMetadataPlus segmentSchema : segmentSchemas) {
+        schemaPayloadMap.put(segmentSchema.getFingerprint(), 
segmentSchema.getSegmentSchemaMetadata().getSchemaPayload());
+      }
+      persistSchema(handle, schemaPayloadMap);
+      updateSegments(handle, segmentSchemas);
+      return null;
+    }, 1, 3);
+  }
+
+  public void persistSchema(Handle handle, Map<String, SchemaPayload> 
schemaPayloadMap)
+      throws JsonProcessingException
+  {
+    try {
+      // find out all the unique schema insert them and get their id
+      // update the segment table with the schema id
+
+      // Filter already existing schema
+      Set<String> existingSchemas = schemaExistBatch(handle, 
schemaPayloadMap.keySet());
+      log.info("Found already existing schema in the DB: %s", existingSchemas);
+      schemaPayloadMap.keySet().removeAll(existingSchemas);
+
+      final List<List<String>> partitionedFingerprints = Lists.partition(
+          new ArrayList<>(schemaPayloadMap.keySet()),
+          DB_ACTION_PARTITION_SIZE
+      );
+
+      String insertSql = StringUtils.format(
+          "INSERT INTO %1$s (fingerprint, created_date, payload) "
+          + "VALUES (:fingerprint, :created_date, :payload)",
+          dbTables.getSegmentSchemaTable()
+      );
+
+      // insert schemas
+      PreparedBatch schemaInsertBatch = handle.prepareBatch(insertSql);
+      for (List<String> partition : partitionedFingerprints) {
+        for (String fingerprint : partition) {
+          final String now = DateTimes.nowUtc().toString();
+          schemaInsertBatch.add()
+                           .bind("created_date", now)
+                           .bind("fingerprint", fingerprint)
+                           .bind("payload", 
jsonMapper.writeValueAsBytes(schemaPayloadMap.get(fingerprint)));
+        }
+        final int[] affectedRows = schemaInsertBatch.execute();
+        final boolean succeeded = 
Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1);
+        if (succeeded) {
+          log.info("Published schemas to DB: %s", partition);
+        } else {
+          final List<String> failedToPublish =
+              IntStream.range(0, partition.size())
+                       .filter(i -> affectedRows[i] != 1)
+                       .mapToObj(partition::get)
+                       .collect(Collectors.toList());
+          throw new ISE("Failed to publish schemas to DB: %s", 
failedToPublish);
+        }
+      }
+    }
+    catch (Exception e) {
+      log.error("Exception inserting schemas to DB: %s", schemaPayloadMap);
+      throw e;
+    }
+  }
+
+  private void logSchema(Map<String, SchemaPayload> schemaPayloadMap)

Review Comment:
   ## Useless parameter
   
   The parameter 'schemaPayloadMap' is never used.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/6557)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to