swaroopak commented on a change in pull request #517: PHOENIX-5211 Consistent 
Immutable Global Indexes for Non-Transactiona…
URL: https://github.com/apache/phoenix/pull/517#discussion_r292753308
 
 

 ##########
 File path: 
phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
 ##########
 @@ -931,220 +944,323 @@ private void send(Iterator<TableRef> tableRefIterator) 
throws SQLException {
                     joinMutationState(new TableRef(tableRef), 
multiRowMutationState, txMutations);
                 }
             }
-            long serverTimestamp = HConstants.LATEST_TIMESTAMP;
+            Map<TableInfo, List<Mutation>> deletedIndexMutations = new 
LinkedHashMap<>();
+            Map<TableInfo, List<Mutation>> verifiedMutationsForIndex = 
filterIndexCheckerMutations(
+                    physicalTableMutationMap, deletedIndexMutations);
+            // We need to clone the verifiedMutations because sendMutations 
will remove entries from the map
+            // We will need the whole map because we will need to set the 
verified to true.
+            List<TableInfo> clonedIndexTables = new ArrayList<>();
+            boolean hasVerifiedMutations = verifiedMutationsForIndex.size() > 
0;
+
+            // Phase 1: Send verified indexes with VERIFIED=false
+            if (hasVerifiedMutations) {
+                clonedIndexTables.addAll(verifiedMutationsForIndex.keySet());
+                // addRowMutations generates the mutations with VERIFIED=false
+                sendMutations(verifiedMutationsForIndex.entrySet().iterator(), 
span, indexMetaDataPtr);
+            }
+
+            // Phase 2: Send data table and other indexes
             Iterator<Entry<TableInfo, List<Mutation>>> mutationsIterator = 
physicalTableMutationMap.entrySet()
                     .iterator();
