/**
   * Helper method to perform idempotent operation to refresh all watches (related to real-time
   * segments): - Data change listener for all existing real-time tables. - Child creation listener
   * for all existing real-time tables. - Data change listener for all existing real-time segments
   *
   * @param path
   */
  private void refreshWatchers(String path) {
    LOGGER.info("Received change notification for path: {}", path);
    List<Stat> stats = new ArrayList<>();
    List<ZNRecord> tableConfigs =
        _pinotHelixResourceManager.getPropertyStore().getChildren(TABLE_CONFIG, stats, 0);

    if (tableConfigs == null) {
      return;
    }

    for (ZNRecord tableConfig : tableConfigs) {
      try {
        AbstractTableConfig abstractTableConfig = AbstractTableConfig.fromZnRecord(tableConfig);
        if (abstractTableConfig.isRealTime()) {
          String realtimeTable = abstractTableConfig.getTableName();
          String realtimeSegmentsPathForTable =
              _propertyStorePath + SEGMENTS_PATH + "/" + realtimeTable;

          LOGGER.info("Setting data/child changes watch for real-time table '{}'", realtimeTable);
          _zkClient.subscribeDataChanges(realtimeSegmentsPathForTable, this);
          _zkClient.subscribeChildChanges(realtimeSegmentsPathForTable, this);

          List<String> childNames =
              _pinotHelixResourceManager
                  .getPropertyStore()
                  .getChildNames(SEGMENTS_PATH + "/" + realtimeTable, 0);

          if (childNames != null && !childNames.isEmpty()) {
            for (String segmentName : childNames) {
              String segmentPath = realtimeSegmentsPathForTable + "/" + segmentName;
              LOGGER.info("Setting data change watch for real-time segment: {}", segmentPath);
              _zkClient.subscribeDataChanges(segmentPath, this);
            }
          }
        }
      } catch (JSONException e) {
        LOGGER.error("Caught exception while reading table config", e);
      } catch (IOException e) {
        LOGGER.error(
            "Caught exception while setting change listeners for realtime tables/segments", e);
      }
    }
  }
  private synchronized void assignRealtimeSegmentsToServerInstancesIfNecessary()
      throws JSONException, IOException {
    // Fetch current ideal state snapshot
    Map<String, IdealState> idealStateMap = new HashMap<String, IdealState>();

    for (String resource : _pinotHelixResourceManager.getAllRealtimeTables()) {
      idealStateMap.put(
          resource,
          _pinotHelixResourceManager
              .getHelixAdmin()
              .getResourceIdealState(_pinotHelixResourceManager.getHelixClusterName(), resource));
    }

    List<String> listOfSegmentsToAdd = new ArrayList<String>();

    for (String resource : idealStateMap.keySet()) {
      IdealState state = idealStateMap.get(resource);

      // Are there any partitions?
      if (state.getPartitionSet().size() == 0) {
        // No, this is a brand new ideal state, so we will add one new segment to every partition
        // and replica
        List<String> instancesInResource = new ArrayList<String>();
        try {
          instancesInResource.addAll(
              _pinotHelixResourceManager.getServerInstancesForTable(resource, TableType.REALTIME));
        } catch (Exception e) {
          LOGGER.error("Caught exception while fetching instances for resource {}", resource, e);
        }

        // Assign a new segment to all server instances
        for (String instanceId : instancesInResource) {
          InstanceZKMetadata instanceZKMetadata =
              _pinotHelixResourceManager.getInstanceZKMetadata(instanceId);
          String groupId = instanceZKMetadata.getGroupId(resource);
          String partitionId = instanceZKMetadata.getPartition(resource);
          listOfSegmentsToAdd.add(
              SegmentNameBuilder.Realtime.build(
                  resource,
                  instanceId,
                  groupId,
                  partitionId,
                  String.valueOf(System.currentTimeMillis())));
        }
      } else {
        // Add all server instances to the list of instances for which to assign a realtime segment
        Set<String> instancesToAssignRealtimeSegment = new HashSet<String>();
        instancesToAssignRealtimeSegment.addAll(
            _pinotHelixResourceManager.getServerInstancesForTable(resource, TableType.REALTIME));

        // Remove server instances that are currently processing a segment
        for (String partition : state.getPartitionSet()) {
          RealtimeSegmentZKMetadata realtimeSegmentZKMetadata =
              ZKMetadataProvider.getRealtimeSegmentZKMetadata(
                  _pinotHelixResourceManager.getPropertyStore(),
                  SegmentNameBuilder.Realtime.extractTableName(partition),
                  partition);
          if (realtimeSegmentZKMetadata.getStatus() == Status.IN_PROGRESS) {
            String instanceName = SegmentNameBuilder.Realtime.extractInstanceName(partition);
            instancesToAssignRealtimeSegment.remove(instanceName);
          }
        }

        // Assign a new segment to the server instances not currently processing this segment
        for (String instanceId : instancesToAssignRealtimeSegment) {
          InstanceZKMetadata instanceZKMetadata =
              _pinotHelixResourceManager.getInstanceZKMetadata(instanceId);
          String groupId = instanceZKMetadata.getGroupId(resource);
          String partitionId = instanceZKMetadata.getPartition(resource);
          listOfSegmentsToAdd.add(
              SegmentNameBuilder.Realtime.build(
                  resource,
                  instanceId,
                  groupId,
                  partitionId,
                  String.valueOf(System.currentTimeMillis())));
        }
      }
    }

    LOGGER.info(
        "Computed list of new segments to add : " + Arrays.toString(listOfSegmentsToAdd.toArray()));

    // Add the new segments to the server instances
    for (String segmentId : listOfSegmentsToAdd) {
      String resourceName = SegmentNameBuilder.Realtime.extractTableName(segmentId);
      String instanceName = SegmentNameBuilder.Realtime.extractInstanceName(segmentId);

      // Does the ideal state already contain this segment?
      if (!idealStateMap.get(resourceName).getPartitionSet().contains(segmentId)) {
        // No, add it
        // Create the realtime segment metadata
        RealtimeSegmentZKMetadata realtimeSegmentMetadataToAdd = new RealtimeSegmentZKMetadata();
        realtimeSegmentMetadataToAdd.setTableName(
            TableNameBuilder.extractRawTableName(resourceName));
        realtimeSegmentMetadataToAdd.setSegmentType(SegmentType.REALTIME);
        realtimeSegmentMetadataToAdd.setStatus(Status.IN_PROGRESS);
        realtimeSegmentMetadataToAdd.setSegmentName(segmentId);

        // Add the new metadata to the property store
        ZKMetadataProvider.setRealtimeSegmentZKMetadata(
            _pinotHelixResourceManager.getPropertyStore(), realtimeSegmentMetadataToAdd);

        // Update the ideal state to add the new realtime segment
        HelixHelper.updateIdealState(
            _pinotHelixResourceManager.getHelixZkManager(),
            resourceName,
            idealState ->
                PinotTableIdealStateBuilder.addNewRealtimeSegmentToIdealState(
                    segmentId, idealState, instanceName),
            RetryPolicies.exponentialBackoffRetryPolicy(5, 500L, 2.0f));
      }
    }
  }
 public void stop() {
   LOGGER.info("Stopping realtime segments manager, stopping property store.");
   _pinotHelixResourceManager.getPropertyStore().stop();
 }