int newFileNumber = metadataFiles.get(metadataFiles.size() - 1).getFileNumber() + 1; List<ExternalFile> fileSystemFiles = getSnapshotFromExternalFileSystem(dataset);
public static JobSpecification buildRecoverOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider) throws AlgebricksException { JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext()); IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager(); ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>(); AlgebricksPartitionConstraint constraints = null; for (Index index : indexes) { IFileSplitProvider indexSplitProvider; if (isValidIndexName(index.getDatasetName(), index.getIndexName())) { Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints = metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName()); indexSplitProvider = sAndConstraints.first; constraints = sAndConstraints.second; } else { indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds, IndexingConstants.getFilesIndexName(ds.getDatasetName())).first; } IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(storageMgr, indexSplitProvider); treeDataflowHelperFactories.add(indexDataflowHelperFactory); } ExternalDatasetIndexesRecoverOperatorDescriptor op = new ExternalDatasetIndexesRecoverOperatorDescriptor(spec, treeDataflowHelperFactories); spec.addRoot(op); AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints); spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy()); return spec; }
public static JobSpecification buildFilesIndexUpdateOp(Dataset ds, List<ExternalFile> metadataFiles, List<ExternalFile> addedFiles, List<ExternalFile> appendedFiles, MetadataProvider metadataProvider) throws AlgebricksException { ArrayList<ExternalFile> files = new ArrayList<>(); for (ExternalFile file : metadataFiles) { if (file.getPendingOp() == ExternalFilePendingOp.DROP_OP) { files.add(file); } else if (file.getPendingOp() == ExternalFilePendingOp.APPEND_OP) { for (ExternalFile appendedFile : appendedFiles) { if (appendedFile.getFileName().equals(file.getFileName())) { files.add(new ExternalFile(file.getDataverseName(), file.getDatasetName(), file.getFileNumber(), file.getFileName(), file.getLastModefiedTime(), appendedFile.getSize(), ExternalFilePendingOp.NO_OP)); } } } } for (ExternalFile file : addedFiles) { files.add(file); } Collections.sort(files); return buildFilesIndexUpdateJobSpec(ds, files, metadataProvider); }
if (ExternalIndexingOperations.isDatasetUptodate(ds, metadataFiles, addedFiles, deletedFiles, appendedFiles)) { ((ExternalDatasetDetails) ds.getDatasetDetails()).setRefreshTimestamp(txnTime); transactionDataset = ExternalIndexingOperations.createTransactionDataset(ds); spec = ExternalIndexingOperations.buildFilesIndexUpdateOp(ds, metadataFiles, addedFiles, appendedFiles, metadataProvider); if (!ExternalIndexingOperations.isFileIndex(index)) { spec = ExternalIndexingOperations.buildIndexUpdateOp(ds, index, metadataFiles, addedFiles, appendedFiles, metadataProvider, sourceLoc); spec = ExternalIndexingOperations.buildCommitJob(ds, indexes, metadataProvider); bActiveTxn = true; metadataProvider.setMetadataTxnContext(mdTxnCtx); spec = ExternalIndexingOperations.buildAbortOp(ds, indexes, metadataProvider); MetadataManager.INSTANCE.commitTransaction(mdTxnCtx); bActiveTxn = false;
if (!ExternalIndexingOperations.isIndexible((ExternalDatasetDetails) ds.getDatasetDetails())) { throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "dataset using " + ((ExternalDatasetDetails) ds.getDatasetDetails()).getAdapter() if (!ExternalIndexingOperations.isValidIndexName(index.getDatasetName(), index.getIndexName())) { throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "external dataset index name is invalid"); externalFilesSnapshot = ExternalIndexingOperations.getSnapshotFromExternalFileSystem(ds); spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot, metadataProvider); if (spec == null) { try { JobSpecification jobSpec = ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds); MetadataManager.INSTANCE.commitTransaction(mdTxnCtx); bActiveTxn = false;
MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName); for (int k = 0; k < indexes.size(); k++) { if (ExternalIndexingOperations.isFileIndex(indexes.get(k))) { jobsToExecute.add( ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, dataset)); } else { jobsToExecute.add(IndexUtil.buildDropIndexJobSpec(indexes.get(k), metadataProvider, dataset,
MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx.getValue(), dataverseName, datasetName); for (int j = 0; j < indexes.size(); j++) { if (ExternalIndexingOperations.isFileIndex(indexes.get(j))) { jobsToExecute .add(IndexUtil.buildDropIndexJobSpec(indexes.get(j), metadataProvider, this, sourceLoc));
ExternalIndexingOperations.buildAbortOp(dataset, indexes, metadataProvider); executeHyracksJob(jobSpec); ExternalIndexingOperations.buildRecoverOp(dataset, indexes, metadataProvider); executeHyracksJob(jobSpec);
throw new CompilationException(ErrorCode.UNKNOWN_INDEX, sourceLoc, indexName); } else if (ExternalIndexingOperations.isFileIndex(index)) { throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Dropping a dataset's files index is not allowed."); if (ExternalIndexingOperations.isFileIndex(externalIndex)) { jobsToExecute .add(ExternalIndexingOperations.buildDropFilesIndexJobSpec(metadataProvider, ds));
public static JobSpecification buildCommitJob(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider) throws AlgebricksException { JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext()); IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager(); ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>(); AlgebricksPartitionConstraint constraints = null; for (Index index : indexes) { IFileSplitProvider indexSplitProvider; if (isValidIndexName(index.getDatasetName(), index.getIndexName())) { Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints = metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName()); indexSplitProvider = sAndConstraints.first; constraints = sAndConstraints.second; } else { indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds, IndexingConstants.getFilesIndexName(ds.getDatasetName())).first; } IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(storageMgr, indexSplitProvider); treeDataflowHelperFactories.add(indexDataflowHelperFactory); } ExternalDatasetIndexesCommitOperatorDescriptor op = new ExternalDatasetIndexesCommitOperatorDescriptor(spec, treeDataflowHelperFactories); spec.addRoot(op); AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints); spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy()); return spec; }
public static JobSpecification buildAbortOp(Dataset ds, List<Index> indexes, MetadataProvider metadataProvider) throws AlgebricksException { JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext()); IStorageManager storageMgr = metadataProvider.getStorageComponentProvider().getStorageManager(); ArrayList<IIndexDataflowHelperFactory> treeDataflowHelperFactories = new ArrayList<>(); AlgebricksPartitionConstraint constraints = null; for (Index index : indexes) { IFileSplitProvider indexSplitProvider; if (isValidIndexName(index.getDatasetName(), index.getIndexName())) { Pair<IFileSplitProvider, AlgebricksPartitionConstraint> sAndConstraints = metadataProvider.getSplitProviderAndConstraints(ds, index.getIndexName()); indexSplitProvider = sAndConstraints.first; constraints = sAndConstraints.second; } else { indexSplitProvider = metadataProvider.getSplitProviderAndConstraints(ds, IndexingConstants.getFilesIndexName(ds.getDatasetName())).first; } IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(storageMgr, indexSplitProvider); treeDataflowHelperFactories.add(indexDataflowHelperFactory); } ExternalDatasetIndexesAbortOperatorDescriptor op = new ExternalDatasetIndexesAbortOperatorDescriptor(spec, treeDataflowHelperFactories); spec.addRoot(op); AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, op, constraints); spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy()); return spec; }