-            while (mutationsIterator.hasNext()) {
-                Entry<TableInfo, List<Mutation>> pair = 
mutationsIterator.next();
-                TableInfo tableInfo = pair.getKey();
-                byte[] htableName = tableInfo.getHTableName().getBytes();
-                List<Mutation> mutationList = pair.getValue();
-                List<List<Mutation>> mutationBatchList =
-                        getMutationBatchList(batchSize, batchSizeBytes, 
mutationList);
-
-                // create a span per target table
-                // TODO maybe we can be smarter about the table name to string 
here?
-                Span child = Tracing.child(span, "Writing mutation batch for 
table: " + Bytes.toString(htableName));
-
-                int retryCount = 0;
-                boolean shouldRetry = false;
-                long numMutations = 0;
-                long mutationSizeBytes = 0;
-                long mutationCommitTime = 0;
-                long numFailedMutations = 0;
-                ;
-                long startTime = 0;
-                boolean shouldRetryIndexedMutation = false;
-                IndexWriteException iwe = null;
-                do {
-                    TableRef origTableRef = tableInfo.getOrigTableRef();
-                    PTable table = origTableRef.getTable();
-                    table.getIndexMaintainers(indexMetaDataPtr, connection);
-                    final ServerCache cache = tableInfo.isDataTable() ? 
-                            
IndexMetaDataCacheClient.setMetaDataOnMutations(connection, table,
-                                    mutationList, indexMetaDataPtr) : null;
-                    // If we haven't retried yet, retry for this case only, as 
it's possible that
-                    // a split will occur after we send the index metadata 
cache to all known
-                    // region servers.
-                    shouldRetry = cache != null;
-                    SQLException sqlE = null;
-                    Table hTable = 
connection.getQueryServices().getTable(htableName);
-                    try {
-                        if (table.isTransactional()) {
-                            // Track tables to which we've sent uncommitted 
data
-                            if (tableInfo.isDataTable()) {
-                                
uncommittedPhysicalNames.add(table.getPhysicalName().getString());
-                                phoenixTransactionContext.markDMLFence(table);
-                            }
-                            // Only pass true for last argument if the index 
is being written to on it's own (i.e. initial
-                            // index population), not if it's being written to 
for normal maintenance due to writes to
-                            // the data table. This case is different because 
the initial index population does not need
-                            // to be done transactionally since the index is 
only made active after all writes have
-                            // occurred successfully.
-                            hTable = 
phoenixTransactionContext.getTransactionalTableWriter(connection, table, 
hTable, tableInfo.isDataTable() && table.getType() == PTableType.INDEX);
+            sendMutations(mutationsIterator, span, indexMetaDataPtr);
+
+            // Phase 3: Send verfied indexes with VERIFIED = true
+            if (hasVerifiedMutations) {
+                Map<TableInfo, List<Mutation>>
+                            clonedVerifiedMutations =
+                            setVerifiedColumnForIndexes(clonedIndexTables, 
clonedDataMutations, deletedIndexMutations,
+                                    TRUE_BYTES);
+                try {
+                    
sendMutations(clonedVerifiedMutations.entrySet().iterator(), span,
+                            indexMetaDataPtr);
+                } catch (SQLException ex) {
+                    LOGGER.warn("Ignoring exception that happened during 
setting index verified value to TRUE", ex);
+                }
+            }
+        }
+    }
+
+    private void sendMutations(Iterator<Entry<TableInfo, List<Mutation>>> 
mutationsIterator, Span span, ImmutableBytesWritable indexMetaDataPtr)
+            throws SQLException {
+        while (mutationsIterator.hasNext()) {
+            Entry<TableInfo, List<Mutation>> pair = mutationsIterator.next();
+            TableInfo tableInfo = pair.getKey();
+            byte[] htableName = tableInfo.getHTableName().getBytes();
+            List<Mutation> mutationList = pair.getValue();
+            List<List<Mutation>> mutationBatchList =
+                    getMutationBatchList(batchSize, batchSizeBytes, 
mutationList);
+
+            // create a span per target table
+            // TODO maybe we can be smarter about the table name to string 
here?
+            Span child = Tracing.child(span, "Writing mutation batch for 
table: " + Bytes.toString(htableName));
+
+            int retryCount = 0;
+            boolean shouldRetry = false;
+            long numMutations = 0;
+            long mutationSizeBytes = 0;
+            long mutationCommitTime = 0;
+            long numFailedMutations = 0;
+            ;
+            long startTime = 0;
+            boolean shouldRetryIndexedMutation = false;
+            IndexWriteException iwe = null;
+            do {
+                TableRef origTableRef = tableInfo.getOrigTableRef();
+                PTable table = origTableRef.getTable();
+                table.getIndexMaintainers(indexMetaDataPtr, connection);
+                final ServerCache cache = tableInfo.isDataTable() ?
+                        
IndexMetaDataCacheClient.setMetaDataOnMutations(connection, table,
+                                mutationList, indexMetaDataPtr) : null;
+                // If we haven't retried yet, retry for this case only, as 
it's possible that
+                // a split will occur after we send the index metadata cache 
to all known
+                // region servers.
+                shouldRetry = cache != null;
+                SQLException sqlE = null;
+                Table hTable = 
connection.getQueryServices().getTable(htableName);
+                try {
+                    if (table.isTransactional()) {
+                        // Track tables to which we've sent uncommitted data
+                        if (tableInfo.isDataTable()) {
+                            
uncommittedPhysicalNames.add(table.getPhysicalName().getString());
+                            phoenixTransactionContext.markDMLFence(table);
                         }
-                        numMutations = mutationList.size();
-                        GLOBAL_MUTATION_BATCH_SIZE.update(numMutations);
-                        mutationSizeBytes = 
calculateMutationSize(mutationList);
-
-                        startTime = System.currentTimeMillis();
-                        child.addTimelineAnnotation("Attempt " + retryCount);
-                        Iterator<List<Mutation>> itrListMutation = 
mutationBatchList.iterator();
-                        while (itrListMutation.hasNext()) {
-                            final List<Mutation> mutationBatch = 
itrListMutation.next();
-                            if (shouldRetryIndexedMutation) {
-                                // if there was an index write failure, retry 
the mutation in a loop
-                                final Table finalHTable = hTable;
-                                final ImmutableBytesWritable 
finalindexMetaDataPtr =
-                                        indexMetaDataPtr;
-                                final PTable finalPTable = table;
-                                
PhoenixIndexFailurePolicy.doBatchWithRetries(new MutateCommand() {
-                                    @Override
-                                    public void doMutation() throws 
IOException {
-                                        try {
-                                            finalHTable.batch(mutationBatch, 
null);
-                                        } catch (InterruptedException e) {
-                                            Thread.currentThread().interrupt();
-                                            throw new IOException(e);
-                                        } catch (IOException e) {
-                                            e = 
updateTableRegionCacheIfNecessary(e);
-                                            throw e;
-                                        }
+                        // Only pass true for last argument if the index is 
being written to on it's own (i.e. initial
+                        // index population), not if it's being written to for 
normal maintenance due to writes to
+                        // the data table. This case is different because the 
initial index population does not need
+                        // to be done transactionally since the index is only 
made active after all writes have
+                        // occurred successfully.
+                        hTable = 
phoenixTransactionContext.getTransactionalTableWriter(connection, table, 
hTable, tableInfo.isDataTable() && table.getType() == PTableType.INDEX);
+                    }
+                    numMutations = mutationList.size();
+                    GLOBAL_MUTATION_BATCH_SIZE.update(numMutations);
+                    mutationSizeBytes = calculateMutationSize(mutationList);
+
+                    startTime = System.currentTimeMillis();
+                    child.addTimelineAnnotation("Attempt " + retryCount);
+                    Iterator<List<Mutation>> itrListMutation = 
mutationBatchList.iterator();
+                    while (itrListMutation.hasNext()) {
+                        final List<Mutation> mutationBatch = 
itrListMutation.next();
+                        if (shouldRetryIndexedMutation) {
+                            // if there was an index write failure, retry the 
mutation in a loop
+                            final Table finalHTable = hTable;
+                            final ImmutableBytesWritable finalindexMetaDataPtr 
=
+                                    indexMetaDataPtr;
+                            final PTable finalPTable = table;
+                            PhoenixIndexFailurePolicy.doBatchWithRetries(new 
MutateCommand() {
+                                @Override
+                                public void doMutation() throws IOException {
+                                    try {
+                                        finalHTable.batch(mutationBatch, null);
+                                    } catch (InterruptedException e) {
+                                        Thread.currentThread().interrupt();
+                                        throw new IOException(e);
+                                    } catch (IOException e) {
+                                        e = 
updateTableRegionCacheIfNecessary(e);
+                                        throw e;
                                     }
+                                }
 
-                                    @Override
-                                    public List<Mutation> getMutationList() {
-                                        return mutationBatch;
-                                    }
+                                @Override
+                                public List<Mutation> getMutationList() {
+                                    return mutationBatch;
+                                }
 
-                                    private IOException
-                                            
updateTableRegionCacheIfNecessary(IOException ioe) {
-                                        SQLException sqlE =
-                                                
ServerUtil.parseLocalOrRemoteServerException(ioe);
-                                        if (sqlE != null
-                                                && sqlE.getErrorCode() == 
SQLExceptionCode.INDEX_METADATA_NOT_FOUND
-                                                        .getErrorCode()) {
-                                            try {
-                                                
connection.getQueryServices().clearTableRegionCache(
+                                private IOException
+                                updateTableRegionCacheIfNecessary(IOException 
ioe) {
+                                    SQLException sqlE =
+                                            
ServerUtil.parseLocalOrRemoteServerException(ioe);
+                                    if (sqlE != null
+                                            && sqlE.getErrorCode() == 
SQLExceptionCode.INDEX_METADATA_NOT_FOUND
+                                            .getErrorCode()) {
+                                        try {
+                                            
connection.getQueryServices().clearTableRegionCache(
                                                     finalHTable.getName());
-                                                
IndexMetaDataCacheClient.setMetaDataOnMutations(
+                                            
IndexMetaDataCacheClient.setMetaDataOnMutations(
                                                     connection, finalPTable, 
mutationBatch,
                                                     finalindexMetaDataPtr);
-                                            } catch (SQLException e) {
-                                                return 
ServerUtil.createIOException(
+                                        } catch (SQLException e) {
+                                            return 
ServerUtil.createIOException(
                                                     "Exception during updating 
index meta data cache",
                                                     ioe);
-                                            }
                                         }
-                                        return ioe;
                                     }
-                                }, iwe, connection, 
connection.getQueryServices().getProps());
-                                shouldRetryIndexedMutation = false;
-                            } else {
-                                hTable.batch(mutationBatch, null);
-                            }
-                            // remove each batch from the list once it gets 
applied
-                            // so when failures happens for any batch we only 
start
-                            // from that batch only instead of doing duplicate 
reply of already
-                            // applied batches from entire list, also we can 
set
-                            // REPLAY_ONLY_INDEX_WRITES for first batch
-                            // only in case of 1121 SQLException
-                            itrListMutation.remove();
-
-                            batchCount++;
-                            if (LOGGER.isDebugEnabled())
-                                LOGGER.debug("Sent batch of " + 
mutationBatch.size() + " for "
-                                        + Bytes.toString(htableName));
-                        }
-                        child.stop();
-                        child.stop();
-                        shouldRetry = false;
-                        mutationCommitTime = System.currentTimeMillis() - 
startTime;
-                        GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
-                        numFailedMutations = 0;
-
-                        // Remove batches as we process them
-                        mutations.remove(origTableRef);
-                        if (tableInfo.isDataTable()) {
-                            numRows -= numMutations;
-                            // recalculate the estimated size
-                            estimatedSize = 
PhoenixKeyValueUtil.getEstimatedRowMutationSize(mutations);
+                                    return ioe;
+                                }
+                            }, iwe, connection, 
connection.getQueryServices().getProps());
+                            shouldRetryIndexedMutation = false;
+                        } else {
+                            hTable.batch(mutationBatch, null);
                         }
-                    } catch (Exception e) {
-                        mutationCommitTime = System.currentTimeMillis() - 
startTime;
-                        serverTimestamp = ServerUtil.parseServerTimestamp(e);
-                        SQLException inferredE = 
ServerUtil.parseServerExceptionOrNull(e);
-                        if (inferredE != null) {
-                            if (shouldRetry
-                                    && retryCount == 0
-                                    && inferredE.getErrorCode() == 
SQLExceptionCode.INDEX_METADATA_NOT_FOUND
-                                            .getErrorCode()) {
-                                // Swallow this exception once, as it's 
possible that we split after sending the index
-                                // metadata
-                                // and one of the region servers doesn't have 
it. This will cause it to have it the next
-                                // go around.
-                                // If it fails again, we don't retry.
-                                String msg = "Swallowing exception and 
retrying after clearing meta cache on connection. "
-                                        + inferredE;
-                                LOGGER.warn(LogUtil.addCustomAnnotations(msg, 
connection));
-                                
connection.getQueryServices().clearTableRegionCache(TableName.valueOf(htableName));
-
-                                // add a new child span as this one failed
-                                child.addTimelineAnnotation(msg);
-                                child.stop();
-                                child = Tracing.child(span, "Failed batch, 
attempting retry");
-
-                                continue;
-                            } else if (inferredE.getErrorCode() == 
SQLExceptionCode.INDEX_WRITE_FAILURE.getErrorCode()) {
-                                iwe = 
PhoenixIndexFailurePolicy.getIndexWriteException(inferredE);
-                                if (iwe != null && 
!shouldRetryIndexedMutation) {
-                                    // For an index write failure, the data 
table write succeeded,
-                                    // so when we retry we need to set 
REPLAY_WRITES
-                                    // for first batch in list only.
-                                    for (Mutation m : 
mutationBatchList.get(0)) {
-                                        if 
(!PhoenixIndexMetaData.isIndexRebuild(
+                        // remove each batch from the list once it gets applied
+                        // so when failures happens for any batch we only start
+                        // from that batch only instead of doing duplicate 
reply of already
+                        // applied batches from entire list, also we can set
+                        // REPLAY_ONLY_INDEX_WRITES for first batch
+                        // only in case of 1121 SQLException
+                        itrListMutation.remove();
+
+                        batchCount++;
+                        if (LOGGER.isDebugEnabled())
+                            LOGGER.debug("Sent batch of " + 
mutationBatch.size() + " for "
+                                    + Bytes.toString(htableName));
+                    }
+                    child.stop();
+                    child.stop();
+                    shouldRetry = false;
+                    mutationCommitTime = System.currentTimeMillis() - 
startTime;
+                    GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
+                    numFailedMutations = 0;
+
+                    // Remove batches as we process them
+                    mutations.remove(origTableRef);
+                    if (tableInfo.isDataTable()) {
+                        numRows -= numMutations;
+                        // recalculate the estimated size
+                        estimatedSize = 
PhoenixKeyValueUtil.getEstimatedRowMutationSize(mutations);
+                    }
+                } catch (Exception e) {
+                    mutationCommitTime = System.currentTimeMillis() - 
startTime;
+                    long serverTimestamp = ServerUtil.parseServerTimestamp(e);
+                    SQLException inferredE = 
ServerUtil.parseServerExceptionOrNull(e);
+                    if (inferredE != null) {
+                        if (shouldRetry
+                                && retryCount == 0
+                                && inferredE.getErrorCode() == 
SQLExceptionCode.INDEX_METADATA_NOT_FOUND
+                                .getErrorCode()) {
+                            // Swallow this exception once, as it's possible 
that we split after sending the index
+                            // metadata
+                            // and one of the region servers doesn't have it. 
This will cause it to have it the next
+                            // go around.
+                            // If it fails again, we don't retry.
+                            String msg = "Swallowing exception and retrying 
after clearing meta cache on connection. "
+                                    + inferredE;
+                            LOGGER.warn(LogUtil.addCustomAnnotations(msg, 
connection));
+                            
connection.getQueryServices().clearTableRegionCache(TableName.valueOf(htableName));
+
+                            // add a new child span as this one failed
+                            child.addTimelineAnnotation(msg);
+                            child.stop();
+                            child = Tracing.child(span, "Failed batch, 
attempting retry");
+
+                            continue;
+                        } else if (inferredE.getErrorCode() == 
SQLExceptionCode.INDEX_WRITE_FAILURE.getErrorCode()) {
+                            iwe = 
PhoenixIndexFailurePolicy.getIndexWriteException(inferredE);
+                            if (iwe != null && !shouldRetryIndexedMutation) {
+                                // For an index write failure, the data table 
write succeeded,
+                                // so when we retry we need to set 
REPLAY_WRITES
+                                // for first batch in list only.
+                                for (Mutation m : mutationBatchList.get(0)) {
+                                    if (!PhoenixIndexMetaData.isIndexRebuild(
                                             m.getAttributesMap())){
-                                            
m.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
+                                        
m.setAttribute(BaseScannerRegionObserver.REPLAY_WRITES,
                                                 
BaseScannerRegionObserver.REPLAY_ONLY_INDEX_WRITES
-                                                );
-                                        }
-                                        PhoenixKeyValueUtil.setTimestamp(m, 
serverTimestamp);
+                                        );
                                     }
-                                    shouldRetry = true;
-                                    shouldRetryIndexedMutation = true;
-                                    continue;
+                                    PhoenixKeyValueUtil.setTimestamp(m, 
serverTimestamp);
                                 }
+                                shouldRetry = true;
+                                shouldRetryIndexedMutation = true;
+                                continue;
                             }
-                            e = inferredE;
                         }
-                        // Throw to client an exception that indicates the 
statements that
-                        // were not committed successfully.
-                        int[] uncommittedStatementIndexes = 
getUncommittedStatementIndexes();
-                        sqlE = new CommitException(e, 
uncommittedStatementIndexes, serverTimestamp);
-                        numFailedMutations = 
uncommittedStatementIndexes.length;
-                        
GLOBAL_MUTATION_BATCH_FAILED_COUNT.update(numFailedMutations);
+                        e = inferredE;
+                    }
+                    // Throw to client an exception that indicates the 
statements that
+                    // were not committed successfully.
+                    int[] uncommittedStatementIndexes = 
getUncommittedStatementIndexes();
+                    sqlE = new CommitException(e, uncommittedStatementIndexes, 
serverTimestamp);
+                    numFailedMutations = uncommittedStatementIndexes.length;
+                    
GLOBAL_MUTATION_BATCH_FAILED_COUNT.update(numFailedMutations);
+                } finally {
+                    MutationMetric mutationsMetric = new 
MutationMetric(numMutations, mutationSizeBytes,
+                            mutationCommitTime, numFailedMutations);
+                    
mutationMetricQueue.addMetricsForTable(Bytes.toString(htableName), 
mutationsMetric);
+                    try {
+                        if (cache != null) cache.close();
                     } finally {
-                        MutationMetric mutationsMetric = new 
MutationMetric(numMutations, mutationSizeBytes,
-                                mutationCommitTime, numFailedMutations);
-                        
mutationMetricQueue.addMetricsForTable(Bytes.toString(htableName), 
mutationsMetric);
                         try {
-                            if (cache != null) cache.close();
-                        } finally {
-                            try {
-                                hTable.close();
-                            } catch (IOException e) {
-                                if (sqlE != null) {
-                                    
sqlE.setNextException(ServerUtil.parseServerException(e));
-                                } else {
-                                    sqlE = ServerUtil.parseServerException(e);
-                                }
+                            hTable.close();
+                        } catch (IOException e) {
+                            if (sqlE != null) {
+                                
sqlE.setNextException(ServerUtil.parseServerException(e));
+                            } else {
+                                sqlE = ServerUtil.parseServerException(e);
                             }
-                            if (sqlE != null) { throw sqlE; }
                         }
+                        if (sqlE != null) { throw sqlE; }
                     }
-                } while (shouldRetry && retryCount++ < 1);
+                }
+            } while (shouldRetry && retryCount++ < 1);
+        }
+    }
+
+    private Map<TableInfo, List<Mutation>> 
filterIndexCheckerMutations(Map<TableInfo, List<Mutation>> mutationMap,
+            Map<TableInfo, List<Mutation>> deletedIndexMutations) {
+        Map<TableInfo, List<Mutation>> verifiedMutationsMap = new 
LinkedHashMap<>();
+        Iterator<Entry<TableInfo, List<Mutation>>> mapIter = 
mutationMap.entrySet().iterator();
+        while (mapIter.hasNext()) {
+            Entry<TableInfo, List<Mutation>> pair = mapIter.next();
+            TableInfo tableInfo = pair.getKey();
+            if (IndexUtil.isGlobalIndexCheckerEnabled(connection, 
tableInfo.hTableName)) {
+                PTable table = tableInfo.getOrigTableRef().getTable();
+                byte[] emptyCF = SchemaUtil.getEmptyColumnFamily(table);
+                byte[] emptyCQ = 
EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
+
+                List<Mutation> mutations = pair.getValue();
+
+                if (mutations.size() == 0) {
+                    continue;
+                }
+
+                List<Mutation> indexMutations = new LinkedList<>();
+                for (Mutation m : mutations) {
+                    if (m instanceof Delete) {
+                        Put put = new Put(m.getRow());
+                        put.addColumn(emptyCF, emptyCQ, m.getTimestamp(), 
FALSE_BYTES);
+                        indexMutations.add(put);
+                        if (deletedIndexMutations.containsKey(tableInfo)) {
+                            deletedIndexMutations.get(tableInfo).add(m);
+                        } else {
+                            List<Mutation> mts = Lists.newArrayList();
+                            mts.add(m);
+                            deletedIndexMutations.put(tableInfo, mts);
+                        }
+                    }
+                }
+
+                verifiedMutationsMap.put(pair.getKey(), indexMutations);
+                mapIter.remove();
+            }
+
+        }
+        return verifiedMutationsMap;
+    }
+
+    private Map<TableInfo, List<Mutation>> 
setVerifiedColumnForIndexes(List<TableInfo> indexTables, List<Mutation> 
dataMutations,
+            Map<TableInfo, List<Mutation>> deletedIndexMutations, byte[] 
verifiedBytes)
+            throws SQLException {
+        Map<TableInfo, List<Mutation>> updatedMap = new LinkedHashMap<>();
+        updatedMap.putAll(deletedIndexMutations);
+        for (TableInfo tableInfo : indexTables) {
+            PTable indexTable = null;
+            PTable dataTable = tableInfo.origTableRef.getTable();
+
+            List<PTable> indexes = dataTable.getIndexes();
+            for (PTable index : indexes) {
+                if (index.getName().equals(tableInfo.hTableName)) {
 
 Review comment:
   Although private member hTableName is accessible in the Outer class, better 
to use the public method to access it. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to