Cleanup Feed CodeBase
In order to expedite cleaning feeds up and reaching a maintainable
state, we disabled the following:
1. Policies (At least once, throttling, discarding, elasticity).
2. Statistics Reporting.
3. Load management.
4. Feed re-activation upon System reboot.
Right now on master, none of these features work reliably.
We will re-introduce them one feature at a time.
The rules followed in this change:
1. Keep X if X is tested in a test case.
2. Remove X if X is not used in test cases.
After a few meetings with Mike and Till, the policies
1. Buffering
2. Spill
3. Discard
4. Throttle
have been fixed and unit tests have been added.
Change-Id: I545bc4f8560564e4c868a80d27c77a4edd97a8b8
Reviewed-on: https://asterix-gerrit.ics.uci.edu/798
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
index 8342be5..5b3e453 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
@@ -51,7 +51,6 @@
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataNode;
@@ -120,7 +119,7 @@
private IIOManager ioManager;
private boolean isShuttingdown;
- private IFeedManager feedManager;
+ private FeedManager feedManager;
private IReplicationChannel replicationChannel;
private IReplicationManager replicationManager;
@@ -173,7 +172,8 @@
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
ioManager, ncApplicationContext.getNodeId(), metadataProperties);
- localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
+ localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
+ .createRepository();
IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
this);
@@ -377,7 +377,7 @@
}
@Override
- public IFeedManager getFeedManager() {
+ public FeedManager getFeedManager() {
return feedManager;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
index 79ce721..c70950a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ConnectorAPIServlet.java
@@ -29,7 +29,6 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.app.external.CentralFeedManager;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
@@ -84,7 +83,7 @@
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
// Retrieves file splits of the dataset.
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null, CentralFeedManager.getInstance());
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null);
metadataProvider.setMetadataTxnContext(mdTxnCtx);
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
if (dataset == null) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
index eacee6d..8bc613a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServlet.java
@@ -32,9 +32,6 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.app.external.CentralFeedManager;
-import org.apache.asterix.external.feed.api.IFeedLoadManager;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.watch.FeedActivity;
@@ -86,52 +83,26 @@
}
String outStr = null;
- if (requestURI.startsWith("/webui/static")) {
- outStr = sb.toString();
- } else {
- Collection<FeedActivity> lfa = CentralFeedManager.getInstance().getFeedLoadManager().getFeedActivities();
- StringBuilder ldStr = new StringBuilder();
- ldStr.append("<br />");
- ldStr.append("<br />");
- if (lfa == null || lfa.isEmpty()) {
- ldStr.append("Currently there are no active feeds in AsterixDB");
- } else {
- ldStr.append("Active Feeds");
- }
- insertTable(ldStr, lfa);
- outStr = String.format(sb.toString(), ldStr.toString());
-
- }
+ outStr = sb.toString();
PrintWriter out = response.getWriter();
out.println(outStr);
}
+ @SuppressWarnings("unused")
private void insertTable(StringBuilder html, Collection<FeedActivity> list) {
- html.append("<table style=\"width:100%\">");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_FEED_NAME + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_DATASET_NAME + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_ACTIVE_SINCE + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_INTAKE_STAGE + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_COMPUTE_STAGE + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_STORE_STAGE + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_INTAKE_RATE + "</th>");
- html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_STORE_RATE + "</th>");
- for (FeedActivity activity : list) {
- insertRow(html, activity);
- }
}
+ @SuppressWarnings("null")
private void insertRow(StringBuilder html, FeedActivity activity) {
String intake = activity.getFeedActivityDetails().get(FeedActivityDetails.INTAKE_LOCATIONS);
String compute = activity.getFeedActivityDetails().get(FeedActivityDetails.COMPUTE_LOCATIONS);
String store = activity.getFeedActivityDetails().get(FeedActivityDetails.STORAGE_LOCATIONS);
- IFeedLoadManager loadManager = CentralFeedManager.getInstance().getFeedLoadManager();
FeedConnectionId connectionId = new FeedConnectionId(
new FeedId(activity.getDataverseName(), activity.getFeedName()), activity.getDatasetName());
- int intakeRate = loadManager.getOutflowRate(connectionId, FeedRuntimeType.COLLECT) * intake.split(",").length;
- int storeRate = loadManager.getOutflowRate(connectionId, FeedRuntimeType.STORE) * store.split(",").length;
+ int intakeRate = 0;
+ int storeRate = 0;
html.append("<tr>");
html.append("<td>" + activity.getFeedName() + "</td>");
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServletUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServletUtil.java
deleted file mode 100644
index 52a140d..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/FeedServletUtil.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.asterix.api.http.servlet;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Socket;
-import java.nio.CharBuffer;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.app.external.FeedLifecycleListener;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.RemoteSocketMessageListener;
-
-public class FeedServletUtil {
-
- private static final Logger LOGGER = Logger.getLogger(FeedServletUtil.class.getName());
- private static final char EOL = (char) "\n".getBytes()[0];
-
- public static final class Constants {
- public static final String TABLE_HEADER_FEED_NAME = "Feed";
- public static final String TABLE_HEADER_DATASET_NAME = "Dataset";
- public static final String TABLE_HEADER_ACTIVE_SINCE = "Timestamp";
- public static final String TABLE_HEADER_INTAKE_STAGE = "Intake Stage";
- public static final String TABLE_HEADER_COMPUTE_STAGE = "Compute Stage";
- public static final String TABLE_HEADER_STORE_STAGE = "Store Stage";
- public static final String TABLE_HEADER_INTAKE_RATE = "Intake";
- public static final String TABLE_HEADER_STORE_RATE = "Store";
- }
-
- public static void initiateSubscription(FeedConnectionId feedId, String host, int port) throws IOException {
- LinkedBlockingQueue<String> outbox = new LinkedBlockingQueue<String>();
- int subscriptionPort = port + 1;
- Socket sc = new Socket(host, subscriptionPort);
- InputStream in = sc.getInputStream();
-
- CharBuffer buffer = CharBuffer.allocate(50);
- char ch = 0;
- while (ch != EOL) {
- buffer.put(ch);
- ch = (char) in.read();
- }
- buffer.flip();
- sc.close();
-
- String s = new String(buffer.array());
- int feedSubscriptionPort = Integer.parseInt(s.trim());
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Response from Super Feed Manager Report Service " + port + " will connect at " + host + " "
- + port);
- }
-
- // register the feed subscription queue with FeedLifecycleListener
- FeedLifecycleListener.INSTANCE.registerFeedReportQueue(feedId, outbox);
- RemoteSocketMessageListener listener = new RemoteSocketMessageListener(host, feedSubscriptionPort, outbox);
- listener.start();
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/CentralFeedManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/CentralFeedManager.java
deleted file mode 100644
index cab5e64..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/CentralFeedManager.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringReader;
-import java.util.List;
-
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.feed.api.ICentralFeedManager;
-import org.apache.asterix.external.feed.api.IFeedLoadManager;
-import org.apache.asterix.external.feed.api.IFeedTrackingManager;
-import org.apache.asterix.external.feed.message.SocketMessageListener;
-import org.apache.asterix.lang.aql.parser.AQLParserFactory;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class CentralFeedManager implements ICentralFeedManager {
-
- private static final ICentralFeedManager centralFeedManager = new CentralFeedManager();
- private static final ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
-
- public static ICentralFeedManager getInstance() {
- return centralFeedManager;
- }
-
- private final int port;
- private final IFeedLoadManager feedLoadManager;
- private final IFeedTrackingManager feedTrackingManager;
- private final SocketMessageListener messageListener;
-
- private CentralFeedManager() {
- this.port = AsterixAppContextInfo.getInstance().getFeedProperties().getFeedCentralManagerPort();
- this.feedLoadManager = new FeedLoadManager();
- this.feedTrackingManager = new FeedTrackingManager();
- this.messageListener = new SocketMessageListener(port, new FeedMessageReceiver(this));
- }
-
- @Override
- public void start() throws AsterixException {
- messageListener.start();
- }
-
- @Override
- public void stop() throws AsterixException, IOException {
- messageListener.stop();
- }
-
- public static JobId runJob(JobSpecification spec, boolean waitForCompletion) throws Exception {
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobId jobId = hcc.startJob(spec);
- if (waitForCompletion) {
- hcc.waitForCompletion(jobId);
- }
- return jobId;
- }
-
- @Override
- public IFeedLoadManager getFeedLoadManager() {
- return feedLoadManager;
- }
-
- @Override
- public IFeedTrackingManager getFeedTrackingManager() {
- return feedTrackingManager;
- }
-
- public static class AQLExecutor {
-
- private static final PrintWriter out = new PrintWriter(System.out, true);
- private static final IParserFactory parserFactory = new AQLParserFactory();
-
- public static void executeAQL(String aql) throws Exception {
- IParser parser = parserFactory.createParser(new StringReader(aql));
- List<Statement> statements = parser.parse();
- SessionConfig pc = new SessionConfig(out, OutputFormat.ADM);
- QueryTranslator translator = new QueryTranslator(statements, pc, compilationProvider);
- translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
- QueryTranslator.ResultDelivery.SYNC);
- }
- }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java
index a143578..cfc2125 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedJobNotificationHandler.java
@@ -21,7 +21,6 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -40,15 +39,12 @@
import org.apache.asterix.external.feed.api.IFeedJoint.State;
import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.management.FeedJointKey;
import org.apache.asterix.external.feed.management.FeedWorkManager;
-import org.apache.asterix.external.feed.message.StorageReportFeedMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedActivity;
import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
import org.apache.asterix.external.feed.watch.FeedJobInfo;
@@ -57,9 +53,7 @@
import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
-import org.apache.asterix.metadata.feeds.BuiltinFeedPolicies;
import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.commons.lang3.StringUtils;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
@@ -85,7 +79,6 @@
private final Map<FeedId, FeedIntakeInfo> intakeJobInfos;
private final Map<FeedConnectionId, FeedConnectJobInfo> connectJobInfos;
private final Map<FeedId, Pair<FeedOperationCounter, List<IFeedJoint>>> feedPipeline;
- private final Map<FeedConnectionId, Pair<IIntakeProgressTracker, Long>> feedIntakeProgressTrackers;
public FeedJobNotificationHandler(LinkedBlockingQueue<FeedEvent> inbox) {
this.inbox = inbox;
@@ -94,7 +87,6 @@
this.connectJobInfos = new HashMap<FeedConnectionId, FeedConnectJobInfo>();
this.feedPipeline = new HashMap<FeedId, Pair<FeedOperationCounter, List<IFeedJoint>>>();
this.eventSubscribers = new HashMap<FeedConnectionId, List<IFeedLifecycleEventSubscriber>>();
- this.feedIntakeProgressTrackers = new HashMap<FeedConnectionId, Pair<IIntakeProgressTracker, Long>>();
}
@Override
@@ -124,29 +116,6 @@
}
}
- public void registerFeedIntakeProgressTracker(FeedConnectionId connectionId,
- IIntakeProgressTracker feedIntakeProgressTracker) {
- if (feedIntakeProgressTrackers.get(connectionId) == null) {
- this.feedIntakeProgressTrackers.put(connectionId,
- new Pair<IIntakeProgressTracker, Long>(feedIntakeProgressTracker, 0L));
- } else {
- throw new IllegalStateException(
- " Progress tracker for connection " + connectionId + " is alreader registered");
- }
- }
-
- public void deregisterFeedIntakeProgressTracker(FeedConnectionId connectionId) {
- this.feedIntakeProgressTrackers.remove(connectionId);
- }
-
- public void updateTrackingInformation(StorageReportFeedMessage srm) {
- Pair<IIntakeProgressTracker, Long> p = feedIntakeProgressTrackers.get(srm.getConnectionId());
- if (p != null && p.second < srm.getLastPersistedTupleIntakeTimestamp()) {
- p.second = srm.getLastPersistedTupleIntakeTimestamp();
- p.first.notifyIngestedTupleTimestamp(p.second);
- }
- }
-
public Collection<FeedIntakeInfo> getFeedIntakeInfos() {
return intakeJobInfos.values();
}
@@ -358,8 +327,6 @@
}
}
cInfo.setState(FeedJobState.ACTIVE);
- // register activity in metadata
- registerFeedActivity(cInfo);
}
private void notifyFeedEventSubscribers(FeedJobInfo jobInfo, FeedLifecycleEvent event) {
@@ -489,63 +456,12 @@
connectJobInfos.remove(connectionId);
jobInfos.remove(cInfo.getJobId());
- feedIntakeProgressTrackers.remove(cInfo.getConnectionId());
- deregisterFeedActivity(cInfo);
// notify event listeners
FeedLifecycleEvent event = failure ? FeedLifecycleEvent.FEED_COLLECT_FAILURE
: FeedLifecycleEvent.FEED_COLLECT_ENDED;
notifyFeedEventSubscribers(cInfo, event);
}
- private void registerFeedActivity(FeedConnectJobInfo cInfo) {
- Map<String, String> feedActivityDetails = new HashMap<String, String>();
-
- if (cInfo.getCollectLocations() != null) {
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.INTAKE_LOCATIONS,
- StringUtils.join(cInfo.getCollectLocations().iterator(), ','));
- }
-
- if (cInfo.getComputeLocations() != null) {
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.COMPUTE_LOCATIONS,
- StringUtils.join(cInfo.getComputeLocations().iterator(), ','));
- }
-
- if (cInfo.getStorageLocations() != null) {
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.STORAGE_LOCATIONS,
- StringUtils.join(cInfo.getStorageLocations().iterator(), ','));
- }
-
- String policyName = cInfo.getFeedPolicy().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_POLICY_NAME, policyName);
-
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_CONNECT_TIMESTAMP, (new Date()).toString());
- try {
- FeedActivity feedActivity = new FeedActivity(cInfo.getConnectionId().getFeedId().getDataverse(),
- cInfo.getConnectionId().getFeedId().getFeedName(), cInfo.getConnectionId().getDatasetName(),
- feedActivityDetails);
- CentralFeedManager.getInstance().getFeedLoadManager().reportFeedActivity(cInfo.getConnectionId(),
- feedActivity);
-
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to register feed activity for " + cInfo + " " + e.getMessage());
- }
-
- }
-
- }
-
- public void deregisterFeedActivity(FeedConnectJobInfo cInfo) {
- try {
- CentralFeedManager.getInstance().getFeedLoadManager().removeFeedActivity(cInfo.getConnectionId());
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to deregister feed activity for " + cInfo + " " + e.getMessage());
- }
- }
- }
-
public boolean isRegisteredFeedJob(JobId jobId) {
return jobInfos.get(jobId) != null;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java
index 161c863..b8435af 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLifecycleListener.java
@@ -18,56 +18,27 @@
*/
package org.apache.asterix.app.external;
-import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
-import org.apache.asterix.common.api.IClusterManagementWork;
-import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.common.api.IClusterManagementWorkResponse;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.external.feed.api.IFeedJoint;
import org.apache.asterix.external.feed.api.IFeedLifecycleEventSubscriber;
import org.apache.asterix.external.feed.api.IFeedLifecycleListener;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
-import org.apache.asterix.external.feed.management.FeedCollectInfo;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.management.FeedJointKey;
-import org.apache.asterix.external.feed.message.StorageReportFeedMessage;
import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
-import org.apache.asterix.external.feed.watch.FeedJobInfo;
import org.apache.asterix.external.feed.watch.FeedJobInfo.FeedJobState;
import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.statement.DataverseDecl;
-import org.apache.asterix.lang.common.statement.DisconnectFeedStatement;
-import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.cluster.AddNodeWork;
-import org.apache.asterix.metadata.cluster.ClusterManager;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.asterix.om.util.AsterixClusterProperties;
-import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
@@ -84,31 +55,17 @@
public class FeedLifecycleListener implements IFeedLifecycleListener {
private static final Logger LOGGER = Logger.getLogger(FeedLifecycleListener.class.getName());
-
public static FeedLifecycleListener INSTANCE = new FeedLifecycleListener();
- private static final ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
private final LinkedBlockingQueue<FeedEvent> jobEventInbox;
- private final LinkedBlockingQueue<IClusterManagementWorkResponse> responseInbox;
- private final Map<FeedCollectInfo, List<String>> dependentFeeds = new HashMap<FeedCollectInfo, List<String>>();
- private final Map<FeedConnectionId, LinkedBlockingQueue<String>> feedReportQueue;
private final FeedJobNotificationHandler feedJobNotificationHandler;
- private final FeedWorkRequestResponseHandler feedWorkRequestResponseHandler;
private final ExecutorService executorService;
- private ClusterState state;
-
private FeedLifecycleListener() {
this.jobEventInbox = new LinkedBlockingQueue<FeedEvent>();
this.feedJobNotificationHandler = new FeedJobNotificationHandler(jobEventInbox);
- this.responseInbox = new LinkedBlockingQueue<IClusterManagementWorkResponse>();
- this.feedWorkRequestResponseHandler = new FeedWorkRequestResponseHandler(responseInbox);
- this.feedReportQueue = new HashMap<FeedConnectionId, LinkedBlockingQueue<String>>();
this.executorService = Executors.newCachedThreadPool();
this.executorService.execute(feedJobNotificationHandler);
- this.executorService.execute(feedWorkRequestResponseHandler);
- ClusterManager.INSTANCE.registerSubscriber(this);
- this.state = AsterixClusterProperties.INSTANCE.getState();
}
@Override
@@ -133,19 +90,6 @@
return feedJobNotificationHandler.getFeedConnectJobInfo(connectionId);
}
- public void registerFeedIntakeProgressTracker(FeedConnectionId connectionId,
- IIntakeProgressTracker feedIntakeProgressTracker) {
- feedJobNotificationHandler.registerFeedIntakeProgressTracker(connectionId, feedIntakeProgressTracker);
- }
-
- public void deregisterFeedIntakeProgressTracker(FeedConnectionId connectionId) {
- feedJobNotificationHandler.deregisterFeedIntakeProgressTracker(connectionId);
- }
-
- public void updateTrackingInformation(StorageReportFeedMessage srm) {
- feedJobNotificationHandler.updateTrackingInformation(srm);
- }
-
/*
* Traverse job specification to categorize job as a feed intake job or a feed collection job
*/
@@ -201,209 +145,6 @@
}
}
- @Override
- public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
- Set<IClusterManagementWork> workToBeDone = new HashSet<IClusterManagementWork>();
-
- Collection<FeedIntakeInfo> intakeInfos = feedJobNotificationHandler.getFeedIntakeInfos();
- Collection<FeedConnectJobInfo> connectJobInfos = feedJobNotificationHandler.getFeedConnectInfos();
-
- Map<String, List<FeedJobInfo>> impactedJobs = new HashMap<String, List<FeedJobInfo>>();
-
- for (String deadNode : deadNodeIds) {
- for (FeedIntakeInfo intakeInfo : intakeInfos) {
- if (intakeInfo.getIntakeLocation().contains(deadNode)) {
- List<FeedJobInfo> infos = impactedJobs.get(deadNode);
- if (infos == null) {
- infos = new ArrayList<FeedJobInfo>();
- impactedJobs.put(deadNode, infos);
- }
- infos.add(intakeInfo);
- intakeInfo.setState(FeedJobState.UNDER_RECOVERY);
- }
- }
-
- for (FeedConnectJobInfo connectInfo : connectJobInfos) {
- if (connectInfo.getStorageLocations().contains(deadNode)) {
- continue;
- }
- if (connectInfo.getComputeLocations().contains(deadNode)
- || connectInfo.getCollectLocations().contains(deadNode)) {
- List<FeedJobInfo> infos = impactedJobs.get(deadNode);
- if (infos == null) {
- infos = new ArrayList<FeedJobInfo>();
- impactedJobs.put(deadNode, infos);
- }
- infos.add(connectInfo);
- connectInfo.setState(FeedJobState.UNDER_RECOVERY);
- feedJobNotificationHandler.deregisterFeedActivity(connectInfo);
- }
- }
-
- }
-
- if (impactedJobs.size() > 0) {
- AddNodeWork addNodeWork = new AddNodeWork(deadNodeIds, deadNodeIds.size(), this);
- feedWorkRequestResponseHandler.registerFeedWork(addNodeWork.getWorkId(), impactedJobs);
- workToBeDone.add(addNodeWork);
- }
- return workToBeDone;
-
- }
-
- public static class FailureReport {
-
- private final List<Pair<FeedConnectJobInfo, List<String>>> recoverableConnectJobs;
- private final Map<IFeedJoint, List<String>> recoverableIntakeFeedIds;
-
- public FailureReport(Map<IFeedJoint, List<String>> recoverableIntakeFeedIds,
- List<Pair<FeedConnectJobInfo, List<String>>> recoverableSubscribers) {
- this.recoverableConnectJobs = recoverableSubscribers;
- this.recoverableIntakeFeedIds = recoverableIntakeFeedIds;
- }
-
- public List<Pair<FeedConnectJobInfo, List<String>>> getRecoverableSubscribers() {
- return recoverableConnectJobs;
- }
-
- public Map<IFeedJoint, List<String>> getRecoverableIntakeFeedIds() {
- return recoverableIntakeFeedIds;
- }
-
- }
-
- @Override
- public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
- ClusterState newState = AsterixClusterProperties.INSTANCE.getState();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(joinedNodeId + " joined the cluster. " + "Asterix state: " + newState);
- }
-
- boolean needToReActivateFeeds = !newState.equals(state) && (newState == ClusterState.ACTIVE);
- if (needToReActivateFeeds) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(joinedNodeId + " Resuming loser feeds (if any)");
- }
- try {
- FeedsActivator activator = new FeedsActivator();
- (new Thread(activator)).start();
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Exception in resuming feeds" + e.getMessage());
- }
- }
- state = newState;
- } else {
- List<FeedCollectInfo> feedsThatCanBeRevived = new ArrayList<FeedCollectInfo>();
- for (Entry<FeedCollectInfo, List<String>> entry : dependentFeeds.entrySet()) {
- List<String> requiredNodeIds = entry.getValue();
- if (requiredNodeIds.contains(joinedNodeId)) {
- requiredNodeIds.remove(joinedNodeId);
- if (requiredNodeIds.isEmpty()) {
- feedsThatCanBeRevived.add(entry.getKey());
- }
- }
- }
- if (!feedsThatCanBeRevived.isEmpty()) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(joinedNodeId + " Resuming feeds after rejoining of node " + joinedNodeId);
- }
- FeedsActivator activator = new FeedsActivator(feedsThatCanBeRevived);
- (new Thread(activator)).start();
- }
- }
- return null;
- }
-
- @Override
- public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
- try {
- responseInbox.put(response);
- } catch (InterruptedException e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Interrupted exception");
- }
- }
- }
-
- @Override
- public void notifyStateChange(ClusterState previousState, ClusterState newState) {
- switch (newState) {
- case ACTIVE:
- if (previousState.equals(ClusterState.UNUSABLE)) {
- try {
- // TODO: Figure out why code was commented
- // FeedsActivator activator = new FeedsActivator();
- // (new Thread(activator)).start();
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Exception in resuming feeds" + e.getMessage());
- }
- }
- }
- break;
- default:
- break;
- }
-
- }
-
- public static class FeedsDeActivator implements Runnable {
-
- private List<FeedConnectJobInfo> failedConnectjobs;
-
- public FeedsDeActivator(List<FeedConnectJobInfo> failedConnectjobs) {
- this.failedConnectjobs = failedConnectjobs;
- }
-
- @Override
- public void run() {
- for (FeedConnectJobInfo failedConnectJob : failedConnectjobs) {
- endFeed(failedConnectJob);
- }
- }
-
- private void endFeed(FeedConnectJobInfo cInfo) {
- MetadataTransactionContext ctx = null;
- PrintWriter writer = new PrintWriter(System.out, true);
- SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
-
- try {
- ctx = MetadataManager.INSTANCE.beginTransaction();
- FeedId feedId = cInfo.getConnectionId().getFeedId();
- DisconnectFeedStatement stmt = new DisconnectFeedStatement(new Identifier(feedId.getDataverse()),
- new Identifier(feedId.getFeedName()), new Identifier(cInfo.getConnectionId().getDatasetName()));
- List<Statement> statements = new ArrayList<Statement>();
- DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(feedId.getDataverse()));
- statements.add(dataverseDecl);
- statements.add(stmt);
- QueryTranslator translator = new QueryTranslator(statements, pc, compilationProvider);
- translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
- QueryTranslator.ResultDelivery.SYNC);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("End irrecoverable feed: " + cInfo.getConnectionId());
- }
- MetadataManager.INSTANCE.commitTransaction(ctx);
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Exception in ending loser feed: " + cInfo.getConnectionId() + " Exception "
- + e.getMessage());
- }
- e.printStackTrace();
- try {
- MetadataManager.INSTANCE.abortTransaction(ctx);
- } catch (Exception e2) {
- e2.addSuppressed(e);
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Exception in aborting transaction! System is in inconsistent state");
- }
- }
-
- }
-
- }
- }
-
public void submitFeedConnectionRequest(IFeedJoint feedPoint, FeedConnectionRequest subscriptionRequest)
throws Exception {
feedJobNotificationHandler.submitFeedConnectionRequest(feedPoint, subscriptionRequest);
@@ -451,21 +192,6 @@
return feedJobNotificationHandler.isFeedConnectionActive(connectionId, eventSubscriber);
}
- public void reportPartialDisconnection(FeedConnectionId connectionId) {
- }
-
- public void registerFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
- feedReportQueue.put(feedId, queue);
- }
-
- public void deregisterFeedReportQueue(FeedConnectionId feedId, LinkedBlockingQueue<String> queue) {
- feedReportQueue.remove(feedId);
- }
-
- public LinkedBlockingQueue<String> getFeedReportQueue(FeedConnectionId feedId) {
- return feedReportQueue.get(feedId);
- }
-
@Override
public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
return feedJobNotificationHandler.getAvailableFeedJoint(feedJointKey);
@@ -495,14 +221,6 @@
}
- public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
- return feedJobNotificationHandler.getCollectJobSpecification(connectionId);
- }
-
- public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
- return feedJobNotificationHandler.getFeedCollectJobId(connectionId);
- }
-
public synchronized void notifyPartitionStart(FeedId feedId, JobId jobId) {
jobEventInbox.add(new FeedEvent(jobId, FeedEvent.EventKind.PARTITION_START, feedId));
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLoadManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLoadManager.java
deleted file mode 100644
index b6be1e7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedLoadManager.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.api.IFeedLoadManager;
-import org.apache.asterix.external.feed.api.IFeedTrackingManager;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedCongestionMessage;
-import org.apache.asterix.external.feed.message.FeedReportMessage;
-import org.apache.asterix.external.feed.message.PrepareStallMessage;
-import org.apache.asterix.external.feed.message.ScaleInReportMessage;
-import org.apache.asterix.external.feed.message.TerminateDataFlowMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.feed.watch.FeedActivity;
-import org.apache.asterix.external.feed.watch.NodeLoadReport;
-import org.apache.asterix.external.feed.watch.FeedJobInfo.FeedJobState;
-import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedLoadManager implements IFeedLoadManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedLoadManager.class.getName());
-
- private static final long MIN_MODIFICATION_INTERVAL = 180000; // 10 seconds
- private final TreeSet<NodeLoadReport> nodeReports;
- private final Map<FeedConnectionId, FeedActivity> feedActivities;
- private final Map<String, Pair<Integer, Integer>> feedMetrics;
-
- private FeedConnectionId lastModified;
- private long lastModifiedTimestamp;
-
- private static final int UNKNOWN = -1;
-
- public FeedLoadManager() {
- this.nodeReports = new TreeSet<NodeLoadReport>();
- this.feedActivities = new HashMap<FeedConnectionId, FeedActivity>();
- this.feedMetrics = new HashMap<String, Pair<Integer, Integer>>();
- }
-
- @Override
- public void submitNodeLoadReport(NodeLoadReport report) {
- nodeReports.remove(report);
- nodeReports.add(report);
- }
-
- @Override
- public void reportCongestion(FeedCongestionMessage message) throws AsterixException {
- FeedRuntimeId runtimeId = message.getRuntimeId();
- FeedJobState jobState = FeedLifecycleListener.INSTANCE.getFeedJobState(message.getConnectionId());
- if (jobState == null
- || (jobState.equals(FeedJobState.UNDER_RECOVERY))
- || (message.getConnectionId().equals(lastModified) && System.currentTimeMillis()
- - lastModifiedTimestamp < MIN_MODIFICATION_INTERVAL)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ignoring congestion report from " + runtimeId);
- }
- return;
- } else {
- try {
- FeedLifecycleListener.INSTANCE.setJobState(message.getConnectionId(), FeedJobState.UNDER_RECOVERY);
- int inflowRate = message.getInflowRate();
- int outflowRate = message.getOutflowRate();
- List<String> currentComputeLocations = new ArrayList<String>();
- currentComputeLocations.addAll(FeedLifecycleListener.INSTANCE.getComputeLocations(message
- .getConnectionId().getFeedId()));
- int computeCardinality = currentComputeLocations.size();
- int requiredCardinality = (int) Math
- .ceil((double) ((computeCardinality * inflowRate) / (double) outflowRate)) + 5;
- int additionalComputeNodes = requiredCardinality - computeCardinality;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("INCREASING COMPUTE CARDINALITY from " + computeCardinality + " by "
- + additionalComputeNodes);
- }
-
- List<String> helperComputeNodes = getNodeForSubstitution(additionalComputeNodes);
-
- // Step 1) Alter the original feed job to adjust the cardinality
- JobSpecification jobSpec = FeedLifecycleListener.INSTANCE.getCollectJobSpecification(message
- .getConnectionId());
- helperComputeNodes.addAll(currentComputeLocations);
- List<String> newLocations = new ArrayList<String>();
- newLocations.addAll(currentComputeLocations);
- newLocations.addAll(helperComputeNodes);
- FeedMetadataUtil.increaseCardinality(jobSpec, FeedRuntimeType.COMPUTE, requiredCardinality, newLocations);
-
- // Step 2) send prepare to stall message
- gracefullyTerminateDataFlow(message.getConnectionId(), Integer.MAX_VALUE);
-
- // Step 3) run the altered job specification
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("New Job after adjusting to the workload " + jobSpec);
- }
-
- Thread.sleep(10000);
- runJob(jobSpec, false);
- lastModified = message.getConnectionId();
- lastModifiedTimestamp = System.currentTimeMillis();
-
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Unable to form the required job for scaling in/out" + e.getMessage());
- }
- throw new AsterixException(e);
- }
- }
- }
-
- @Override
- public void submitScaleInPossibleReport(ScaleInReportMessage message) throws Exception {
- FeedJobState jobState = FeedLifecycleListener.INSTANCE.getFeedJobState(message.getConnectionId());
- if (jobState == null || (jobState.equals(FeedJobState.UNDER_RECOVERY))) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("JobState information for job " + "[" + message.getConnectionId() + "]" + " not found ");
- }
- return;
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Processing scale-in message " + message);
- }
- FeedLifecycleListener.INSTANCE.setJobState(message.getConnectionId(), FeedJobState.UNDER_RECOVERY);
- JobSpecification jobSpec = FeedLifecycleListener.INSTANCE.getCollectJobSpecification(message
- .getConnectionId());
- int reducedCardinality = message.getReducedCardinaliy();
- List<String> currentComputeLocations = new ArrayList<String>();
- currentComputeLocations.addAll(FeedLifecycleListener.INSTANCE.getComputeLocations(message.getConnectionId()
- .getFeedId()));
- FeedMetadataUtil.decreaseComputeCardinality(jobSpec, FeedRuntimeType.COMPUTE, reducedCardinality,
- currentComputeLocations);
-
- gracefullyTerminateDataFlow(message.getConnectionId(), reducedCardinality - 1);
- Thread.sleep(3000);
- JobId newJobId = runJob(jobSpec, false);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Launch modified job" + "[" + newJobId + "]" + "for scale-in \n" + jobSpec);
- }
-
- }
- }
-
- private void gracefullyTerminateDataFlow(FeedConnectionId connectionId, int computePartitionRetainLimit)
- throws Exception {
- // Step 1) send prepare to stall message
- PrepareStallMessage stallMessage = new PrepareStallMessage(connectionId, computePartitionRetainLimit);
- List<String> intakeLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
- List<String> computeLocations = FeedLifecycleListener.INSTANCE.getComputeLocations(connectionId.getFeedId());
- List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
-
- Set<String> operatorLocations = new HashSet<String>();
-
- operatorLocations.addAll(intakeLocations);
- operatorLocations.addAll(computeLocations);
- operatorLocations.addAll(storageLocations);
-
- JobSpecification messageJobSpec = FeedOperations.buildPrepareStallMessageJob(stallMessage, operatorLocations);
- runJob(messageJobSpec, true);
-
- // Step 2)
- TerminateDataFlowMessage terminateMesg = new TerminateDataFlowMessage(connectionId);
- messageJobSpec = FeedOperations.buildTerminateFlowMessageJob(terminateMesg, intakeLocations);
- runJob(messageJobSpec, true);
- }
-
- public static JobId runJob(JobSpecification spec, boolean waitForCompletion) throws Exception {
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- JobId jobId = hcc.startJob(spec);
- if (waitForCompletion) {
- hcc.waitForCompletion(jobId);
- }
- return jobId;
- }
-
- @Override
- public void submitFeedRuntimeReport(FeedReportMessage report) {
- String key = "" + report.getConnectionId() + ":" + report.getRuntimeId().getFeedRuntimeType();
- Pair<Integer, Integer> value = feedMetrics.get(key);
- if (value == null) {
- value = new Pair<Integer, Integer>(report.getValue(), 1);
- feedMetrics.put(key, value);
- } else {
- value.first = value.first + report.getValue();
- value.second = value.second + 1;
- }
- }
-
- @Override
- public int getOutflowRate(FeedConnectionId connectionId, FeedRuntimeType runtimeType) {
- int rVal;
- String key = "" + connectionId + ":" + runtimeType;
- Pair<Integer, Integer> value = feedMetrics.get(key);
- if (value == null) {
- rVal = UNKNOWN;
- } else {
- rVal = value.first / value.second;
- }
- return rVal;
- }
-
- private List<String> getNodeForSubstitution(int nRequired) {
- List<String> nodeIds = new ArrayList<String>();
- Iterator<NodeLoadReport> it = null;
- int nAdded = 0;
- while (nAdded < nRequired) {
- it = nodeReports.iterator();
- while (it.hasNext()) {
- nodeIds.add(it.next().getNodeId());
- nAdded++;
- }
- }
- return nodeIds;
- }
-
- @Override
- public synchronized List<String> getNodes(int required) {
- Iterator<NodeLoadReport> it;
- List<String> allocated = new ArrayList<String>();
- while (allocated.size() < required) {
- it = nodeReports.iterator();
- while (it.hasNext() && allocated.size() < required) {
- allocated.add(it.next().getNodeId());
- }
- }
- return allocated;
- }
-
- @Override
- public void reportThrottlingEnabled(ThrottlingEnabledFeedMessage mesg) throws AsterixException, Exception {
- System.out.println("Throttling Enabled for " + mesg.getConnectionId() + " " + mesg.getFeedRuntimeId());
- FeedConnectionId connectionId = mesg.getConnectionId();
- List<String> destinationLocations = new ArrayList<String>();
- List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
- List<String> collectLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
-
- destinationLocations.addAll(storageLocations);
- destinationLocations.addAll(collectLocations);
- JobSpecification messageJobSpec = FeedOperations.buildNotifyThrottlingEnabledMessageJob(mesg,
- destinationLocations);
- runJob(messageJobSpec, true);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.warning("Acking disabled for " + mesg.getConnectionId() + " in view of activated throttling");
- }
- IFeedTrackingManager trackingManager = CentralFeedManager.getInstance().getFeedTrackingManager();
- trackingManager.disableAcking(connectionId);
- }
-
- @Override
- public void reportFeedActivity(FeedConnectionId connectionId, FeedActivity activity) {
- feedActivities.put(connectionId, activity);
- }
-
- @Override
- public FeedActivity getFeedActivity(FeedConnectionId connectionId) {
- return feedActivities.get(connectionId);
- }
-
- @Override
- public Collection<FeedActivity> getFeedActivities() {
- return feedActivities.values();
- }
-
- @Override
- public void removeFeedActivity(FeedConnectionId connectionId) {
- feedActivities.remove(connectionId);
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedMessageReceiver.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedMessageReceiver.java
deleted file mode 100644
index bff1a4d..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedMessageReceiver.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.util.logging.Level;
-
-import org.apache.asterix.app.external.CentralFeedManager.AQLExecutor;
-import org.apache.asterix.external.feed.api.IFeedLoadManager;
-import org.apache.asterix.external.feed.api.IFeedMessage.MessageType;
-import org.apache.asterix.external.feed.api.IFeedTrackingManager;
-import org.apache.asterix.external.feed.message.FeedCongestionMessage;
-import org.apache.asterix.external.feed.message.FeedReportMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitAckMessage;
-import org.apache.asterix.external.feed.message.MessageReceiver;
-import org.apache.asterix.external.feed.message.ScaleInReportMessage;
-import org.apache.asterix.external.feed.message.StorageReportFeedMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
-import org.apache.asterix.external.feed.watch.NodeLoadReport;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.hyracks.bootstrap.FeedBootstrap;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.json.JSONObject;
-
-public class FeedMessageReceiver extends MessageReceiver<String> {
-
- private static boolean initialized;
-
- private final IFeedLoadManager feedLoadManager;
- private final IFeedTrackingManager feedTrackingManager;
-
- public FeedMessageReceiver(CentralFeedManager centralFeedManager) {
- this.feedLoadManager = centralFeedManager.getFeedLoadManager();
- this.feedTrackingManager = centralFeedManager.getFeedTrackingManager();
- }
-
- @Override
- public void processMessage(String message) throws Exception {
- JSONObject obj = new JSONObject(message);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Received message " + obj);
- }
- MessageType messageType = MessageType.valueOf(obj.getString(FeedConstants.MessageConstants.MESSAGE_TYPE));
- switch (messageType) {
- case XAQL:
- if (!initialized) {
- FeedBootstrap.setUpInitialArtifacts();
- initialized = true;
- }
- AQLExecutor.executeAQL(obj.getString(FeedConstants.MessageConstants.AQL));
- break;
- case CONGESTION:
- feedLoadManager.reportCongestion(FeedCongestionMessage.read(obj));
- break;
- case FEED_REPORT:
- feedLoadManager.submitFeedRuntimeReport(FeedReportMessage.read(obj));
- break;
- case NODE_REPORT:
- feedLoadManager.submitNodeLoadReport(NodeLoadReport.read(obj));
- break;
- case SCALE_IN_REQUEST:
- feedLoadManager.submitScaleInPossibleReport(ScaleInReportMessage.read(obj));
- break;
- case STORAGE_REPORT:
- FeedLifecycleListener.INSTANCE.updateTrackingInformation(StorageReportFeedMessage.read(obj));
- break;
- case COMMIT_ACK:
- feedTrackingManager.submitAckReport(FeedTupleCommitAckMessage.read(obj));
- break;
- case THROTTLING_ENABLED:
- feedLoadManager.reportThrottlingEnabled(ThrottlingEnabledFeedMessage.read(obj));
- default:
- break;
- }
-
- }
-
- @Override
- public void emptyInbox() throws HyracksDataException {
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
index d8f1893..2de0266 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedOperations.java
@@ -32,10 +32,6 @@
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.message.EndFeedMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitResponseMessage;
-import org.apache.asterix.external.feed.message.PrepareStallMessage;
-import org.apache.asterix.external.feed.message.TerminateDataFlowMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
import org.apache.asterix.external.operators.FeedMessageOperatorDescriptor;
@@ -92,29 +88,6 @@
return new Pair<JobSpecification, IAdapterFactory>(spec, adapterFactory);
}
- public static JobSpecification buildDiscontinueFeedSourceSpec(AqlMetadataProvider metadataProvider, FeedId feedId)
- throws AsterixException, AlgebricksException {
-
- JobSpecification spec = JobSpecificationUtils.createJobSpecification();
- IOperatorDescriptor feedMessenger = null;
- AlgebricksPartitionConstraint messengerPc = null;
-
- List<String> locations = FeedLifecycleListener.INSTANCE.getIntakeLocations(feedId);
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = buildDiscontinueFeedMessengerRuntime(spec, feedId,
- locations);
-
- feedMessenger = p.first;
- messengerPc = p.second;
-
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedMessenger, messengerPc);
- NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, messengerPc);
- spec.connect(new OneToOneConnectorDescriptor(spec), feedMessenger, 0, nullSink, 0);
- spec.addRoot(nullSink);
-
- return spec;
- }
-
/**
* Builds the job spec for sending message to an active feed to disconnect it from the
* its source.
@@ -162,66 +135,6 @@
}
- public static JobSpecification buildPrepareStallMessageJob(PrepareStallMessage stallMessage,
- Collection<String> collectLocations) throws AsterixException {
- JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
- try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
- messageJobSpec, stallMessage.getConnectionId(), stallMessage, collectLocations);
- buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
- } catch (AlgebricksException ae) {
- throw new AsterixException(ae);
- }
- return messageJobSpec;
- }
-
- public static JobSpecification buildNotifyThrottlingEnabledMessageJob(
- ThrottlingEnabledFeedMessage throttlingEnabledMesg, Collection<String> locations) throws AsterixException {
- JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
- try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
- messageJobSpec, throttlingEnabledMesg.getConnectionId(), throttlingEnabledMesg, locations);
- buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
- } catch (AlgebricksException ae) {
- throw new AsterixException(ae);
- }
- return messageJobSpec;
- }
-
- public static JobSpecification buildTerminateFlowMessageJob(TerminateDataFlowMessage terminateMessage,
- List<String> collectLocations) throws AsterixException {
- JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
- try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
- messageJobSpec, terminateMessage.getConnectionId(), terminateMessage, collectLocations);
- buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
- } catch (AlgebricksException ae) {
- throw new AsterixException(ae);
- }
- return messageJobSpec;
- }
-
- public static JobSpecification buildCommitAckResponseJob(FeedTupleCommitResponseMessage commitResponseMessage,
- Collection<String> targetLocations) throws AsterixException {
- JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
- try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
- messageJobSpec, commitResponseMessage.getConnectionId(), commitResponseMessage, targetLocations);
- buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
- } catch (AlgebricksException ae) {
- throw new AsterixException(ae);
- }
- return messageJobSpec;
- }
-
- public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDiscontinueFeedMessengerRuntime(
- JobSpecification jobSpec, FeedId feedId, List<String> locations) throws AlgebricksException {
- FeedConnectionId feedConnectionId = new FeedConnectionId(feedId, null);
- IFeedMessage feedMessage = new EndFeedMessage(feedConnectionId, FeedRuntimeType.INTAKE,
- feedConnectionId.getFeedId(), true, EndFeedMessage.EndMessageType.DISCONTINUE_SOURCE);
- return buildSendFeedMessageRuntime(jobSpec, feedConnectionId, feedMessage, locations);
- }
-
private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
JobSpecification jobSpec, FeedConnectionId feedConenctionId, IFeedMessage feedMessage,
Collection<String> locations) throws AlgebricksException {
@@ -232,17 +145,6 @@
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
}
- private static JobSpecification buildSendFeedMessageJobSpec(IOperatorDescriptor operatorDescriptor,
- AlgebricksPartitionConstraint messengerPc, JobSpecification messageJobSpec) {
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(messageJobSpec, operatorDescriptor,
- messengerPc);
- NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(messageJobSpec);
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(messageJobSpec, nullSink, messengerPc);
- messageJobSpec.connect(new OneToOneConnectorDescriptor(messageJobSpec), operatorDescriptor, 0, nullSink, 0);
- messageJobSpec.addRoot(nullSink);
- return messageJobSpec;
- }
-
private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
JobSpecification jobSpec, FeedConnectionId feedConenctionId, List<String> locations,
FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, FeedId sourceFeedId)
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedTrackingManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedTrackingManager.java
deleted file mode 100644
index 29230c1..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedTrackingManager.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedTrackingManager;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedTupleCommitAckMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitResponseMessage;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedTrackingManager implements IFeedTrackingManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedTrackingManager.class.getName());
-
- private final BitSet allOnes;
-
- private Map<FeedConnectionId, Map<AckId, BitSet>> ackHistory;
- private Map<FeedConnectionId, Map<AckId, Integer>> maxBaseAcked;
-
- public FeedTrackingManager() {
- byte[] allOneBytes = new byte[128];
- Arrays.fill(allOneBytes, (byte) 0xff);
- allOnes = BitSet.valueOf(allOneBytes);
- ackHistory = new HashMap<FeedConnectionId, Map<AckId, BitSet>>();
- maxBaseAcked = new HashMap<FeedConnectionId, Map<AckId, Integer>>();
- }
-
- @Override
- public synchronized void submitAckReport(FeedTupleCommitAckMessage ackMessage) {
- AckId ackId = getAckId(ackMessage);
- Map<AckId, BitSet> acksForConnection = ackHistory.get(ackMessage.getConnectionId());
- if (acksForConnection == null) {
- acksForConnection = new HashMap<AckId, BitSet>();
- acksForConnection.put(ackId, BitSet.valueOf(ackMessage.getCommitAcks()));
- ackHistory.put(ackMessage.getConnectionId(), acksForConnection);
- }
- BitSet currentAcks = acksForConnection.get(ackId);
- if (currentAcks == null) {
- currentAcks = BitSet.valueOf(ackMessage.getCommitAcks());
- acksForConnection.put(ackId, currentAcks);
- } else {
- currentAcks.or(BitSet.valueOf(ackMessage.getCommitAcks()));
- }
- if (Arrays.equals(currentAcks.toByteArray(), allOnes.toByteArray())) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(ackMessage.getIntakePartition() + " (" + ackMessage.getBase() + ")" + " is convered");
- }
- Map<AckId, Integer> maxBaseAckedForConnection = maxBaseAcked.get(ackMessage.getConnectionId());
- if (maxBaseAckedForConnection == null) {
- maxBaseAckedForConnection = new HashMap<AckId, Integer>();
- maxBaseAcked.put(ackMessage.getConnectionId(), maxBaseAckedForConnection);
- }
- Integer maxBaseAckedValue = maxBaseAckedForConnection.get(ackId);
- if (maxBaseAckedValue == null) {
- maxBaseAckedValue = ackMessage.getBase();
- maxBaseAckedForConnection.put(ackId, ackMessage.getBase());
- sendCommitResponseMessage(ackMessage.getConnectionId(), ackMessage.getIntakePartition(),
- ackMessage.getBase());
- } else if (ackMessage.getBase() == maxBaseAckedValue + 1) {
- maxBaseAckedForConnection.put(ackId, ackMessage.getBase());
- sendCommitResponseMessage(ackMessage.getConnectionId(), ackMessage.getIntakePartition(),
- ackMessage.getBase());
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ignoring discountiuous acked base " + ackMessage.getBase() + " for " + ackId);
- }
- }
-
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("AckId " + ackId + " pending number of acks " + (128 * 8 - currentAcks.cardinality()));
- }
- }
- }
-
- public synchronized void disableTracking(FeedConnectionId connectionId) {
- ackHistory.remove(connectionId);
- maxBaseAcked.remove(connectionId);
- }
-
- private void sendCommitResponseMessage(FeedConnectionId connectionId, int partition, int base) {
- FeedTupleCommitResponseMessage response = new FeedTupleCommitResponseMessage(connectionId, partition, base);
- List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
- List<String> collectLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
- String collectLocation = collectLocations.get(partition);
- Set<String> messageDestinations = new HashSet<String>();
- messageDestinations.add(collectLocation);
- messageDestinations.addAll(storageLocations);
- try {
- JobSpecification spec = FeedOperations.buildCommitAckResponseJob(response, messageDestinations);
- CentralFeedManager.runJob(spec, false);
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to send commit response message " + response + " exception " + e.getMessage());
- }
- }
- }
-
- private static AckId getAckId(FeedTupleCommitAckMessage ackMessage) {
- return new AckId(ackMessage.getConnectionId(), ackMessage.getIntakePartition(), ackMessage.getBase());
- }
-
- private static class AckId {
- private FeedConnectionId connectionId;
- private int intakePartition;
- private int base;
-
- public AckId(FeedConnectionId connectionId, int intakePartition, int base) {
- this.connectionId = connectionId;
- this.intakePartition = intakePartition;
- this.base = base;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof AckId)) {
- return false;
- }
- AckId other = (AckId) o;
- return other.getConnectionId().equals(connectionId) && other.getIntakePartition() == intakePartition
- && other.getBase() == base;
- }
-
- @Override
- public String toString() {
- return connectionId + "[" + intakePartition + "]" + "(" + base + ")";
- }
-
- @Override
- public int hashCode() {
- return toString().hashCode();
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public int getIntakePartition() {
- return intakePartition;
- }
-
- public int getBase() {
- return base;
- }
-
- }
-
- @Override
- public void disableAcking(FeedConnectionId connectionId) {
- ackHistory.remove(connectionId);
- maxBaseAcked.remove(connectionId);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Acking disabled for " + connectionId);
- }
- }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
index 2f338d0..a807515 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkCollection.java
@@ -21,8 +21,6 @@
import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
import org.apache.asterix.api.common.SessionConfig;
import org.apache.asterix.api.common.SessionConfig.OutputFormat;
@@ -31,7 +29,6 @@
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.external.feed.api.IFeedWork;
import org.apache.asterix.external.feed.api.IFeedWorkEventListener;
-import org.apache.asterix.external.feed.management.FeedCollectInfo;
import org.apache.asterix.external.feed.management.FeedConnectionRequest;
import org.apache.asterix.external.feed.management.FeedConnectionRequest.ConnectionStatus;
import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
@@ -39,7 +36,8 @@
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.api.job.JobId;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
/**
* A collection of feed management related task, each represented as an implementation of {@code IFeedWork}.
@@ -81,7 +79,8 @@
@Override
public void run() {
try {
- PrintWriter writer = new PrintWriter(System.out, true);
+ //TODO(amoudi): route PrintWriter to log file
+ PrintWriter writer = new PrintWriter(System.err, true);
SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
DataverseDecl dataverseDecl = new DataverseDecl(
new Identifier(request.getReceivingFeedId().getDataverse()));
@@ -92,12 +91,12 @@
QueryTranslator translator = new QueryTranslator(statements, pc, compilationProvider);
translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
QueryTranslator.ResultDelivery.SYNC);
- if (LOGGER.isLoggable(Level.INFO)) {
+ if (LOGGER.isEnabledFor(Level.INFO)) {
LOGGER.info("Submitted connection requests for execution: " + request);
}
} catch (Exception e) {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Exception in executing " + request);
+ if (LOGGER.isEnabledFor(Level.FATAL)) {
+ LOGGER.fatal("Exception in executing " + request);
}
}
}
@@ -107,8 +106,8 @@
@Override
public void workFailed(IFeedWork work, Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning(" Feed subscription request " + ((SubscribeFeedWork) work).request
+ if (LOGGER.isEnabledFor(Level.WARN)) {
+ LOGGER.warn(" Feed subscription request " + ((SubscribeFeedWork) work).request
+ " failed with exception " + e);
}
}
@@ -116,8 +115,8 @@
@Override
public void workCompleted(IFeedWork work) {
((SubscribeFeedWork) work).request.setSubscriptionStatus(ConnectionStatus.ACTIVE);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.warning(" Feed subscription request " + ((SubscribeFeedWork) work).request + " completed ");
+ if (LOGGER.isEnabledFor(Level.INFO)) {
+ LOGGER.info(" Feed subscription request " + ((SubscribeFeedWork) work).request + " completed ");
}
}
@@ -131,75 +130,5 @@
public String toString() {
return "SubscribeFeedWork for [" + request + "]";
}
-
- }
-
- /**
- * The task of activating a set of feeds.
- */
- public static class ActivateFeedWork implements IFeedWork {
-
- private final Runnable runnable;
-
- @Override
- public Runnable getRunnable() {
- return runnable;
- }
-
- public ActivateFeedWork(List<FeedCollectInfo> feedsToRevive) {
- this.runnable = new FeedsActivateRunnable(feedsToRevive);
- }
-
- public ActivateFeedWork() {
- this.runnable = new FeedsActivateRunnable();
- }
-
- private static class FeedsActivateRunnable implements Runnable {
-
- private List<FeedCollectInfo> feedsToRevive;
- private Mode mode;
-
- public enum Mode {
- REVIVAL_POST_NODE_REJOIN
- }
-
- public FeedsActivateRunnable(List<FeedCollectInfo> feedsToRevive) {
- this.feedsToRevive = feedsToRevive;
- }
-
- public FeedsActivateRunnable() {
- }
-
- @Override
- public void run() {
- switch (mode) {
- case REVIVAL_POST_NODE_REJOIN:
- try {
- Thread.sleep(10000);
- } catch (InterruptedException e1) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Attempt to resume feed interrupted");
- }
- throw new IllegalStateException(e1.getMessage());
- }
- for (FeedCollectInfo finfo : feedsToRevive) {
- try {
- JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(finfo.jobSpec);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Resumed feed :" + finfo.feedConnectionId + " job id " + jobId);
- LOGGER.info("Job:" + finfo.jobSpec);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning(
- "Unable to resume feed " + finfo.feedConnectionId + " " + e.getMessage());
- }
- }
- }
- }
- }
-
- }
-
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkRequestResponseHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkRequestResponseHandler.java
deleted file mode 100644
index 2dc1162..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedWorkRequestResponseHandler.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.api.IClusterManagementWork;
-import org.apache.asterix.common.api.IClusterManagementWorkResponse;
-import org.apache.asterix.external.feed.watch.FeedConnectJobInfo;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
-import org.apache.asterix.external.feed.watch.FeedJobInfo;
-import org.apache.asterix.metadata.cluster.AddNodeWork;
-import org.apache.asterix.metadata.cluster.AddNodeWorkResponse;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.asterix.om.util.AsterixClusterProperties;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.constraints.Constraint;
-import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
-import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
-import org.apache.hyracks.api.constraints.expressions.ConstraintExpression;
-import org.apache.hyracks.api.constraints.expressions.ConstraintExpression.ExpressionTag;
-import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
-import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
-import org.apache.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.OperatorDescriptorId;
-import org.apache.hyracks.api.job.JobSpecification;
-
-public class FeedWorkRequestResponseHandler implements Runnable {
-
- private static final Logger LOGGER = Logger.getLogger(FeedWorkRequestResponseHandler.class.getName());
-
- private final LinkedBlockingQueue<IClusterManagementWorkResponse> inbox;
-
- private Map<Integer, Map<String, List<FeedJobInfo>>> feedsWaitingForResponse = new HashMap<Integer, Map<String, List<FeedJobInfo>>>();
-
- public FeedWorkRequestResponseHandler(LinkedBlockingQueue<IClusterManagementWorkResponse> inbox) {
- this.inbox = inbox;
- }
-
- @Override
- public void run() {
- while (true) {
- IClusterManagementWorkResponse response = null;
- try {
- response = inbox.take();
- } catch (InterruptedException e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Interrupted exception " + e.getMessage());
- }
- }
- IClusterManagementWork submittedWork = response.getWork();
- Map<String, String> nodeSubstitution = new HashMap<String, String>();
- switch (submittedWork.getClusterManagementWorkType()) {
- case ADD_NODE:
- AddNodeWork addNodeWork = (AddNodeWork) submittedWork;
- int workId = addNodeWork.getWorkId();
- Map<String, List<FeedJobInfo>> failureAnalysis = feedsWaitingForResponse.get(workId);
- AddNodeWorkResponse resp = (AddNodeWorkResponse) response;
- List<String> nodesAdded = resp.getNodesAdded();
- List<String> unsubstitutedNodes = new ArrayList<String>();
- unsubstitutedNodes.addAll(addNodeWork.getDeadNodes());
- int nodeIndex = 0;
-
- /** form a mapping between the failed node and its substitute **/
- if (nodesAdded != null && nodesAdded.size() > 0) {
- for (String failedNodeId : addNodeWork.getDeadNodes()) {
- String substitute = nodesAdded.get(nodeIndex);
- nodeSubstitution.put(failedNodeId, substitute);
- nodeIndex = (nodeIndex + 1) % nodesAdded.size();
- unsubstitutedNodes.remove(failedNodeId);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Node " + substitute + " chosen to substiute lost node " + failedNodeId);
- }
- }
- }
- if (unsubstitutedNodes.size() > 0) {
- String[] participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes()
- .toArray(new String[] {});
- nodeIndex = 0;
- for (String unsubstitutedNode : unsubstitutedNodes) {
- nodeSubstitution.put(unsubstitutedNode, participantNodes[nodeIndex]);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Node " + participantNodes[nodeIndex] + " chosen to substiute lost node "
- + unsubstitutedNode);
- }
- nodeIndex = (nodeIndex + 1) % participantNodes.length;
- }
-
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Request " + resp.getWork() + " completed using internal nodes");
- }
- }
-
- // alter failed feed intake jobs
-
- for (Entry<String, List<FeedJobInfo>> entry : failureAnalysis.entrySet()) {
- String failedNode = entry.getKey();
- List<FeedJobInfo> impactedJobInfos = entry.getValue();
- for (FeedJobInfo info : impactedJobInfos) {
- JobSpecification spec = info.getSpec();
- replaceNode(spec, failedNode, nodeSubstitution.get(failedNode));
- info.setSpec(spec);
- }
- }
-
- Set<FeedIntakeInfo> revisedIntakeJobs = new HashSet<FeedIntakeInfo>();
- Set<FeedConnectJobInfo> revisedConnectJobInfos = new HashSet<FeedConnectJobInfo>();
-
- for (List<FeedJobInfo> infos : failureAnalysis.values()) {
- for (FeedJobInfo info : infos) {
- switch (info.getJobType()) {
- case INTAKE:
- revisedIntakeJobs.add((FeedIntakeInfo) info);
- break;
- case FEED_CONNECT:
- revisedConnectJobInfos.add((FeedConnectJobInfo) info);
- break;
- }
- }
- }
-
- IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
- try {
- for (FeedIntakeInfo info : revisedIntakeJobs) {
- hcc.startJob(info.getSpec());
- }
- Thread.sleep(2000);
- for (FeedConnectJobInfo info : revisedConnectJobInfos) {
- hcc.startJob(info.getSpec());
- Thread.sleep(2000);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start revised job post failure");
- }
- }
-
- break;
- case REMOVE_NODE:
- throw new IllegalStateException("Invalid work submitted");
- }
- }
- }
-
- private void replaceNode(JobSpecification jobSpec, String failedNodeId, String replacementNode) {
- Set<Constraint> userConstraints = jobSpec.getUserConstraints();
- List<Constraint> locationConstraintsToReplace = new ArrayList<Constraint>();
- List<Constraint> countConstraintsToReplace = new ArrayList<Constraint>();
- List<OperatorDescriptorId> modifiedOperators = new ArrayList<OperatorDescriptorId>();
- Map<OperatorDescriptorId, List<Constraint>> candidateConstraints = new HashMap<OperatorDescriptorId, List<Constraint>>();
- Map<OperatorDescriptorId, Map<Integer, String>> newConstraints = new HashMap<OperatorDescriptorId, Map<Integer, String>>();
- OperatorDescriptorId opId = null;
- for (Constraint constraint : userConstraints) {
- LValueConstraintExpression lexpr = constraint.getLValue();
- ConstraintExpression cexpr = constraint.getRValue();
- switch (lexpr.getTag()) {
- case PARTITION_COUNT:
- opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
- if (modifiedOperators.contains(opId)) {
- countConstraintsToReplace.add(constraint);
- } else {
- List<Constraint> clist = candidateConstraints.get(opId);
- if (clist == null) {
- clist = new ArrayList<Constraint>();
- candidateConstraints.put(opId, clist);
- }
- clist.add(constraint);
- }
- break;
- case PARTITION_LOCATION:
- opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
- String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
- if (oldLocation.equals(failedNodeId)) {
- locationConstraintsToReplace.add(constraint);
- modifiedOperators.add(((PartitionLocationExpression) lexpr).getOperatorDescriptorId());
- Map<Integer, String> newLocs = newConstraints.get(opId);
- if (newLocs == null) {
- newLocs = new HashMap<Integer, String>();
- newConstraints.put(opId, newLocs);
- }
- int partition = ((PartitionLocationExpression) lexpr).getPartition();
- newLocs.put(partition, replacementNode);
- } else {
- if (modifiedOperators.contains(opId)) {
- locationConstraintsToReplace.add(constraint);
- Map<Integer, String> newLocs = newConstraints.get(opId);
- if (newLocs == null) {
- newLocs = new HashMap<Integer, String>();
- newConstraints.put(opId, newLocs);
- }
- int partition = ((PartitionLocationExpression) lexpr).getPartition();
- newLocs.put(partition, oldLocation);
- } else {
- List<Constraint> clist = candidateConstraints.get(opId);
- if (clist == null) {
- clist = new ArrayList<Constraint>();
- candidateConstraints.put(opId, clist);
- }
- clist.add(constraint);
- }
- }
- break;
- default:
- break;
- }
- }
-
- jobSpec.getUserConstraints().removeAll(locationConstraintsToReplace);
- jobSpec.getUserConstraints().removeAll(countConstraintsToReplace);
-
- for (OperatorDescriptorId mopId : modifiedOperators) {
- List<Constraint> clist = candidateConstraints.get(mopId);
- if (clist != null && !clist.isEmpty()) {
- jobSpec.getUserConstraints().removeAll(clist);
-
- for (Constraint c : clist) {
- if (c.getLValue().getTag().equals(ExpressionTag.PARTITION_LOCATION)) {
- ConstraintExpression cexpr = c.getRValue();
- int partition = ((PartitionLocationExpression) c.getLValue()).getPartition();
- String oldLocation = (String) ((ConstantExpression) cexpr).getValue();
- newConstraints.get(mopId).put(partition, oldLocation);
- }
- }
- }
- }
-
- for (Entry<OperatorDescriptorId, Map<Integer, String>> entry : newConstraints.entrySet()) {
- OperatorDescriptorId nopId = entry.getKey();
- Map<Integer, String> clist = entry.getValue();
- IOperatorDescriptor op = jobSpec.getOperatorMap().get(nopId);
- String[] locations = new String[clist.size()];
- for (int i = 0; i < locations.length; i++) {
- locations[i] = clist.get(i);
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, op, locations);
- }
-
- }
-
- public void registerFeedWork(int workId, Map<String, List<FeedJobInfo>> impactedJobs) {
- feedsWaitingForResponse.put(workId, impactedJobs);
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedsActivator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedsActivator.java
deleted file mode 100644
index 5a6d28e..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/FeedsActivator.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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.asterix.app.external;
-
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.api.common.SessionConfig;
-import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.aql.translator.QueryTranslator;
-import org.apache.asterix.compiler.provider.AqlCompilationProvider;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.external.feed.management.FeedCollectInfo;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
-import org.apache.asterix.lang.common.statement.DataverseDecl;
-import org.apache.asterix.lang.common.struct.Identifier;
-import org.apache.asterix.om.util.AsterixAppContextInfo;
-import org.apache.hyracks.api.job.JobId;
-
-public class FeedsActivator implements Runnable {
-
- private static final Logger LOGGER = Logger.getLogger(FeedJobNotificationHandler.class.getName());
- private static final ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
-
- private List<FeedCollectInfo> feedsToRevive;
- private Mode mode;
-
- public enum Mode {
- REVIVAL_POST_CLUSTER_REBOOT,
- REVIVAL_POST_NODE_REJOIN
- }
-
- public FeedsActivator() {
- this.mode = Mode.REVIVAL_POST_CLUSTER_REBOOT;
- }
-
- public FeedsActivator(List<FeedCollectInfo> feedsToRevive) {
- this.feedsToRevive = feedsToRevive;
- this.mode = Mode.REVIVAL_POST_NODE_REJOIN;
- }
-
- @Override
- public void run() {
- switch (mode) {
- case REVIVAL_POST_CLUSTER_REBOOT:
- //revivePostClusterReboot();
- break;
- case REVIVAL_POST_NODE_REJOIN:
- try {
- Thread.sleep(10000);
- } catch (InterruptedException e1) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Attempt to resume feed interrupted");
- }
- throw new IllegalStateException(e1.getMessage());
- }
- for (FeedCollectInfo finfo : feedsToRevive) {
- try {
- JobId jobId = AsterixAppContextInfo.getInstance().getHcc().startJob(finfo.jobSpec);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Resumed feed :" + finfo.feedConnectionId + " job id " + jobId);
- LOGGER.info("Job:" + finfo.jobSpec);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to resume feed " + finfo.feedConnectionId + " " + e.getMessage());
- }
- }
- }
- }
- }
-
- public void reviveFeed(String dataverse, String feedName, String dataset, String feedPolicy) {
- PrintWriter writer = new PrintWriter(System.out, true);
- SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
- try {
- DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(dataverse));
- ConnectFeedStatement stmt = new ConnectFeedStatement(new Identifier(dataverse), new Identifier(feedName),
- new Identifier(dataset), feedPolicy, 0);
- stmt.setForceConnect(true);
- List<Statement> statements = new ArrayList<Statement>();
- statements.add(dataverseDecl);
- statements.add(stmt);
- QueryTranslator translator = new QueryTranslator(statements, pc, compilationProvider);
- translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
- QueryTranslator.ResultDelivery.SYNC);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Resumed feed: " + dataverse + ":" + dataset + " using policy " + feedPolicy);
- }
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Exception in resuming loser feed: " + dataverse + ":" + dataset + " using policy "
- + feedPolicy + " Exception " + e.getMessage());
- }
- }
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
index 132c8c9..7a7aa40 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/aql/translator/QueryTranslator.java
@@ -40,7 +40,6 @@
import org.apache.asterix.api.common.APIFramework;
import org.apache.asterix.api.common.SessionConfig;
import org.apache.asterix.api.common.SessionConfig.OutputFormat;
-import org.apache.asterix.app.external.CentralFeedManager;
import org.apache.asterix.app.external.ExternalIndexingOperations;
import org.apache.asterix.app.external.FeedJoint;
import org.apache.asterix.app.external.FeedLifecycleListener;
@@ -268,8 +267,7 @@
}
validateOperation(activeDefaultDataverse, stmt);
rewriteStatement(stmt); // Rewrite the statement's AST.
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse,
- CentralFeedManager.getInstance());
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse);
metadataProvider.setWriterFactory(writerFactory);
metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
metadataProvider.setOutputFile(outputFile);
@@ -2366,18 +2364,12 @@
throw new AsterixException(
"Unknown dataset :" + cfs.getDatasetName().getValue() + " in dataverse " + dataverseName);
}
-
Pair<JobSpecification, Boolean> specDisconnectType = FeedOperations
.buildDisconnectFeedJobSpec(metadataProvider, connectionId);
JobSpecification jobSpec = specDisconnectType.first;
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
JobUtils.runJob(hcc, jobSpec, true);
-
- if (!specDisconnectType.second) {
- CentralFeedManager.getInstance().getFeedLoadManager().removeFeedActivity(connectionId);
- FeedLifecycleListener.INSTANCE.reportPartialDisconnection(connectionId);
- }
eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_COLLECT_ENDED);
} catch (Exception e) {
if (bActiveTxn) {
@@ -2418,9 +2410,7 @@
String dataset = feedConnectionId.getDatasetName();
MetadataLockManager.INSTANCE.subscribeFeedBegin(dataverse, dataverse + "." + dataset,
dataverse + "." + feedConnectionId.getFeedId().getFeedName());
-
try {
-
JobSpecification alteredJobSpec = FeedMetadataUtil.alterJobSpecificationForFeed(compiled, feedConnectionId,
bfs.getSubscriptionRequest().getPolicyParameters());
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index e683ef4..4c42594 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -34,7 +34,6 @@
import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
import org.apache.asterix.api.http.servlet.VersionAPIServlet;
-import org.apache.asterix.app.external.CentralFeedManager;
import org.apache.asterix.app.external.ExternalLibraryUtils;
import org.apache.asterix.app.external.FeedLifecycleListener;
import org.apache.asterix.common.api.AsterixThreadFactory;
@@ -45,7 +44,6 @@
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.apache.asterix.event.service.ILookupService;
-import org.apache.asterix.external.feed.api.ICentralFeedManager;
import org.apache.asterix.messaging.CCMessageBroker;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.api.IAsterixStateProxy;
@@ -73,7 +71,6 @@
private Server webServer;
private Server jsonAPIServer;
private Server feedServer;
- private ICentralFeedManager centralFeedManager;
private static IAsterixStateProxy proxy;
private ICCApplicationContext appCtx;
@@ -110,11 +107,7 @@
setupFeedServer(externalProperties);
feedServer.start();
-
ExternalLibraryUtils.setUpExternaLibraries(false);
- centralFeedManager = CentralFeedManager.getInstance();
- centralFeedManager.start();
-
ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.INSTANCE);
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index 4514fee..26f4bc8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -78,9 +78,7 @@
Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
for (IClusterEventsSubscriber sub : subscribers) {
Set<IClusterManagementWork> workRequest = sub.notifyNodeJoin(nodeId);
- if (workRequest != null && !workRequest.isEmpty()) {
- work.addAll(workRequest);
- }
+ work.addAll(workRequest);
}
if (!work.isEmpty()) {
executeWorkSet(work);
@@ -104,9 +102,7 @@
Set<IClusterManagementWork> work = new HashSet<IClusterManagementWork>();
for (IClusterEventsSubscriber sub : subscribers) {
Set<IClusterManagementWork> workRequest = sub.notifyNodeFailure(deadNodeIds);
- if (workRequest != null && !workRequest.isEmpty()) {
- work.addAll(workRequest);
- }
+ work.addAll(workRequest);
}
if (!work.isEmpty()) {
executeWorkSet(work);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/FeedBootstrap.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/FeedBootstrap.java
deleted file mode 100644
index a6be075..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/FeedBootstrap.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.asterix.hyracks.bootstrap;
-
-import org.apache.asterix.app.external.CentralFeedManager;
-import org.apache.asterix.common.config.MetadataConstants;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-
-public class FeedBootstrap {
-
- public static void setUpInitialArtifacts() throws Exception {
-
- StringBuilder builder = new StringBuilder();
- try {
- builder.append("create dataverse " + FeedConstants.FEEDS_METADATA_DV + ";" + "\n");
- builder.append("use dataverse " + FeedConstants.FEEDS_METADATA_DV + ";" + "\n");
- builder.append("create type " + FeedConstants.FAILED_TUPLE_DATASET_TYPE + " as open { ");
- String[] fieldNames = new String[] { "id", "dataverseName", "feedName", "targetDataset", "tuple", "message",
- "timestamp" };
- IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
-
- for (int i = 0; i < fieldNames.length; i++) {
- if (i > 0) {
- builder.append(",");
- }
- builder.append(fieldNames[i] + ":");
- builder.append(fieldTypes[i].getTypeName());
- }
- builder.append("}" + ";" + "\n");
- builder.append("create dataset " + FeedConstants.FAILED_TUPLE_DATASET + " " + "("
- + FeedConstants.FAILED_TUPLE_DATASET_TYPE + ")" + " " + "primary key "
- + FeedConstants.FAILED_TUPLE_DATASET_KEY + " on " + MetadataConstants.METADATA_NODEGROUP_NAME
- + ";");
-
- CentralFeedManager.AQLExecutor.executeAQL(builder.toString());
- } catch (Exception e) {
- e.printStackTrace();
- System.out.println("Error: " + builder.toString());
- throw e;
- }
- }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 8132d4b..af8aa31 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -18,12 +18,12 @@
*/
package org.apache.asterix.hyracks.bootstrap;
+import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.app.external.CentralFeedManager;
import org.apache.asterix.app.external.ExternalIndexingOperations;
import org.apache.asterix.common.api.IClusterManagementWork;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
@@ -62,13 +62,13 @@
public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
state = AsterixClusterProperties.INSTANCE.getState();
AsterixClusterProperties.INSTANCE.setGlobalRecoveryCompleted(false);
- return null;
+ return Collections.emptySet();
}
@Override
public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
startGlobalRecovery();
- return null;
+ return Collections.emptySet();
}
private void executeHyracksJob(JobSpecification spec) throws Exception {
@@ -106,8 +106,7 @@
List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
for (Dataverse dataverse : dataverses) {
if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse,
- CentralFeedManager.getInstance());
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse);
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx,
dataverse.getDataverseName());
for (Dataset dataset : datasets) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
index 023de30..2051a35 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
@@ -36,7 +36,6 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
-import org.apache.asterix.app.external.CentralFeedManager;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.declared.AqlMetadataProvider;
@@ -172,7 +171,7 @@
private ARecordType getMetadataRecordType(String dataverseName, String datasetName) throws Exception {
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
// Retrieves file splits of the dataset.
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null, CentralFeedManager.getInstance());
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null);
metadataProvider.setMetadataTxnContext(mdTxnCtx);
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
ARecordType recordType = (ARecordType) metadataProvider.findType(dataset.getItemTypeDataverseName(),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index c040e9d..f77b38b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -28,6 +28,28 @@
ResultOffsetPath="results"
QueryOffsetPath="queries"
QueryFileExtension=".aql">
+ <test-group name="external-library">
+ <test-case FilePath="external-library">
+ <compilation-unit name="typed_adapter">
+ <output-dir compare="Text">typed_adapter</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-library">
+ <compilation-unit name="classad-parser-new">
+ <output-dir compare="Text">classad-parser-new</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-library">
+ <compilation-unit name="classad-parser-old">
+ <output-dir compare="Text">classad-parser-old</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-library">
+ <compilation-unit name="getCapital">
+ <output-dir compare="Text">getCapital</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="external">
<test-case FilePath="external">
<compilation-unit name="invalid-format">
@@ -68,28 +90,6 @@
</compilation-unit>
</test-case>
</test-group>
- <test-group name="external-library">
- <test-case FilePath="external-library">
- <compilation-unit name="typed_adapter">
- <output-dir compare="Text">typed_adapter</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="external-library">
- <compilation-unit name="classad-parser-new">
- <output-dir compare="Text">classad-parser-new</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="external-library">
- <compilation-unit name="classad-parser-old">
- <output-dir compare="Text">classad-parser-old</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="external-library">
- <compilation-unit name="getCapital">
- <output-dir compare="Text">getCapital</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
<test-group name="feeds">
<test-case FilePath="feeds">
<compilation-unit name="feed-with-multiple-indexes">
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IExceptionHandler.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/IExceptionHandler.java
similarity index 89%
rename from asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IExceptionHandler.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/IExceptionHandler.java
index ec0af1c..e0c7f25 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IExceptionHandler.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/IExceptionHandler.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.external.feed.api;
+package org.apache.asterix.common.exceptions;
import java.nio.ByteBuffer;
@@ -26,7 +26,7 @@
* Handles an exception encountered during processing of a data frame.
* In the case when the exception is of type {@code FrameDataException}, the causing
* tuple is logged and a new frame with tuple after the exception-generating tuple
- * is returned. This funcitonality is used during feed ingestion to bypass an exception
+ * is returned. This functionality is used during feed ingestion to bypass an exception
* generating tuple and thus avoid the data flow from terminating
*/
public interface IExceptionHandler {
@@ -39,5 +39,5 @@
* @return returns a new frame with tuples after the exception generating tuple
* @throws HyracksDataException
*/
- public ByteBuffer handleException(Exception e, ByteBuffer frame);
-}
+ public ByteBuffer handle(HyracksDataException e, ByteBuffer frame);
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 7b25f5a..5217846 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -137,7 +137,7 @@
</plugins>
<pluginManagement>
<plugins>
- <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+ <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
<plugin>
<groupId>org.eclipse.m2e</groupId>
<artifactId>lifecycle-mapping</artifactId>
@@ -147,8 +147,8 @@
<pluginExecutions>
<pluginExecution>
<pluginExecutionFilter>
- <groupId>org.apache.asterix</groupId>
- <artifactId>lexer-generator-maven-plugin</artifactId>
+ <groupId> org.apache.asterix</groupId>
+ <artifactId> lexer-generator-maven-plugin</artifactId>
<versionRange>[0.1,)</versionRange>
<goals>
<goal>generate-lexer</goal>
@@ -162,7 +162,7 @@
</pluginExecution>
<pluginExecution>
<pluginExecutionFilter>
- <groupId>org.codehaus.mojo</groupId>
+ <groupId> org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<versionRange>[1.7,)</versionRange>
<goals>
@@ -170,7 +170,7 @@
</goals>
</pluginExecutionFilter>
<action>
- <ignore/>
+ <ignore />
</action>
</pluginExecution>
</pluginExecutions>
@@ -194,6 +194,12 @@
<scope>compile</scope>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-test-support</artifactId>
+ <version>0.2.18-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-runtime</artifactId>
<version>0.8.9-SNAPSHOT</version>
@@ -289,5 +295,28 @@
<artifactId>core-io</artifactId>
<version>1.2.7</version>
</dependency>
+ <dependency>
+ <groupId>io.reactivex</groupId>
+ <artifactId>rxjava</artifactId>
+ <version>1.0.15</version>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>2.0.2-beta</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.powermock</groupId>
+ <artifactId>powermock-api-mockito</artifactId>
+ <version>1.6.2</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.powermock</groupId>
+ <artifactId>powermock-module-junit4</artifactId>
+ <version>1.6.2</version>
+ <scope>test</scope>
+ </dependency>
</dependencies>
-</project>
+</project>
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterRuntimeManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterRuntimeManager.java
deleted file mode 100644
index e5b22e9..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterRuntimeManager.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.asterix.external.api;
-
-import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
-import org.apache.asterix.external.feed.management.FeedId;
-
-public interface IAdapterRuntimeManager {
-
- public enum State {
- /**
- * Indicates that AsterixDB is maintaining the flow of data from external source into its storage.
- */
- ACTIVE_INGESTION,
-
- /**
- * Indicates that data from external source is being buffered and not
- * pushed downstream
- */
-
- INACTIVE_INGESTION,
- /**
- * Indicates that feed ingestion activity has finished.
- */
- FINISHED_INGESTION,
-
- /** Indicates the occurrence of a failure during the intake stage of a data ingestion pipeline **/
- FAILED_INGESTION
- }
-
- /**
- * Start feed ingestion
- * @throws Exception
- */
- public void start() throws Exception;
-
- /**
- * Stop feed ingestion.
- * @throws Exception
- */
- public void stop() throws Exception;
-
- /**
- * @return feedId associated with the feed that is being ingested.
- */
- public FeedId getFeedId();
-
- /**
- * @return an instance of the {@code FeedAdapter} in use.
- */
- public FeedAdapter getFeedAdapter();
-
- /**
- * @return state associated with the AdapterRuntimeManager. See {@code State}.
- */
- public State getState();
-
- /**
- * @param state
- */
- public void setState(State state);
-
- public IIntakeProgressTracker getProgressTracker();
-
- public int getPartition();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index ac30172..b09bef9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -92,13 +92,13 @@
try {
tupleForwarder.close();
} catch (Throwable th) {
- hde = ExternalDataExceptionUtils.suppress(hde, th);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, th);
}
try {
recordReader.close();
} catch (Throwable th) {
LOGGER.warn("Failure during while operating a feed sourcec", th);
- hde = ExternalDataExceptionUtils.suppress(hde, th);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, th);
} finally {
closeSignal();
if (hde != null) {
@@ -137,12 +137,12 @@
try {
tupleForwarder.close();
} catch (Throwable th) {
- hde = ExternalDataExceptionUtils.suppress(hde, th);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, th);
}
try {
recordReader.close();
} catch (Throwable th) {
- hde = ExternalDataExceptionUtils.suppress(hde, th);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, th);
}
if (hde != null) {
throw hde;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
index 60aaea2..7ae2f41 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedTupleForwarder.java
@@ -95,13 +95,30 @@
@Override
public void close() throws HyracksDataException {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame.getBuffer(), writer);
- }
+ Throwable throwable = null;
try {
- feedLogManager.close();
- } catch (IOException e) {
- throw new HyracksDataException(e);
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+ } catch (Throwable th) {
+ throwable = th;
+ throw th;
+ } finally {
+ try {
+ feedLogManager.close();
+ } catch (IOException e) {
+ if (throwable != null) {
+ throwable.addSuppressed(e);
+ } else {
+ throw new HyracksDataException(e);
+ }
+ } catch (Throwable th) {
+ if (throwable != null) {
+ throwable.addSuppressed(th);
+ } else {
+ throw th;
+ }
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ICentralFeedManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ICentralFeedManager.java
deleted file mode 100644
index 4f0ed77..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ICentralFeedManager.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.io.IOException;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-
-public interface ICentralFeedManager {
-
- public void start() throws AsterixException;
-
- public void stop() throws AsterixException, IOException;
-
- public IFeedTrackingManager getFeedTrackingManager();
-
- public IFeedLoadManager getFeedLoadManager();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java
index 32d551a..503715b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedConnectionManager.java
@@ -37,7 +37,7 @@
* @param feedRuntime
* @throws Exception
*/
- public void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime) throws Exception;
+ public void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime);
/**
* Obtain feed runtime corresponding to a feedRuntimeId
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedFrameHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedFrameHandler.java
deleted file mode 100644
index 3b37500..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedFrameHandler.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-
-import org.apache.asterix.external.feed.dataflow.DataBucket;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IFeedFrameHandler {
-
- public void handleFrame(ByteBuffer frame) throws HyracksDataException, InterruptedException;
-
- public void handleDataBucket(DataBucket bucket) throws InterruptedException;
-
- public void close();
-
- public Iterator<ByteBuffer> replayData() throws HyracksDataException;
-
- public String getSummary();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleIntakeEventSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleIntakeEventSubscriber.java
deleted file mode 100644
index b9caa0d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleIntakeEventSubscriber.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.watch.FeedIntakeInfo;
-
-public interface IFeedLifecycleIntakeEventSubscriber extends IFeedLifecycleEventSubscriber {
-
- public void handleFeedEvent(FeedIntakeInfo iInfo, FeedLifecycleEvent event) throws AsterixException;
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java
index 448ea47..3302856 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLifecycleListener.java
@@ -20,13 +20,12 @@
import java.util.List;
-import org.apache.asterix.common.api.IClusterEventsSubscriber;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.management.FeedJointKey;
import org.apache.hyracks.api.job.IJobLifecycleListener;
-public interface IFeedLifecycleListener extends IJobLifecycleListener, IClusterEventsSubscriber {
+public interface IFeedLifecycleListener extends IJobLifecycleListener {
public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJoinKey);
public boolean isFeedJointAvailable(FeedJointKey feedJoinKey);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLoadManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLoadManager.java
deleted file mode 100644
index 1b6347a..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedLoadManager.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.json.JSONException;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedCongestionMessage;
-import org.apache.asterix.external.feed.message.FeedReportMessage;
-import org.apache.asterix.external.feed.message.ScaleInReportMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
-import org.apache.asterix.external.feed.watch.FeedActivity;
-import org.apache.asterix.external.feed.watch.NodeLoadReport;
-
-public interface IFeedLoadManager {
-
- public void submitNodeLoadReport(NodeLoadReport report);
-
- public void reportCongestion(FeedCongestionMessage message) throws JSONException, AsterixException;
-
- public void submitFeedRuntimeReport(FeedReportMessage message);
-
- public void submitScaleInPossibleReport(ScaleInReportMessage sm) throws AsterixException, Exception;
-
- public List<String> getNodes(int required);
-
- public void reportThrottlingEnabled(ThrottlingEnabledFeedMessage mesg) throws AsterixException, Exception;
-
- int getOutflowRate(FeedConnectionId connectionId, FeedRuntimeType runtimeType);
-
- void reportFeedActivity(FeedConnectionId connectionId, FeedActivity activity);
-
- void removeFeedActivity(FeedConnectionId connectionId);
-
- public FeedActivity getFeedActivity(FeedConnectionId connectionId);
-
- public Collection<FeedActivity> getFeedActivities();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedManager.java
deleted file mode 100644
index b3ad0a5..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedManager.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.common.config.AsterixFeedProperties;
-
-/**
- * Provides access to services related to feed management within a node controller
- */
-public interface IFeedManager {
-
- /**
- * gets the handle to the singleton instance of subscription manager
- * @return the singleton instance of subscription manager
- * @see IFeedSubscriptionManager
- */
- public IFeedSubscriptionManager getFeedSubscriptionManager();
-
- /**
- * gets the handle to the singleton instance of connection manager
- * @return the singleton instance of connection manager
- * @see IFeedConnectionManager
- */
- public IFeedConnectionManager getFeedConnectionManager();
-
- /**
- * gets the handle to the singleton instance of memory manager
- * @return the singleton instance of memory manager
- * @see IFeedMemoryManager
- */
- public IFeedMemoryManager getFeedMemoryManager();
-
- /**
- * gets the handle to the singleton instance of feed metadata manager
- * @return the singleton instance of feed metadata manager
- * @see IFeedMetadataManager
- */
- public IFeedMetadataManager getFeedMetadataManager();
-
- /**
- * gets the handle to the singleton instance of feed metric collector
- * @return the singleton instance of feed metric collector
- * @see IFeedMetricCollector
- */
- public IFeedMetricCollector getFeedMetricCollector();
-
- /**
- * gets the handle to the singleton instance of feed message service
- * @return the singleton instance of feed message service
- * @see IFeedMessageService
- */
- public IFeedMessageService getFeedMessageService();
-
- public AsterixFeedProperties getAsterixFeedProperties();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryComponent.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryComponent.java
deleted file mode 100644
index 8e25b69..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryComponent.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-/**
- * Represents an in-memory components required for storing frames that contain feed tuples.
- * The component's memory footprint is measured and regulated by the {@link IFeedMemoryManager}.
- * Any expansion in size is accounted and can be restricted by the {@link IFeedMemoryManager}
- **/
-public interface IFeedMemoryComponent {
-
- public enum Type {
-
- /** A pool of reusable frames **/
- POOL,
-
- /** An ordered list of frames **/
- COLLECTION
- }
-
- /** Gets the unique id associated with the memory component **/
- public int getComponentId();
-
- /** Gets the type associated with the component. **/
- public Type getType();
-
- /** Gets the current size (number of allocated frames) of the component. **/
- public int getTotalAllocation();
-
- /**
- * Expands this memory component by the speficied number of frames
- *
- * @param delta
- * the amount (measured in number of frames) by which this memory component
- * should be expanded
- */
- public void expand(int delta);
-
- /** Clears the allocated frames as a step to reclaim the memory **/
- public void reset();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryManager.java
deleted file mode 100644
index 508602c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMemoryManager.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent.Type;
-
-/**
- * Provides management of memory allocated for handling feed data flow through the node controller
- */
-public interface IFeedMemoryManager {
-
- public static final int START_COLLECTION_SIZE = 20;
- public static final int START_POOL_SIZE = 10;
-
- /**
- * Gets a memory component allocated from the feed memory budget
- *
- * @param type
- * the kind of memory component that needs to be allocated
- * @return
- * @see Type
- */
- public IFeedMemoryComponent getMemoryComponent(Type type);
-
- /**
- * Expand a memory component by the default increment
- *
- * @param memoryComponent
- * @return true if the expansion succeeded
- * false if the requested expansion violates the configured budget
- */
- public boolean expandMemoryComponent(IFeedMemoryComponent memoryComponent);
-
- /**
- * Releases the given memory component to reclaim the memory allocated for the component
- *
- * @param memoryComponent
- * the memory component that is being reclaimed/released
- */
- public void releaseMemoryComponent(IFeedMemoryComponent memoryComponent);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java
index bc2c938..15e2de6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessage.java
@@ -28,18 +28,7 @@
public interface IFeedMessage extends Serializable, JSONSerializable {
public enum MessageType {
- END,
- XAQL,
- FEED_REPORT,
- NODE_REPORT,
- STORAGE_REPORT,
- CONGESTION,
- PREPARE_STALL,
- TERMINATE_FLOW,
- SCALE_IN_REQUEST,
- COMMIT_ACK,
- COMMIT_ACK_RESPONSE,
- THROTTLING_ENABLED
+ END
}
/**
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessageService.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessageService.java
deleted file mode 100644
index a98cf2b..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMessageService.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-/**
- * Provides the functionality of sending a meesage ({@code IFeedMessage} to the {@code CentralFeedManager}
- */
-public interface IFeedMessageService extends IFeedService {
-
- /**
- * Sends a message ({@code IFeedMessage} to the {@code CentralFeedManager} running at the CC
- * The message is sent asynchronously.
- *
- * @param message
- * the message to be sent
- */
- public void sendMessage(IFeedMessage message);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetadataManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetadataManager.java
deleted file mode 100644
index 3712678..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetadataManager.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-
-public interface IFeedMetadataManager {
-
- /**
- * @param feedConnectionId
- * connection id corresponding to the feed connection
- * @param tuple
- * the erroneous tuple that raised an exception
- * @param message
- * the message corresponding to the exception being raised
- * @param feedManager
- * @throws AsterixException
- */
- public void logTuple(FeedConnectionId feedConnectionId, String tuple, String message, IFeedManager feedManager)
- throws AsterixException;
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetricCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetricCollector.java
deleted file mode 100644
index c35587c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedMetricCollector.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-public interface IFeedMetricCollector {
-
- public enum ValueType {
- CPU_USAGE,
- INFLOW_RATE,
- OUTFLOW_RATE
- }
-
- public enum MetricType {
- AVG,
- RATE
- }
-
- public boolean sendReport(int senderId, int value);
-
- public int getMetric(int senderId);
-
- public int getMetric(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType);
-
- int createReportSender(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType,
- MetricType metricType);
-
- public void removeReportSender(int senderId);
-
- public void resetReportSender(int senderId);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedOperatorOutputSideHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedOperatorOutputSideHandler.java
deleted file mode 100644
index a8d0552..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedOperatorOutputSideHandler.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.hyracks.api.comm.IFrameWriter;
-
-/**
- * Provides for output-side buffering for a feed runtime.
- * Right now, we only have a single output side handler
- * and we can probably remove it completely.
- * ______
- * | |
- * ============|core |============
- * ============| op |============
- * |______|^^^^^^^^^^^^
- * Output Side
- * Handler
- *
- **/
-public interface IFeedOperatorOutputSideHandler extends IFrameWriter {
-
- public enum Type {
- BASIC_FEED_OUTPUT_HANDLER,
- DISTRIBUTE_FEED_OUTPUT_HANDLER,
- COLLECT_TRANSFORM_FEED_OUTPUT_HANDLER
- }
-
- public FeedId getFeedId();
-
- public Type getType();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedProvider.java
deleted file mode 100644
index 9eced07..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedProvider.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.management.FeedId;
-
-public interface IFeedProvider {
-
- public void subscribeFeed(FeedId sourceDeedId);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java
index 269725d..b8375e3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedRuntime.java
@@ -18,9 +18,7 @@
*/
package org.apache.asterix.external.feed.api;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
public interface IFeedRuntime {
@@ -36,27 +34,14 @@
}
public enum Mode {
- PROCESS,
- SPILL,
- PROCESS_SPILL,
- DISCARD,
- POST_SPILL_DISCARD,
- PROCESS_BACKLOG,
- STALL,
- FAIL,
- END
+ PROCESS, // There is memory
+ SPILL, // Memory budget has been consumed. Now we're writing to disk
+ DISCARD // Memory budget has been consumed. Disk space budget has been consumed. Now we're
+ // discarding
}
/**
* @return the unique runtime id associated with the feedRuntime
*/
public FeedRuntimeId getRuntimeId();
-
- /**
- * @return the frame writer associated with the feed runtime.
- */
- public IFrameWriter getFeedFrameWriter();
-
- public FeedRuntimeInputHandler getInputHandler();
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedService.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedService.java
deleted file mode 100644
index 3d3e0e5..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedService.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-public interface IFeedService {
-
- public void start() throws Exception;
-
- public void stop();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedSubscriptionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedSubscriptionManager.java
deleted file mode 100644
index ec4c396..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedSubscriptionManager.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
-
-public interface IFeedSubscriptionManager {
-
- /**
- * @param subscribableRuntime
- */
- public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime);
-
- /**
- * @param subscribableRuntimeId
- */
- public void deregisterFeedSubscribableRuntime(SubscribableFeedRuntimeId subscribableRuntimeId);
-
- /**
- * @param subscribableRuntimeId
- * @return
- */
- public ISubscribableRuntime getSubscribableRuntime(SubscribableFeedRuntimeId subscribableRuntimeId);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedTrackingManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedTrackingManager.java
deleted file mode 100644
index 6576e09..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFeedTrackingManager.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedTupleCommitAckMessage;
-
-public interface IFeedTrackingManager {
-
- public void submitAckReport(FeedTupleCommitAckMessage ackMessage);
-
- public void disableAcking(FeedConnectionId connectionId);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFrameEventCallback.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFrameEventCallback.java
deleted file mode 100644
index 647d847..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFrameEventCallback.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-public interface IFrameEventCallback {
-
- public enum FrameEvent {
- FINISHED_PROCESSING,
- PENDING_WORK_THRESHOLD_REACHED,
- PENDING_WORK_DONE,
- NO_OP,
- FINISHED_PROCESSING_SPILLAGE
- }
-
- public void frameEvent(FrameEvent frameEvent);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePostProcessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePostProcessor.java
deleted file mode 100644
index eab7a64..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePostProcessor.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public interface IFramePostProcessor {
-
- public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor);
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePreprocessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePreprocessor.java
deleted file mode 100644
index 55461b7..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IFramePreprocessor.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.nio.ByteBuffer;
-
-public interface IFramePreprocessor {
-
- public void preProcess(ByteBuffer frame) throws Exception;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IIntakeProgressTracker.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IIntakeProgressTracker.java
deleted file mode 100644
index 4848ed8..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IIntakeProgressTracker.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import java.util.Map;
-
-public interface IIntakeProgressTracker {
-
- public void configure(Map<String, String> configuration);
-
- public void notifyIngestedTupleTimestamp(long timestamp);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IMessageReceiver.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IMessageReceiver.java
deleted file mode 100644
index e58d99e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/IMessageReceiver.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-public interface IMessageReceiver<T> {
-
- public void sendMessage(T message) throws InterruptedException;
-
- public void close(boolean processPending);
-
- public void start();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
index ee07188..1ca46ce 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscribableRuntime.java
@@ -18,12 +18,8 @@
*/
package org.apache.asterix.external.feed.api;
-import java.util.List;
-
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
* Represent a feed runtime whose output can be routed along other parallel path(s).
@@ -34,28 +30,12 @@
* @param collectionRuntime
* @throws Exception
*/
- public void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime) throws Exception;
+ public void subscribe(CollectionRuntime collectionRuntime) throws HyracksDataException;
/**
* @param collectionRuntime
+ * @throws InterruptedException
* @throws Exception
*/
- public void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception;
-
- /**
- * @return
- * @throws Exception
- */
- public List<ISubscriberRuntime> getSubscribers();
-
- /**
- * @return
- */
- public DistributeFeedFrameWriter getFeedFrameWriter();
-
- /**
- * @return
- */
- public RecordDescriptor getRecordDescriptor();
-
+ public void unsubscribe(CollectionRuntime collectionRuntime) throws HyracksDataException, InterruptedException;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
index 4d3e607..1752054 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriberRuntime.java
@@ -20,11 +20,8 @@
import java.util.Map;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-
public interface ISubscriberRuntime {
public Map<String, String> getFeedPolicy();
- public FeedFrameCollector getFrameCollector();
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriptionProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriptionProvider.java
deleted file mode 100644
index b94a52e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/api/ISubscriptionProvider.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.asterix.external.feed.api;
-
-import org.apache.asterix.external.feed.management.FeedId;
-
-public interface ISubscriptionProvider {
-
- public void subscribeFeed(FeedId sourceFeedId, FeedId recipientFeedId);
-
- public void unsubscribeFeed(FeedId sourceFeedId, FeedId recipientFeedId);
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
deleted file mode 100644
index c4d5014..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/CollectTransformFeedFrameWriter.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.feed.api.IFeedOperatorOutputSideHandler;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-
-// Simply a delivery frame writer. I think we can simply get rid of this at some point {TODO}.
-public class CollectTransformFeedFrameWriter implements IFeedOperatorOutputSideHandler {
-
- private final FeedConnectionId connectionId; // [Dataverse - Feed - Dataset]
- private IFrameWriter downstreamWriter; // Writer to next (Operator/Connector)
- private final FrameTupleAccessor inputFrameTupleAccessor; // Accessing input frame tuples
- private final FrameTupleAppender tupleAppender; // Append tuples to output frame
- private final IFrame frame; // Output frame
-
- private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
-
- public CollectTransformFeedFrameWriter(IHyracksTaskContext ctx, IFrameWriter downstreamWriter,
- ISubscribableRuntime sourceRuntime, RecordDescriptor outputRecordDescriptor, FeedConnectionId connectionId)
- throws HyracksDataException {
- this.connectionId = connectionId;
- this.downstreamWriter = downstreamWriter;
- inputFrameTupleAccessor = new FrameTupleAccessor(sourceRuntime.getRecordDescriptor());
- frame = new VSizeFrame(ctx);
- tupleAppender = new FrameTupleAppender(frame);
- }
-
- @Override
- public void open() throws HyracksDataException {
- downstreamWriter.open();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- // always project the first field only. why?
- inputFrameTupleAccessor.reset(buffer);
- int nTuple = inputFrameTupleAccessor.getTupleCount();
- for (int t = 0; t < nTuple; t++) {
- tupleBuilder.addField(inputFrameTupleAccessor, t, 0);
- appendTupleToFrame();
- tupleBuilder.reset();
- }
- }
-
- private void appendTupleToFrame() throws HyracksDataException {
- if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
- tupleBuilder.getSize())) {
- FrameUtils.flushFrame(frame.getBuffer(), downstreamWriter);
- tupleAppender.reset(frame, true);
- if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
- tupleBuilder.getSize())) {
- throw new IllegalStateException();
- }
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- downstreamWriter.fail();
- }
-
- @Override
- public void close() throws HyracksDataException {
- downstreamWriter.close();
- }
-
- @Override
- public FeedId getFeedId() {
- return connectionId.getFeedId();
- }
-
- @Override
- public Type getType() {
- return Type.COLLECT_TRANSFORM_FEED_OUTPUT_HANDLER;
- }
-
- public IFrameWriter getDownstreamWriter() {
- return downstreamWriter;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public void reset(IFrameWriter writer) {
- this.downstreamWriter = writer;
- }
-
- @Override
- public void flush() throws HyracksDataException {
- tupleAppender.flush(downstreamWriter);
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucket.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucket.java
deleted file mode 100644
index 7f63d1d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucket.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * A {@DataBucket} is a wrapper around {@ByteBuffer} that expects certain number of receipients
- */
-public class DataBucket {
-
- private static final AtomicInteger globalBucketId = new AtomicInteger(0);
-
- private final ByteBuffer content; // Content
- private final AtomicInteger readCount; // How many reads?
- private final int bucketId; // Id
- private int desiredReadCount; // Number of expected readers
- private ContentType contentType; // Data, End of stream, or End of spilled data
- private final DataBucketPool pool; // Pool of buckets
-
- public enum ContentType {
- DATA, // data (feed tuple)
- EOD, // A signal indicating that there shall be no more data
- EOSD // End of processing of spilled data
- }
-
- public DataBucket(DataBucketPool pool) {
- this.content = ByteBuffer.allocate(pool.getFrameSize());
- this.readCount = new AtomicInteger(0);
- this.pool = pool;
- this.contentType = ContentType.DATA;
- this.bucketId = globalBucketId.incrementAndGet();
- }
-
- public synchronized void reset(ByteBuffer frame) {
- if (frame != null) {
- content.flip();
- System.arraycopy(frame.array(), 0, content.array(), 0, frame.limit());
- content.limit(frame.limit());
- content.position(0);
- }
- }
-
- public synchronized void doneReading() {
- if (readCount.incrementAndGet() == desiredReadCount) {
- readCount.set(0);
- pool.returnDataBucket(this);
- }
- }
-
- public void setDesiredReadCount(int rCount) {
- this.desiredReadCount = rCount;
- }
-
- public ContentType getContentType() {
- return contentType;
- }
-
- public void setContentType(ContentType contentType) {
- this.contentType = contentType;
- }
-
- public synchronized ByteBuffer getContent() {
- return content;
- }
-
- @Override
- public String toString() {
- return "DataBucket [" + bucketId + "]" + " (" + readCount + "," + desiredReadCount + ")";
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucketPool.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucketPool.java
deleted file mode 100644
index d1dea51..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DataBucketPool.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.util.Stack;
-
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent;
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
-
-/**
- * Represents a pool of reusable {@link DataBucket}
- */
-public class DataBucketPool implements IFeedMemoryComponent {
-
- /** A unique identifier for the memory component **/
- private final int componentId;
-
- /** The {@link IFeedMemoryManager} for the NodeController **/
- private final IFeedMemoryManager memoryManager;
-
- /** A collection of available data buckets {@link DataBucket} **/
- private final Stack<DataBucket> pool;
-
- /** The total number of data buckets {@link DataBucket} allocated **/
- private int totalAllocation;
-
- /** The fixed frame size as configured for the asterix runtime **/
- private final int frameSize;
-
- public DataBucketPool(int componentId, IFeedMemoryManager memoryManager, int size, int frameSize) {
- this.componentId = componentId;
- this.memoryManager = memoryManager;
- this.pool = new Stack<DataBucket>();
- this.frameSize = frameSize;
- expand(size);
- }
-
- public synchronized void returnDataBucket(DataBucket bucket) {
- pool.push(bucket);
- }
-
- public synchronized DataBucket getDataBucket() {
- if (pool.size() == 0) {
- if (!memoryManager.expandMemoryComponent(this)) {
- return null;
- }
- }
- return pool.pop();
- }
-
- @Override
- public Type getType() {
- return Type.POOL;
- }
-
- @Override
- public int getTotalAllocation() {
- return totalAllocation;
- }
-
- @Override
- public int getComponentId() {
- return componentId;
- }
-
- @Override
- public void expand(int delta) {
- for (int i = 0; i < delta; i++) {
- DataBucket bucket = new DataBucket(this);
- pool.add(bucket);
- }
- totalAllocation += delta;
- }
-
- @Override
- public void reset() {
- totalAllocation -= pool.size();
- pool.clear();
- }
-
- @Override
- public String toString() {
- return "DataBucketPool" + "[" + componentId + "]" + "(" + totalAllocation + ")";
- }
-
- public int getSize() {
- return pool.size();
- }
-
- public int getFrameSize() {
- return frameSize;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
index 8b7e2ba..f356899 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/DistributeFeedFrameWriter.java
@@ -20,17 +20,10 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedOperatorOutputSideHandler;
-import org.apache.asterix.external.feed.api.IFeedOperatorOutputSideHandler.Type;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -44,8 +37,6 @@
**/
public class DistributeFeedFrameWriter implements IFrameWriter {
- private static final Logger LOGGER = Logger.getLogger(DistributeFeedFrameWriter.class.getName());
-
/** A unique identifier for the feed to which the incoming tuples belong. **/
private final FeedId feedId;
@@ -56,7 +47,7 @@
private final FrameDistributor frameDistributor;
/** The original frame writer instantiated as part of job creation **/
- private IFrameWriter writer;
+ private final IFrameWriter writer;
/** The feed operation whose output is being distributed by the DistributeFeedFrameWriter **/
private final FeedRuntimeType feedRuntimeType;
@@ -65,11 +56,9 @@
private final int partition;
public DistributeFeedFrameWriter(IHyracksTaskContext ctx, FeedId feedId, IFrameWriter writer,
- FeedRuntimeType feedRuntimeType, int partition, FrameTupleAccessor fta, IFeedManager feedManager)
- throws IOException {
+ FeedRuntimeType feedRuntimeType, int partition, FrameTupleAccessor fta) throws IOException {
this.feedId = feedId;
- this.frameDistributor = new FrameDistributor(feedId, feedRuntimeType, partition, true,
- feedManager.getFeedMemoryManager(), fta);
+ this.frameDistributor = new FrameDistributor();
this.feedRuntimeType = feedRuntimeType;
this.partition = partition;
this.writer = writer;
@@ -78,38 +67,19 @@
/**
* @param fpa
* Feed policy accessor
- * @param frameWriter
+ * @param nextOnlyWriter
* the writer which will deliver the buffers
* @param connectionId
* (Dataverse - Dataset - Feed)
* @return A frame collector.
- * @throws Exception
+ * @throws HyracksDataException
*/
- public FeedFrameCollector subscribeFeed(FeedPolicyAccessor fpa, IFrameWriter frameWriter,
- FeedConnectionId connectionId) throws Exception {
- FeedFrameCollector collector = null;
- if (!frameDistributor.isRegistered(frameWriter)) {
- collector = new FeedFrameCollector(frameDistributor, fpa, frameWriter, connectionId);
- frameDistributor.registerFrameCollector(collector);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered subscriber, new mode " + frameDistributor.getMode());
- }
- return collector;
- } else {
- throw new IllegalStateException("subscriber " + feedId + " already registered");
- }
+ public void subscribe(FeedFrameCollector collector) throws HyracksDataException {
+ frameDistributor.registerFrameCollector(collector);
}
- public void unsubscribeFeed(IFrameWriter recipientFeedFrameWriter) throws Exception {
- boolean success = frameDistributor.deregisterFrameCollector(recipientFeedFrameWriter);
- if (!success) {
- throw new IllegalStateException(
- "Invalid attempt to unregister FeedFrameWriter " + recipientFeedFrameWriter + " not registered.");
- }
- }
-
- public void notifyEndOfFeed() throws InterruptedException {
- frameDistributor.notifyEndOfFeed();
+ public void unsubscribeFeed(FeedConnectionId connectionId) throws HyracksDataException {
+ frameDistributor.deregisterFrameCollector(connectionId);
}
@Override
@@ -136,27 +106,11 @@
writer.open();
}
- public Map<IFrameWriter, FeedFrameCollector> getRegisteredReaders() {
- return frameDistributor.getRegisteredReaders();
- }
-
- public void setWriter(IFrameWriter writer) {
- this.writer = writer;
- }
-
- public Type getType() {
- return IFeedOperatorOutputSideHandler.Type.DISTRIBUTE_FEED_OUTPUT_HANDLER;
- }
-
@Override
public String toString() {
return feedId.toString() + feedRuntimeType + "[" + partition + "]";
}
- public FrameDistributor.DistributionMode getDistributionMode() {
- return frameDistributor.getDistributionMode();
- }
-
@Override
public void flush() throws HyracksDataException {
frameDistributor.flush();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedCollectRuntimeInputHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedCollectRuntimeInputHandler.java
deleted file mode 100644
index 447c4bc..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedCollectRuntimeInputHandler.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.io.IOException;
-
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class FeedCollectRuntimeInputHandler extends FeedRuntimeInputHandler {
-
- private final FeedFrameCache feedFrameCache;
-
- public FeedCollectRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId,
- FeedRuntimeId runtimeId, IFrameWriter coreOperator, FeedPolicyAccessor fpa, boolean bufferingEnabled,
- FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedManager feedManager, int nPartitions)
- throws IOException {
- super(ctx, connectionId, runtimeId, coreOperator, fpa, bufferingEnabled, fta, recordDesc, feedManager,
- nPartitions);
- this.feedFrameCache = new FeedFrameCache(ctx, fta, coreOperator);
- }
-
- public void replayFrom(int recordId) throws HyracksDataException {
- feedFrameCache.replayRecords(recordId);
- }
-
- public void dropTill(int recordId) {
- feedFrameCache.dropTillRecordId(recordId);
- }
-
- public void replayCached() throws HyracksDataException {
- feedFrameCache.replayAll();
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
index f102f93..b98f123 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
@@ -24,12 +24,9 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.FrameDataException;
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.common.exceptions.IExceptionHandler;
import org.apache.asterix.external.util.FeedFrameUtil;
import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -37,12 +34,11 @@
private static Logger LOGGER = Logger.getLogger(FeedExceptionHandler.class.getName());
- //TODO: Enable logging
+ // TODO: Enable logging
private final IHyracksTaskContext ctx;
private final FrameTupleAccessor fta;
- public FeedExceptionHandler(IHyracksTaskContext ctx, FrameTupleAccessor fta, RecordDescriptor recordDesc,
- IFeedManager feedManager, FeedConnectionId connectionId) {
+ public FeedExceptionHandler(IHyracksTaskContext ctx, FrameTupleAccessor fta) {
this.ctx = ctx;
this.fta = fta;
}
@@ -53,20 +49,21 @@
}
@Override
- public ByteBuffer handleException(Exception e, ByteBuffer frame) {
+ public ByteBuffer handle(HyracksDataException th, ByteBuffer frame) {
try {
- if (e instanceof FrameDataException) {
+ if (th instanceof FrameDataException) {
fta.reset(frame);
- FrameDataException fde = (FrameDataException) e;
+ FrameDataException fde = (FrameDataException) th;
int tupleIndex = fde.getTupleIndex();
try {
- logExceptionCausingTuple(tupleIndex, e);
+ logExceptionCausingTuple(tupleIndex, th);
} catch (Exception ex) {
- ex.addSuppressed(e);
+ ex.addSuppressed(th);
if (LOGGER.isLoggable(Level.WARNING)) {
LOGGER.warning("Unable to log exception causing tuple due to..." + ex.getMessage());
}
}
+ // TODO: Improve removeBadTuple. Right now, it creates lots of objects
return FeedFrameUtil.removeBadTuple(ctx, tupleIndex, fta);
} else {
return null;
@@ -80,6 +77,8 @@
}
}
- private void logExceptionCausingTuple(int tupleIndex, Exception e) throws HyracksDataException, AsterixException {
+ // TODO: Fix logging of exceptions
+ private void logExceptionCausingTuple(int tupleIndex, Throwable e) throws HyracksDataException, AsterixException {
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
deleted file mode 100644
index 159bc43..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCache.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.asterix.external.feed.message.MessageReceiver;
-import org.apache.asterix.external.util.FeedConstants.StatisticsConstants;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-
-/**
- * Allows caching of feed frames. This class is used in providing upstream backup.
- * The tuples at the intake layer are held in this cache until these are acked by
- * the storage layer post their persistence. On receiving an ack, appropriate tuples
- * (recordsId < ackedRecordId) are dropped from the cache.
- */
-public class FeedFrameCache extends MessageReceiver<ByteBuffer> {
-
- /**
- * Value represents a cache feed frame
- * Key represents the largest record Id in the frame.
- * At the intake side, the largest record id corresponds to the last record in the frame
- **/
- private final Map<Integer, ByteBuffer> orderedCache;
- private final FrameTupleAccessor tupleAccessor;
- private final IFrameWriter frameWriter;
- private final IHyracksTaskContext ctx;
-
- public FeedFrameCache(IHyracksTaskContext ctx, FrameTupleAccessor tupleAccessor, IFrameWriter frameWriter) {
- this.tupleAccessor = tupleAccessor;
- this.frameWriter = frameWriter;
- /** A LinkedHashMap ensures entries are retrieved in order of their insertion **/
- this.orderedCache = new LinkedHashMap<Integer, ByteBuffer>();
- this.ctx = ctx;
- }
-
- @Override
- public void processMessage(ByteBuffer frame) throws Exception {
- int lastRecordId = getLastRecordId(frame);
- ByteBuffer clone = cloneFrame(frame);
- orderedCache.put(lastRecordId, clone);
- }
-
- public void dropTillRecordId(int recordId) {
- List<Integer> dropRecordIds = new ArrayList<Integer>();
- for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
- int recId = entry.getKey();
- if (recId <= recordId) {
- dropRecordIds.add(recId);
- } else {
- break;
- }
- }
- for (Integer r : dropRecordIds) {
- orderedCache.remove(r);
- }
- }
-
- public void replayRecords(int startingRecordId) throws HyracksDataException {
- boolean replayPositionReached = false;
- for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
- // the key increases monotonically
- int maxRecordIdInFrame = entry.getKey();
- if (!replayPositionReached) {
- if (startingRecordId < maxRecordIdInFrame) {
- replayFrame(startingRecordId, entry.getValue());
- break;
- } else {
- continue;
- }
- }
- }
- }
-
- /**
- * Replay the frame from the tuple (inclusive) with recordId as specified.
- *
- * @param recordId
- * @param frame
- * @throws HyracksDataException
- */
- private void replayFrame(int recordId, ByteBuffer frame) throws HyracksDataException {
- tupleAccessor.reset(frame);
- int nTuples = tupleAccessor.getTupleCount();
- for (int i = 0; i < nTuples; i++) {
- int rid = getRecordIdAtTupleIndex(i, frame);
- if (rid == recordId) {
- ByteBuffer slicedFrame = splitFrame(i, frame);
- replayFrame(slicedFrame);
- break;
- }
- }
- }
-
- private ByteBuffer splitFrame(int beginTupleIndex, ByteBuffer frame) throws HyracksDataException {
- IFrame slicedFrame = new VSizeFrame(ctx);
- FrameTupleAppender appender = new FrameTupleAppender();
- appender.reset(slicedFrame, true);
- int totalTuples = tupleAccessor.getTupleCount();
- for (int ti = beginTupleIndex; ti < totalTuples; ti++) {
- appender.append(tupleAccessor, ti);
- }
- return slicedFrame.getBuffer();
- }
-
- /**
- * Replay the frame
- *
- * @param frame
- * @throws HyracksDataException
- */
- private void replayFrame(ByteBuffer frame) throws HyracksDataException {
- frameWriter.nextFrame(frame);
- }
-
- private int getLastRecordId(ByteBuffer frame) {
- tupleAccessor.reset(frame);
- int nTuples = tupleAccessor.getTupleCount();
- return getRecordIdAtTupleIndex(nTuples - 1, frame);
- }
-
- private int getRecordIdAtTupleIndex(int tupleIndex, ByteBuffer frame) {
- tupleAccessor.reset(frame);
- int recordStart = tupleAccessor.getTupleStartOffset(tupleIndex) + tupleAccessor.getFieldSlotsLength();
- int openPartOffset = frame.getInt(recordStart + 6);
- int numOpenFields = frame.getInt(recordStart + openPartOffset);
- int recordIdOffset = frame.getInt(recordStart + openPartOffset + 4 + numOpenFields * 8
- + StatisticsConstants.INTAKE_TUPLEID.length() + 2 + 1);
- int lastRecordId = frame.getInt(recordStart + recordIdOffset);
- return lastRecordId;
- }
-
- private ByteBuffer cloneFrame(ByteBuffer frame) {
- ByteBuffer clone = ByteBuffer.allocate(frame.capacity());
- System.arraycopy(frame.array(), 0, clone.array(), 0, frame.limit());
- return clone;
- }
-
- public void replayAll() throws HyracksDataException {
- for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
- ByteBuffer frame = entry.getValue();
- frameWriter.nextFrame(frame);
- }
- }
-
- @Override
- public void emptyInbox() throws HyracksDataException {
- frameWriter.flush();
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
index ef4b87d..b81f59a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameCollector.java
@@ -19,21 +19,16 @@
package org.apache.asterix.external.feed.dataflow;
import java.nio.ByteBuffer;
-import java.util.logging.Level;
-import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.MessageReceiver;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-public class FeedFrameCollector extends MessageReceiver<DataBucket> {
+public class FeedFrameCollector implements IFrameWriter {
private final FeedConnectionId connectionId;
- private final FrameDistributor frameDistributor;
- private FeedPolicyAccessor fpa;
- private IFrameWriter frameWriter;
+ private IFrameWriter writer;
private State state;
public enum State {
@@ -43,70 +38,27 @@
HANDOVER
}
- public FeedFrameCollector(FrameDistributor frameDistributor, FeedPolicyAccessor feedPolicyAccessor,
- IFrameWriter frameWriter, FeedConnectionId connectionId) {
- super();
- this.frameDistributor = frameDistributor;
- this.fpa = feedPolicyAccessor;
+ public FeedFrameCollector(FeedPolicyAccessor feedPolicyAccessor, IFrameWriter writer,
+ FeedConnectionId connectionId) {
this.connectionId = connectionId;
- this.frameWriter = frameWriter;
+ this.writer = writer;
this.state = State.ACTIVE;
}
@Override
- public void processMessage(DataBucket bucket) throws Exception {
- try {
- ByteBuffer frame = bucket.getContent();
- switch (bucket.getContentType()) {
- case DATA:
- frameWriter.nextFrame(frame);
- break;
- case EOD:
- closeCollector();
- break;
- case EOSD:
- throw new AsterixException("Received data bucket with content of type " + bucket.getContentType());
- }
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to process data bucket " + bucket + ", encountered exception " + e.getMessage());
- }
- } finally {
- bucket.doneReading();
- }
- }
-
- public void closeCollector() {
- if (state.equals(State.TRANSITION)) {
- super.close(true);
- setState(State.ACTIVE);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(this + " is now " + State.ACTIVE + " mode, processing frames synchronously");
- }
- } else {
- flushPendingMessages();
- setState(State.FINISHED);
- synchronized (frameDistributor.getRegisteredCollectors()) {
- frameDistributor.getRegisteredCollectors().notifyAll();
- }
- disconnect();
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Closed collector " + this);
- }
+ public synchronized void close() throws HyracksDataException {
+ writer.close();
+ state = State.FINISHED;
+ notify();
}
public synchronized void disconnect() {
setState(State.FINISHED);
}
+ @Override
public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
- frameWriter.nextFrame(frame);
- }
-
- public FeedPolicyAccessor getFeedPolicyAccessor() {
- return fpa;
+ writer.nextFrame(frame);
}
public synchronized State getState() {
@@ -123,17 +75,14 @@
default:
break;
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Frame Collector " + this.frameDistributor.getFeedRuntimeType() + " switched to " + state);
- }
}
public IFrameWriter getFrameWriter() {
- return frameWriter;
+ return writer;
}
- public void setFrameWriter(IFrameWriter frameWriter) {
- this.frameWriter = frameWriter;
+ public void setFrameWriter(IFrameWriter writer) {
+ this.writer = writer;
}
@Override
@@ -158,12 +107,21 @@
}
@Override
- public void emptyInbox() throws HyracksDataException {
- flush();
- }
-
public synchronized void flush() throws HyracksDataException {
- frameWriter.flush();
+ writer.flush();
}
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameDiscarder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameDiscarder.java
deleted file mode 100644
index 53ee475..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameDiscarder.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class FeedFrameDiscarder {
-
- private static final Logger LOGGER = Logger.getLogger(FeedFrameSpiller.class.getName());
-
- private final FeedRuntimeInputHandler inputHandler;
- private final FeedConnectionId connectionId;
- private final FeedRuntimeId runtimeId;
- private final FeedPolicyAccessor policyAccessor;
- private final float maxFractionDiscard;
- private int nDiscarded;
-
- public FeedFrameDiscarder(FeedConnectionId connectionId, FeedRuntimeId runtimeId, FeedPolicyAccessor policyAccessor,
- FeedRuntimeInputHandler inputHandler) throws HyracksDataException {
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- this.policyAccessor = policyAccessor;
- this.inputHandler = inputHandler;
- this.maxFractionDiscard = policyAccessor.getMaxFractionDiscard();
- }
-
- public boolean processMessage(ByteBuffer message) {
- if (policyAccessor.getMaxFractionDiscard() != 0) {
- long nProcessed = inputHandler.getProcessed();
- long discardLimit = (long) (nProcessed * maxFractionDiscard);
- if (nDiscarded >= discardLimit) {
- return false;
- }
- nDiscarded++;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Discarded frame by " + connectionId + " (" + runtimeId + ")" + " count so far ("
- + nDiscarded + ") Limit [" + discardLimit + "]");
- }
- return true;
- }
- return false;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
deleted file mode 100644
index 2a6fd79..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameHandlers.java
+++ /dev/null
@@ -1,309 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedFrameHandler;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.message.MessageReceiver;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class FeedFrameHandlers {
-
- private static final Logger LOGGER = Logger.getLogger(FeedFrameHandlers.class.getName());
-
- public enum RoutingMode {
- IN_MEMORY_ROUTE,
- SPILL_TO_DISK,
- DISCARD
- }
-
- public static IFeedFrameHandler getFeedFrameHandler(FrameDistributor distributor, FeedId feedId,
- RoutingMode routingMode, FeedRuntimeType runtimeType, int partition, int frameSize) throws IOException {
- IFeedFrameHandler handler = null;
- switch (routingMode) {
- case IN_MEMORY_ROUTE:
- handler = new InMemoryRouter(distributor.getRegisteredReaders().values(), runtimeType, partition);
- break;
- case SPILL_TO_DISK:
- handler = new DiskSpiller(distributor, feedId, runtimeType, partition, frameSize);
- break;
- case DISCARD:
- handler = new DiscardRouter(distributor, feedId, runtimeType, partition);
- break;
- default:
- throw new IllegalArgumentException("Invalid routing mode" + routingMode);
- }
- return handler;
- }
-
- public static class DiscardRouter implements IFeedFrameHandler {
-
- private final FeedId feedId;
- private int nDiscarded;
- private final FeedRuntimeType runtimeType;
- private final int partition;
- private final FrameDistributor distributor;
-
- public DiscardRouter(FrameDistributor distributor, FeedId feedId, FeedRuntimeType runtimeType, int partition)
- throws HyracksDataException {
- this.distributor = distributor;
- this.feedId = feedId;
- this.nDiscarded = 0;
- this.runtimeType = runtimeType;
- this.partition = partition;
- }
-
- @Override
- public void handleFrame(ByteBuffer frame) throws HyracksDataException {
- FrameTupleAccessor fta = distributor.getFta();
- fta.reset(frame);
- int nTuples = fta.getTupleCount();
- nDiscarded += nTuples;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Discarded additional [" + runtimeType + "]" + "(" + partition + ")" + " " + nTuples);
- }
- }
-
- @Override
- public void handleDataBucket(DataBucket bucket) {
- nDiscarded++;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Discard Count" + nDiscarded);
- }
- }
-
- @Override
- public void close() {
- // do nothing, no resource to relinquish
- }
-
- @Override
- public Iterator<ByteBuffer> replayData() throws HyracksDataException {
- throw new IllegalStateException("Invalid operation");
- }
-
- @Override
- public String toString() {
- return "DiscardRouter" + "[" + feedId + "]" + "(" + nDiscarded + ")";
- }
-
- @Override
- public String getSummary() {
- return new String("Number of discarded frames (since last reset)" + " feedId " + "[" + feedId + "]" + "("
- + nDiscarded + ")");
- }
-
- }
-
- public static class InMemoryRouter implements IFeedFrameHandler {
-
- private final Collection<FeedFrameCollector> frameCollectors;
-
- public InMemoryRouter(Collection<FeedFrameCollector> frameCollectors, FeedRuntimeType runtimeType,
- int partition) {
- this.frameCollectors = frameCollectors;
- }
-
- @Override
- public void handleFrame(ByteBuffer frame) throws HyracksDataException {
- throw new IllegalStateException("Operation not supported");
- }
-
- @Override
- public void handleDataBucket(DataBucket bucket) throws InterruptedException {
- for (FeedFrameCollector collector : frameCollectors) {
- collector.sendMessage(bucket);
- }
- }
-
- @Override
- public void close() {
- // do nothing
- }
-
- @Override
- public Iterator<ByteBuffer> replayData() throws HyracksDataException {
- throw new IllegalStateException("Operation not supported");
- }
-
- @Override
- public String getSummary() {
- return "InMemoryRouter Summary";
- }
- }
-
- public static class DiskSpiller implements IFeedFrameHandler {
-
- private FrameSpiller<ByteBuffer> receiver;
- private Iterator<ByteBuffer> iterator;
-
- public DiskSpiller(FrameDistributor distributor, FeedId feedId, FeedRuntimeType runtimeType, int partition,
- int frameSize) throws IOException {
- receiver = new FrameSpiller<ByteBuffer>(distributor, feedId, frameSize);
- }
-
- @Override
- public void handleFrame(ByteBuffer frame) throws HyracksDataException, InterruptedException {
- receiver.sendMessage(frame);
- }
-
- private static class FrameSpiller<T> extends MessageReceiver<ByteBuffer> {
-
- private final FeedId feedId;
- private BufferedOutputStream bos;
- private final ByteBuffer reusableLengthBuffer;
- private final ByteBuffer reusableDataBuffer;
- private long offset;
- private File file;
- private final FrameDistributor frameDistributor;
- private boolean fileCreated = false;
-
- public FrameSpiller(FrameDistributor distributor, FeedId feedId, int frameSize) throws IOException {
- this.feedId = feedId;
- this.frameDistributor = distributor;
- reusableLengthBuffer = ByteBuffer.allocate(4);
- reusableDataBuffer = ByteBuffer.allocate(frameSize);
- this.offset = 0;
- }
-
- @Override
- public void processMessage(ByteBuffer message) throws Exception {
- if (!fileCreated) {
- createFile();
- fileCreated = true;
- }
- reusableLengthBuffer.flip();
- reusableLengthBuffer.putInt(message.array().length);
- bos.write(reusableLengthBuffer.array());
- bos.write(message.array());
- }
-
- private void createFile() throws IOException {
- Date date = new Date();
- String dateSuffix = date.toString().replace(' ', '_');
- String fileName = feedId.toString() + "_" + frameDistributor.getFeedRuntimeType() + "_"
- + frameDistributor.getPartition() + "_" + dateSuffix;
-
- file = new File(fileName);
- if (!file.exists()) {
- boolean success = file.createNewFile();
- if (!success) {
- throw new IOException("Unable to create spill file for feed " + feedId);
- }
- }
- bos = new BufferedOutputStream(new FileOutputStream(file));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Created Spill File for feed " + feedId);
- }
- }
-
- @SuppressWarnings("resource")
- public Iterator<ByteBuffer> replayData() throws Exception {
- final BufferedInputStream bis = new BufferedInputStream(new FileInputStream(file));
- bis.skip(offset);
- return new Iterator<ByteBuffer>() {
-
- @Override
- public boolean hasNext() {
- boolean more = false;
- try {
- more = bis.available() > 0;
- if (!more) {
- bis.close();
- }
- } catch (IOException e) {
- e.printStackTrace();
- }
-
- return more;
- }
-
- @Override
- public ByteBuffer next() {
- reusableLengthBuffer.flip();
- try {
- bis.read(reusableLengthBuffer.array());
- reusableLengthBuffer.flip();
- int frameSize = reusableLengthBuffer.getInt();
- reusableDataBuffer.flip();
- bis.read(reusableDataBuffer.array(), 0, frameSize);
- offset += 4 + frameSize;
- } catch (IOException e) {
- e.printStackTrace();
- }
- return reusableDataBuffer;
- }
-
- @Override
- public void remove() {
- }
-
- };
- }
-
- @Override
- public void emptyInbox() throws HyracksDataException {
- }
-
- }
-
- @Override
- public void handleDataBucket(DataBucket bucket) {
- throw new IllegalStateException("Operation not supported");
- }
-
- @Override
- public void close() {
- receiver.close(true);
- }
-
- @Override
- public Iterator<ByteBuffer> replayData() throws HyracksDataException {
- try {
- iterator = receiver.replayData();
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- return iterator;
- }
-
- //TODO: Form a summary that includes stats related to what has been spilled to disk
- @Override
- public String getSummary() {
- return "Disk Spiller Summary";
- }
-
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameSpiller.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameSpiller.java
deleted file mode 100644
index d11c3de..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameSpiller.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class FeedFrameSpiller {
-
- private static final Logger LOGGER = Logger.getLogger(FeedFrameSpiller.class.getName());
-
- private final IHyracksTaskContext ctx;
- private final FeedConnectionId connectionId;
- private final FeedRuntimeId runtimeId;
- private final FeedPolicyAccessor policyAccessor;
- private BufferedOutputStream bos;
- private File file;
- private boolean fileCreated = false;
- private long bytesWritten = 0;
- private int spilledFrameCount = 0;
-
- public FeedFrameSpiller(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- FeedPolicyAccessor policyAccessor) throws HyracksDataException {
- this.ctx = ctx;
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- this.policyAccessor = policyAccessor;
- }
-
- public boolean processMessage(ByteBuffer message) throws HyracksDataException {
- if (!fileCreated) {
- createFile();
- fileCreated = true;
- }
- long maxAllowed = policyAccessor.getMaxSpillOnDisk();
- if (maxAllowed != FeedPolicyAccessor.NO_LIMIT && bytesWritten + message.array().length > maxAllowed) {
- return false;
- } else {
- try {
- bos.write(message.array());
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- bytesWritten += message.array().length;
- spilledFrameCount++;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Spilled frame by " + runtimeId + " spill count " + spilledFrameCount);
- }
- return true;
- }
- }
-
- private void createFile() throws HyracksDataException {
- try {
- Date date = new Date();
- String dateSuffix = date.toString().replace(' ', '_');
- String fileName = connectionId.getFeedId() + "_" + connectionId.getDatasetName() + "_"
- + runtimeId.getFeedRuntimeType() + "_" + runtimeId.getPartition() + "_" + dateSuffix;
-
- file = new File(fileName);
- if (!file.exists()) {
- boolean success = file.createNewFile();
- if (!success) {
- throw new HyracksDataException(
- "Unable to create spill file " + fileName + " for feed " + runtimeId);
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Created spill file " + file.getAbsolutePath());
- }
- }
- }
- bos = new BufferedOutputStream(new FileOutputStream(file));
- } catch (Throwable th) {
- throw new HyracksDataException(th);
- }
- }
-
- public Iterator<ByteBuffer> replayData() throws Exception {
- bos.flush();
- return new FrameIterator(ctx, file.getName());
- }
-
- private static class FrameIterator implements Iterator<ByteBuffer> {
-
- private final BufferedInputStream bis;
- private final IHyracksTaskContext ctx;
- private int readFrameCount = 0;
-
- public FrameIterator(IHyracksTaskContext ctx, String filename) throws FileNotFoundException {
- bis = new BufferedInputStream(new FileInputStream(new File(filename)));
- this.ctx = ctx;
- }
-
- @Override
- public boolean hasNext() {
- boolean more = false;
- try {
- more = bis.available() > 0;
- if (!more) {
- bis.close();
- }
- } catch (IOException e) {
- e.printStackTrace();
- }
-
- return more;
- }
-
- @Override
- public ByteBuffer next() {
- IFrame frame = null;
- try {
- frame = new VSizeFrame(ctx);
- Arrays.fill(frame.getBuffer().array(), (byte) 0);
- bis.read(frame.getBuffer().array(), 0, frame.getFrameSize());
- readFrameCount++;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Read spill frome " + readFrameCount);
- }
- } catch (IOException e) {
- e.printStackTrace();
- }
- return frame.getBuffer();
- }
-
- @Override
- public void remove() {
- }
-
- }
-
- public void reset() {
- bytesWritten = 0;
- // file.delete();
- fileCreated = false;
- bos = null;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Resetted the FrameSpiller!");
- }
- }
-
- public void close() {
- if (bos != null) {
- try {
- bos.flush();
- bos.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleAccessor.java
deleted file mode 100644
index d3897f3..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleAccessor.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.util.FeedConstants.StatisticsConstants;
-import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class FeedFrameTupleAccessor implements IFrameTupleAccessor {
-
- private final FrameTupleAccessor frameAccessor;
- private final int numOpenFields;
-
- public FeedFrameTupleAccessor(FrameTupleAccessor frameAccessor) {
- this.frameAccessor = frameAccessor;
- int firstRecordStart = frameAccessor.getTupleStartOffset(0) + frameAccessor.getFieldSlotsLength();
- int openPartOffsetOrig = frameAccessor.getBuffer().getInt(firstRecordStart + 6);
- numOpenFields = frameAccessor.getBuffer().getInt(firstRecordStart + openPartOffsetOrig);
- }
-
- public int getFeedIntakePartition(int tupleIndex) {
- ByteBuffer buffer = frameAccessor.getBuffer();
- int recordStart = frameAccessor.getTupleStartOffset(tupleIndex) + frameAccessor.getFieldSlotsLength();
- int openPartOffsetOrig = buffer.getInt(recordStart + 6);
- int partitionOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
- + StatisticsConstants.INTAKE_PARTITION.length() + 2 + 1;
- return buffer.getInt(recordStart + partitionOffset);
- }
-
-
-
- @Override
- public int getFieldCount() {
- return frameAccessor.getFieldCount();
- }
-
- @Override
- public int getFieldSlotsLength() {
- return frameAccessor.getFieldSlotsLength();
- }
-
- @Override
- public int getFieldEndOffset(int tupleIndex, int fIdx) {
- return frameAccessor.getFieldEndOffset(tupleIndex, fIdx);
- }
-
- @Override
- public int getFieldStartOffset(int tupleIndex, int fIdx) {
- return frameAccessor.getFieldStartOffset(tupleIndex, fIdx);
- }
-
- @Override
- public int getFieldLength(int tupleIndex, int fIdx) {
- return frameAccessor.getFieldLength(tupleIndex, fIdx);
- }
-
- @Override
- public int getTupleEndOffset(int tupleIndex) {
- return frameAccessor.getTupleEndOffset(tupleIndex);
- }
-
- @Override
- public int getTupleStartOffset(int tupleIndex) {
- return frameAccessor.getTupleStartOffset(tupleIndex);
- }
-
- @Override
- public int getTupleCount() {
- return frameAccessor.getTupleCount();
- }
-
- @Override
- public ByteBuffer getBuffer() {
- return frameAccessor.getBuffer();
- }
-
- @Override
- public void reset(ByteBuffer buffer) {
- frameAccessor.reset(buffer);
- }
-
- @Override
- public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
- return frameAccessor.getAbsoluteFieldStartOffset(tupleIndex, fIdx);
- }
-
- @Override
- public int getTupleLength(int tupleIndex) {
- return frameAccessor.getTupleLength(tupleIndex);
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleDecorator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleDecorator.java
deleted file mode 100644
index d43f90d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedFrameTupleDecorator.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.asterix.builders.IARecordBuilder;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.util.FeedConstants.StatisticsConstants;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-
-public class FeedFrameTupleDecorator {
-
- private AMutableString aString = new AMutableString("");
- private AMutableInt64 aInt64 = new AMutableInt64(0);
- private AMutableInt32 aInt32 = new AMutableInt32(0);
- private AtomicInteger tupleId;
-
- @SuppressWarnings("unchecked")
- private static ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ASTRING);
- @SuppressWarnings("unchecked")
- private static ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
- @SuppressWarnings("unchecked")
- private static ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
-
- private final int partition;
- private final ArrayBackedValueStorage attrNameStorage;
- private final ArrayBackedValueStorage attrValueStorage;
-
- public FeedFrameTupleDecorator(int partition) {
- this.tupleId = new AtomicInteger(0);
- this.partition = partition;
- this.attrNameStorage = new ArrayBackedValueStorage();
- this.attrValueStorage = new ArrayBackedValueStorage();
- }
-
- public void addLongAttribute(String attrName, long attrValue, IARecordBuilder recordBuilder)
- throws HyracksDataException, AsterixException {
- attrNameStorage.reset();
- aString.setValue(attrName);
- stringSerde.serialize(aString, attrNameStorage.getDataOutput());
-
- attrValueStorage.reset();
- aInt64.setValue(attrValue);
- int64Serde.serialize(aInt64, attrValueStorage.getDataOutput());
-
- recordBuilder.addField(attrNameStorage, attrValueStorage);
- }
-
- public void addIntegerAttribute(String attrName, int attrValue, IARecordBuilder recordBuilder)
- throws HyracksDataException, AsterixException {
- attrNameStorage.reset();
- aString.setValue(attrName);
- stringSerde.serialize(aString, attrNameStorage.getDataOutput());
-
- attrValueStorage.reset();
- aInt32.setValue(attrValue);
- int32Serde.serialize(aInt32, attrValueStorage.getDataOutput());
-
- recordBuilder.addField(attrNameStorage, attrValueStorage);
- }
-
- public void addTupleId(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
- addIntegerAttribute(StatisticsConstants.INTAKE_TUPLEID, tupleId.incrementAndGet(), recordBuilder);
- }
-
- public void addIntakePartition(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
- addIntegerAttribute(StatisticsConstants.INTAKE_PARTITION, partition, recordBuilder);
- }
-
- public void addIntakeTimestamp(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
- addLongAttribute(StatisticsConstants.INTAKE_TIMESTAMP, System.currentTimeMillis(), recordBuilder);
- }
-
- public void addStoreTimestamp(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
- addLongAttribute(StatisticsConstants.STORE_TIMESTAMP, System.currentTimeMillis(), recordBuilder);
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
index dcd03a9..3920a03 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedRuntimeInputHandler.java
@@ -19,328 +19,262 @@
package org.apache.asterix.external.feed.dataflow;
import java.nio.ByteBuffer;
-import java.util.Iterator;
+import java.util.concurrent.LinkedBlockingDeque;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent;
-import org.apache.asterix.external.feed.api.IFeedMessage;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.dataflow.DataBucket.ContentType;
+import org.apache.asterix.external.feed.management.ConcurrentFramePool;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedCongestionMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.feed.watch.MonitoredBuffer;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
/**
+ * TODO: Add unit test cases for this class
* Provides for error-handling and input-side buffering for a feed runtime.
- * The input handler is buffering in:
- * 1. FeedMetaComputeNodePushable.initializeNewFeedRuntime();
- * 2. FeedMetaStoreNodePushable.initializeNewFeedRuntime();
- * ______
- * | |
- * ============| core |============
- * ============| op |============
- * ^^^^^^^^^^^^|______|
- * Input Side
- * Handler
+ * .............______.............
+ * ............|......|............
+ * ============|(core)|============
+ * ============|( op )|============
+ * ^^^^^^^^^^^^|______|............
+ * .Input Side.
+ * ..Handler...
**/
-public class FeedRuntimeInputHandler implements IFrameWriter {
+public class FeedRuntimeInputHandler extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
private static final Logger LOGGER = Logger.getLogger(FeedRuntimeInputHandler.class.getName());
-
- private final FeedConnectionId connectionId;
- private final FeedRuntimeId runtimeId;
- private final FeedPolicyAccessor feedPolicyAccessor;
+ private static final ByteBuffer POISON_PILL = ByteBuffer.allocate(0);
+ private static final double MAX_SPILL_USED_BEFORE_RESUME = 0.8;
private final FeedExceptionHandler exceptionHandler;
- private final FeedFrameDiscarder discarder;
- private final FeedFrameSpiller spiller;
+ private final FrameSpiller spiller;
private final FeedPolicyAccessor fpa;
- private final IFeedManager feedManager;
- private final MonitoredBuffer mBuffer;
- private final DataBucketPool pool;
- private final FrameEventCallback frameEventCallback;
-
- private boolean bufferingEnabled;
- private FrameCollection frameCollection;
- private Mode mode;
- private Mode lastMode;
- private boolean finished;
- private long nProcessed;
- private boolean throttlingEnabled;
- protected IFrameWriter coreOperator;
+ private final FrameAction frameAction;
+ private final int initialFrameSize;
+ private final FrameTransporter consumer;
+ private final Thread consumerThread;
+ private final LinkedBlockingDeque<ByteBuffer> inbox;
+ private final ConcurrentFramePool memoryManager;
+ private Mode mode = Mode.PROCESS;
+ private int numDiscarded = 0;
+ private int numSpilled = 0;
+ private int numProcessedInMemory = 0;
+ private int numStalled = 0;
public FeedRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- IFrameWriter coreOperator, FeedPolicyAccessor fpa, boolean bufferingEnabled, FrameTupleAccessor fta,
- RecordDescriptor recordDesc, IFeedManager feedManager, int nPartitions) throws HyracksDataException {
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- this.coreOperator = coreOperator;
- this.bufferingEnabled = bufferingEnabled;
- this.feedPolicyAccessor = fpa;
- this.spiller = new FeedFrameSpiller(ctx, connectionId, runtimeId, fpa);
- this.discarder = new FeedFrameDiscarder(connectionId, runtimeId, fpa, this);
- this.exceptionHandler = new FeedExceptionHandler(ctx, fta, recordDesc, feedManager, connectionId);
- this.mode = Mode.PROCESS;
- this.lastMode = Mode.PROCESS;
- this.finished = false;
+ IFrameWriter writer, FeedPolicyAccessor fpa, FrameTupleAccessor fta, ConcurrentFramePool feedMemoryManager)
+ throws HyracksDataException {
+ this.writer = writer;
+ this.spiller =
+ new FrameSpiller(ctx,
+ connectionId.getFeedId() + "_" + connectionId.getDatasetName() + "_"
+ + runtimeId.getFeedRuntimeType() + "_" + runtimeId.getPartition(),
+ fpa.getMaxSpillOnDisk());
+ this.exceptionHandler = new FeedExceptionHandler(ctx, fta);
this.fpa = fpa;
- this.feedManager = feedManager;
- this.pool = (DataBucketPool) feedManager.getFeedMemoryManager()
- .getMemoryComponent(IFeedMemoryComponent.Type.POOL);
- this.frameCollection = (FrameCollection) feedManager.getFeedMemoryManager()
- .getMemoryComponent(IFeedMemoryComponent.Type.COLLECTION);
- this.frameEventCallback = new FrameEventCallback(fpa, this, coreOperator);
- this.mBuffer = MonitoredBuffer.getMonitoredBuffer(ctx, this, coreOperator, fta, recordDesc,
- feedManager.getFeedMetricCollector(), connectionId, runtimeId, exceptionHandler, frameEventCallback,
- nPartitions, fpa);
- this.throttlingEnabled = false;
+ this.memoryManager = feedMemoryManager;
+ this.inbox = new LinkedBlockingDeque<>();
+ this.consumer = new FrameTransporter();
+ this.consumerThread = new Thread();
+ this.consumerThread.start();
+ this.initialFrameSize = ctx.getInitialFrameSize();
+ this.frameAction = new FrameAction(inbox);
}
@Override
- public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ public void open() throws HyracksDataException {
+ synchronized (writer) {
+ writer.open();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ synchronized (writer) {
+ writer.fail();
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ inbox.add(POISON_PILL);
+ notify();
try {
+ consumerThread.join();
+ } catch (InterruptedException e) {
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
+ }
+ try {
+ memoryManager.release(inbox);
+ } catch (Throwable th) {
+ LOGGER.log(Level.WARNING, th.getMessage(), th);
+ }
+ try {
+ spiller.close();
+ } catch (Throwable th) {
+ LOGGER.log(Level.WARNING, th.getMessage(), th);
+ }
+ writer.close();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ try {
+ if (consumer.cause() != null) {
+ throw consumer.cause();
+ }
switch (mode) {
case PROCESS:
- switch (lastMode) {
- case SPILL:
- case POST_SPILL_DISCARD:
- setMode(Mode.PROCESS_SPILL);
- processSpilledBacklog();
- break;
- case STALL:
- setMode(Mode.PROCESS_BACKLOG);
- processBufferredBacklog();
- break;
- default:
- break;
- }
- process(frame);
- break;
- case PROCESS_BACKLOG:
- case PROCESS_SPILL:
process(frame);
break;
case SPILL:
spill(frame);
break;
case DISCARD:
- case POST_SPILL_DISCARD:
discard(frame);
break;
- case STALL:
- switch (runtimeId.getFeedRuntimeType()) {
- case COLLECT:
- case COMPUTE_COLLECT:
- case COMPUTE:
- case STORE:
- bufferDataUntilRecovery(frame);
- break;
- default:
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Discarding frame during " + mode + " mode " + this.runtimeId);
- }
- break;
- }
- break;
- case END:
- case FAIL:
+ default:
if (LOGGER.isLoggable(Level.WARNING)) {
LOGGER.warning("Ignoring incoming tuples in " + mode + " mode");
}
break;
}
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
+ } catch (Throwable th) {
+ throw new HyracksDataException(th);
}
}
- private void bufferDataUntilRecovery(ByteBuffer frame) throws Exception {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Bufferring data until recovery is complete " + this.runtimeId);
+ private ByteBuffer getFreeBuffer(int frameSize) throws HyracksDataException {
+ int numFrames = frameSize / initialFrameSize;
+ if (numFrames == 1) {
+ return memoryManager.get();
+ } else {
+ return memoryManager.get(frameSize);
}
- if (frameCollection == null) {
- this.frameCollection = (FrameCollection) feedManager.getFeedMemoryManager()
- .getMemoryComponent(IFeedMemoryComponent.Type.COLLECTION);
- }
- if (frameCollection == null) {
- discarder.processMessage(frame);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Running low on memory! DISCARDING FRAME ");
+ }
+
+ private void discard(ByteBuffer frame) throws HyracksDataException {
+ if (fpa.spillToDiskOnCongestion()) {
+ if (spiller.spill(frame)) {
+ numSpilled++;
+ mode = Mode.SPILL;
+ return;
}
} else {
- boolean success = frameCollection.addFrame(frame);
- if (!success) {
- if (fpa.spillToDiskOnCongestion()) {
- if (frame != null) {
- spiller.processMessage(frame);
- } // TODO handle the else casec
-
- } else {
- discarder.processMessage(frame);
- }
+ ByteBuffer next = getFreeBuffer(frame.capacity());
+ if (next != null) {
+ numProcessedInMemory++;
+ next.put(frame);
+ inbox.offer(next);
+ mode = Mode.PROCESS;
+ return;
}
}
+ numDiscarded++;
+ }
+
+ private synchronized void exitProcessState(ByteBuffer frame) throws HyracksDataException {
+ if (fpa.spillToDiskOnCongestion()) {
+ mode = Mode.SPILL;
+ spiller.open();
+ spill(frame);
+ } else {
+ discardOrStall(frame);
+ }
}
- public void reportUnresolvableCongestion() throws HyracksDataException {
- if (this.runtimeId.getFeedRuntimeType().equals(FeedRuntimeType.COMPUTE)) {
- FeedCongestionMessage congestionReport = new FeedCongestionMessage(connectionId, runtimeId,
- mBuffer.getInflowRate(), mBuffer.getOutflowRate(), mode);
- feedManager.getFeedMessageService().sendMessage(congestionReport);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Congestion reported " + this.connectionId + " " + this.runtimeId);
+ private void discardOrStall(ByteBuffer frame) throws HyracksDataException {
+ if (fpa.discardOnCongestion()) {
+ numDiscarded++;
+ mode = Mode.DISCARD;
+ discard(frame);
+ } else {
+ stall(frame);
+ }
+ }
+
+ private void stall(ByteBuffer frame) throws HyracksDataException {
+ try {
+ numStalled++;
+ // If spilling is enabled, we wait on the spiller
+ if (fpa.spillToDiskOnCongestion()) {
+ synchronized (spiller) {
+ while (spiller.usedBudget() > MAX_SPILL_USED_BEFORE_RESUME) {
+ spiller.wait();
+ }
+ }
+ spiller.spill(frame);
+ synchronized (this) {
+ notify();
+ }
+ return;
+ }
+ // Spilling is disabled, we subscribe to feedMemoryManager
+ frameAction.setFrame(frame);
+ synchronized (frameAction) {
+ if (memoryManager.subscribe(frameAction)) {
+ frameAction.wait();
+ }
+ }
+ synchronized (this) {
+ notify();
+ }
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void process(ByteBuffer frame) throws HyracksDataException {
+ // Get a page from
+ ByteBuffer next = getFreeBuffer(frame.capacity());
+ if (next != null) {
+ numProcessedInMemory++;
+ next.put(frame);
+ inbox.offer(next);
+ if (inbox.size() == 1) {
+ synchronized (this) {
+ notify();
+ }
}
} else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unresolvable congestion at " + this.connectionId + " " + this.runtimeId);
- }
+ // out of memory. we switch to next mode as per policy -- synchronized method
+ exitProcessState(frame);
}
}
- private void processBufferredBacklog() throws HyracksDataException {
- try {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Processing backlog " + this.runtimeId);
- }
-
- if (frameCollection != null) {
- Iterator<ByteBuffer> backlog = frameCollection.getFrameCollectionIterator();
- while (backlog.hasNext()) {
- process(backlog.next());
- nProcessed++;
- }
- DataBucket bucket = pool.getDataBucket();
- bucket.setContentType(ContentType.EOSD);
- bucket.setDesiredReadCount(1);
- mBuffer.sendMessage(bucket);
- feedManager.getFeedMemoryManager().releaseMemoryComponent(frameCollection);
- frameCollection = null;
- }
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
- }
-
- private void processSpilledBacklog() throws HyracksDataException {
- try {
- Iterator<ByteBuffer> backlog = spiller.replayData();
- while (backlog.hasNext()) {
- process(backlog.next());
- nProcessed++;
- }
- DataBucket bucket = pool.getDataBucket();
- bucket.setContentType(ContentType.EOSD);
- bucket.setDesiredReadCount(1);
- mBuffer.sendMessage(bucket);
- spiller.reset();
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
- }
-
- protected void process(ByteBuffer frame) throws HyracksDataException {
- boolean frameProcessed = false;
- while (!frameProcessed) {
- try {
- if (!bufferingEnabled) {
- if (frame == null) {
- setFinished(true);
- synchronized (coreOperator) {
- coreOperator.notifyAll();
- }
- } else {
- coreOperator.nextFrame(frame); // synchronous
- }
+ private void spill(ByteBuffer frame) throws HyracksDataException {
+ if (spiller.switchToMemory()) {
+ synchronized (this) {
+ // Check if there is memory
+ ByteBuffer next = getFreeBuffer(frame.capacity());
+ if (next != null) {
+ spiller.close();
+ numProcessedInMemory++;
+ next.put(frame);
+ inbox.offer(next);
+ mode = Mode.PROCESS;
} else {
- DataBucket bucket = pool.getDataBucket();
- if (bucket != null) {
- if (frame != null) {
- bucket.reset(frame); // created a copy here
- bucket.setContentType(ContentType.DATA);
- } else {
- bucket.setContentType(ContentType.EOD);
- setFinished(true);
- synchronized (coreOperator) {
- coreOperator.notifyAll();
- }
- }
- // TODO: fix handling of eod case with monitored buffers.
- bucket.setDesiredReadCount(1);
- mBuffer.sendMessage(bucket);
- mBuffer.sendReport(frame);
- nProcessed++;
- } else {
- if (fpa.spillToDiskOnCongestion()) {
- if (frame != null) {
- boolean spilled = spiller.processMessage(frame);
- if (spilled) {
- setMode(Mode.SPILL);
- } else {
- reportUnresolvableCongestion();
- }
- }
- } else if (fpa.discardOnCongestion()) {
- boolean discarded = discarder.processMessage(frame);
- if (!discarded) {
- reportUnresolvableCongestion();
- }
- } else if (fpa.throttlingEnabled()) {
- setThrottlingEnabled(true);
- } else {
- reportUnresolvableCongestion();
- }
-
- }
- }
- frameProcessed = true;
- } catch (Exception e) {
- e.printStackTrace();
- if (feedPolicyAccessor.continueOnSoftFailure()) {
- frame = exceptionHandler.handleException(e, frame);
- if (frame == null) {
- frameProcessed = true;
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Encountered exception! " + e.getMessage()
- + "Insufficient information, Cannot extract failing tuple");
- }
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Ingestion policy does not require recovering from tuple. Feed would terminate");
- }
- mBuffer.close(false);
- throw new HyracksDataException(e);
+ // spill. This will always succeed since spilled = 0 (must verify that budget can't be 0)
+ spiller.spill(frame);
+ numSpilled++;
+ notify();
}
}
- }
- }
-
- private void spill(ByteBuffer frame) throws Exception {
- boolean success = spiller.processMessage(frame);
- if (!success) {
- // limit reached
- setMode(Mode.POST_SPILL_DISCARD);
- reportUnresolvableCongestion();
- }
- }
-
- private void discard(ByteBuffer frame) throws Exception {
- boolean success = discarder.processMessage(frame);
- if (!success) { // limit reached
- reportUnresolvableCongestion();
+ } else {
+ // try to spill. If failed switch to either discard or stall
+ if (spiller.spill(frame)) {
+ numSpilled++;
+ } else {
+ if (fpa.discardOnCongestion()) {
+ discard(frame);
+ } else {
+ stall(frame);
+ }
+ }
}
}
@@ -349,120 +283,105 @@
}
public synchronized void setMode(Mode mode) {
- if (mode.equals(this.mode)) {
- return;
- }
- this.lastMode = this.mode;
this.mode = mode;
- if (mode.equals(Mode.END)) {
- this.close();
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Switched from " + lastMode + " to " + mode + " " + this.runtimeId);
- }
- }
-
- @Override
- public void close() {
- boolean disableMonitoring = !this.mode.equals(Mode.STALL);
- if (frameCollection != null) {
- feedManager.getFeedMemoryManager().releaseMemoryComponent(frameCollection);
- }
- if (pool != null) {
- feedManager.getFeedMemoryManager().releaseMemoryComponent(pool);
- }
- mBuffer.close(false, disableMonitoring);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Closed input side handler for " + this.runtimeId + " disabled monitoring " + disableMonitoring
- + " Mode for runtime " + this.mode);
- }
- }
-
- public IFrameWriter getCoreOperator() {
- return coreOperator;
- }
-
- public void setCoreOperator(IFrameWriter coreOperator) {
- this.coreOperator = coreOperator;
- mBuffer.setFrameWriter(coreOperator);
- frameEventCallback.setCoreOperator(coreOperator);
- }
-
- public boolean isFinished() {
- return finished;
- }
-
- public void setFinished(boolean finished) {
- this.finished = finished;
- }
-
- public long getProcessed() {
- return nProcessed;
- }
-
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- @Override
- public void open() throws HyracksDataException {
- coreOperator.open();
- mBuffer.start();
- }
-
- @Override
- public void fail() throws HyracksDataException {
- coreOperator.fail();
- }
-
- public void reset(int nPartitions) {
- this.mBuffer.setNumberOfPartitions(nPartitions);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Reset number of partitions to " + nPartitions + " for " + this.runtimeId);
- }
- mBuffer.reset();
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public IFeedManager getFeedManager() {
- return feedManager;
- }
-
- public MonitoredBuffer getmBuffer() {
- return mBuffer;
- }
-
- public boolean isThrottlingEnabled() {
- return throttlingEnabled;
- }
-
- public void setThrottlingEnabled(boolean throttlingEnabled) {
- if (this.throttlingEnabled != throttlingEnabled) {
- this.throttlingEnabled = throttlingEnabled;
- IFeedMessage throttlingEnabledMesg = new ThrottlingEnabledFeedMessage(connectionId, runtimeId);
- feedManager.getFeedMessageService().sendMessage(throttlingEnabledMesg);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Throttling " + throttlingEnabled + " for " + this.connectionId + "[" + runtimeId + "]");
- }
- }
- }
-
- public boolean isBufferingEnabled() {
- return bufferingEnabled;
- }
-
- public void setBufferingEnabled(boolean bufferingEnabled) {
- this.bufferingEnabled = bufferingEnabled;
}
@Override
public void flush() throws HyracksDataException {
- // Only flush when in process mode.
- if (mode == Mode.PROCESS) {
- coreOperator.flush();
+ synchronized (writer) {
+ writer.flush();
+ }
+ }
+
+ public int getNumDiscarded() {
+ return numDiscarded;
+ }
+
+ public int getNumSpilled() {
+ return numSpilled;
+ }
+
+ public int getNumProcessedInMemory() {
+ return numProcessedInMemory;
+ }
+
+ public int getNumStalled() {
+ return numStalled;
+ }
+
+ private class FrameTransporter implements Runnable {
+ private volatile Throwable cause;
+
+ public Throwable cause() {
+ return cause;
+ }
+
+ private Throwable consume(ByteBuffer frame) {
+ while (frame != null) {
+ try {
+ writer.nextFrame(frame);
+ frame = null;
+ } catch (HyracksDataException e) {
+ // It is fine to catch throwable here since this thread is always expected to terminate gracefully
+ frame = exceptionHandler.handle(e, frame);
+ if (frame == null) {
+ this.cause = e;
+ return e;
+ }
+ } catch (Throwable th) {
+ this.cause = th;
+ return th;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public void run() {
+ try {
+ ByteBuffer frame = inbox.poll();
+ while (frame != POISON_PILL) {
+ if (frame != null) {
+ try {
+ if (consume(frame) != null) {
+ return;
+ }
+ } finally {
+ // Done with frame.
+ memoryManager.release(frame);
+ }
+ }
+ frame = inbox.poll();
+ if (frame == null) {
+ // Memory queue is empty. Check spill
+ frame = spiller.next();
+ while (frame != null) {
+ if (consume(frame) != null) {
+ // We don't release the frame since this is a spill frame that we didn't get from memory
+ // manager
+ return;
+ }
+ frame = spiller.next();
+ }
+ writer.flush();
+ // At this point. We consumed all memory and spilled
+ // We can't assume the next will be in memory. what if there is 0 memory?
+ synchronized (FeedRuntimeInputHandler.this) {
+ frame = inbox.poll();
+ if (frame == null) {
+ // Nothing in memory
+ if (spiller.switchToMemory()) {
+ // Nothing in disk
+ FeedRuntimeInputHandler.this.wait();
+ }
+ }
+ }
+ }
+ }
+ } catch (Throwable th) {
+ this.cause = th;
+ }
+ // cleanup will always be done through the close() call
}
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java
new file mode 100644
index 0000000..4a2120a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameAction.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.external.feed.dataflow;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import rx.functions.Action1;
+
+public class FrameAction implements Action1<ByteBuffer> {
+ private final LinkedBlockingDeque<ByteBuffer> inbox;
+ private ByteBuffer frame;
+
+ public FrameAction(LinkedBlockingDeque<ByteBuffer> inbox) {
+ this.inbox = inbox;
+ }
+
+ @Override
+ public void call(ByteBuffer freeFrame) {
+ freeFrame.put(frame);
+ inbox.add(freeFrame);
+ synchronized (this) {
+ notify();
+ }
+ }
+
+ public ByteBuffer getFrame() {
+ return frame;
+ }
+
+ public void setFrame(ByteBuffer frame) {
+ this.frame = frame;
+ }
+
+ public int getSize() {
+ return frame.capacity();
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameCollection.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameCollection.java
deleted file mode 100644
index 7980712..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameCollection.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent;
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
-
-/**
- * Represents an expandable collection of frames.
- */
-public class FrameCollection implements IFeedMemoryComponent {
-
- /** A unique identifier for the feed memory component **/
- private final int componentId;
-
- /** A collection of frames (each being a ByteBuffer) **/
- private final List<ByteBuffer> frames = new LinkedList<ByteBuffer>();
-
- /** The permitted maximum size, the collection may grow to **/
- private int maxSize;
-
- /** The {@link IFeedMemoryManager} for the NodeController **/
- private final IFeedMemoryManager memoryManager;
-
- public FrameCollection(int componentId, IFeedMemoryManager memoryManager, int maxSize) {
- this.componentId = componentId;
- this.maxSize = maxSize;
- this.memoryManager = memoryManager;
- }
-
- public boolean addFrame(ByteBuffer frame) {
- if (frames.size() == maxSize) {
- boolean expansionGranted = memoryManager.expandMemoryComponent(this);
- if (!expansionGranted) {
- return false;
- }
- }
- ByteBuffer storageBuffer = ByteBuffer.allocate(frame.capacity());
- storageBuffer.put(frame);
- frames.add(storageBuffer);
- storageBuffer.flip();
- return true;
- }
-
- public Iterator<ByteBuffer> getFrameCollectionIterator() {
- return frames.iterator();
- }
-
- @Override
- public int getTotalAllocation() {
- return frames.size();
- }
-
- @Override
- public Type getType() {
- return Type.COLLECTION;
- }
-
- @Override
- public int getComponentId() {
- return componentId;
- }
-
- @Override
- public void expand(int delta) {
- maxSize = maxSize + delta;
- }
-
- @Override
- public void reset() {
- frames.clear();
- maxSize = IFeedMemoryManager.START_COLLECTION_SIZE;
- }
-
- @Override
- public String toString() {
- return "FrameCollection" + "[" + componentId + "]" + "(" + frames.size() + "/" + maxSize + ")";
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
index 802a791..c8b7406 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameDistributor.java
@@ -21,360 +21,162 @@
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent.Type;
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.dataflow.FeedFrameCollector.State;
+import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.log4j.Logger;
-public class FrameDistributor {
+public class FrameDistributor implements IFrameWriter {
- private static final Logger LOGGER = Logger.getLogger(FrameDistributor.class.getName());
-
- private static final long MEMORY_AVAILABLE_POLL_PERIOD = 1000; // 1 second
-
- private final FeedId feedId;
- private final FeedRuntimeType feedRuntimeType;
- private final int partition;
- private final IFeedMemoryManager memoryManager;
- private final boolean enableSynchronousTransfer;
+ public static final Logger LOGGER = Logger.getLogger(FrameDistributor.class.getName());
/** A map storing the registered frame readers ({@code FeedFrameCollector}. **/
- private final Map<IFrameWriter, FeedFrameCollector> registeredCollectors;
- private final FrameTupleAccessor fta;
+ private final Map<FeedConnectionId, FeedFrameCollector> registeredCollectors;
+ private Throwable rootFailureCause = null;
- private DataBucketPool pool;
- private DistributionMode distributionMode;
- private boolean spillToDiskRequired = false;
-
- public enum DistributionMode {
- /**
- * A single feed frame collector is registered for receiving tuples.
- * Tuple is sent via synchronous call, that is no buffering is involved
- **/
- SINGLE,
-
- /**
- * Multiple feed frame collectors are concurrently registered for
- * receiving tuples.
- **/
- SHARED,
-
- /**
- * Feed tuples are not being processed, irrespective of # of registered
- * feed frame collectors.
- **/
- INACTIVE
+ public FrameDistributor() throws HyracksDataException {
+ this.registeredCollectors = new HashMap<FeedConnectionId, FeedFrameCollector>();
}
- public FrameDistributor(FeedId feedId, FeedRuntimeType feedRuntimeType, int partition,
- boolean enableSynchronousTransfer, IFeedMemoryManager memoryManager, FrameTupleAccessor fta)
- throws HyracksDataException {
- this.feedId = feedId;
- this.feedRuntimeType = feedRuntimeType;
- this.partition = partition;
- this.memoryManager = memoryManager;
- this.enableSynchronousTransfer = enableSynchronousTransfer;
- this.registeredCollectors = new HashMap<IFrameWriter, FeedFrameCollector>();
- this.distributionMode = DistributionMode.INACTIVE;
- this.fta = fta;
- }
-
- public void notifyEndOfFeed() throws InterruptedException {
- DataBucket bucket = getDataBucket();
- if (bucket != null) {
- sendEndOfFeedDataBucket(bucket);
- } else {
- while (bucket == null) {
+ public synchronized void registerFrameCollector(FeedFrameCollector frameCollector) throws HyracksDataException {
+ if (rootFailureCause != null) {
+ throw new HyracksDataException("attempt to register to a failed feed data provider", rootFailureCause);
+ }
+ // registering a new collector.
+ try {
+ frameCollector.open();
+ } catch (Throwable th) {
+ rootFailureCause = th;
+ try {
+ frameCollector.fail();
+ } catch (Throwable failThrowable) {
+ th.addSuppressed(failThrowable);
+ } finally {
try {
- Thread.sleep(MEMORY_AVAILABLE_POLL_PERIOD);
- bucket = getDataBucket();
- } catch (InterruptedException e) {
- break;
+ frameCollector.close();
+ } catch (Throwable closeThrowable) {
+ th.addSuppressed(closeThrowable);
}
}
- if (bucket != null) {
- sendEndOfFeedDataBucket(bucket);
- }
+ throw th;
+ }
+ registeredCollectors.put(frameCollector.getConnectionId(), frameCollector);
+ }
+
+ public synchronized void deregisterFrameCollector(FeedFrameCollector frameCollector) throws HyracksDataException {
+ deregisterFrameCollector(frameCollector.getConnectionId());
+ }
+
+ public synchronized void deregisterFrameCollector(FeedConnectionId connectionId) throws HyracksDataException {
+ if (rootFailureCause != null) {
+ throw new HyracksDataException("attempt to register to a failed feed data provider", rootFailureCause);
+ }
+ FeedFrameCollector frameCollector = removeFrameCollector(connectionId);
+ try {
+ frameCollector.close();
+ } catch (Throwable th) {
+ rootFailureCause = th;
+ throw th;
}
}
- private void sendEndOfFeedDataBucket(DataBucket bucket) throws InterruptedException {
- bucket.setContentType(DataBucket.ContentType.EOD);
- nextBucket(bucket);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("End of feed data packet sent " + this.feedId);
- }
+ public synchronized FeedFrameCollector removeFrameCollector(FeedConnectionId connectionId) {
+ return registeredCollectors.remove(connectionId);
}
- public synchronized void registerFrameCollector(FeedFrameCollector frameCollector) {
- DistributionMode currentMode = distributionMode;
- switch (distributionMode) {
- case INACTIVE:
- if (!enableSynchronousTransfer) {
- pool = (DataBucketPool) memoryManager.getMemoryComponent(Type.POOL);
- frameCollector.start();
- }
- registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
- setMode(DistributionMode.SINGLE);
- break;
- case SINGLE:
- pool = (DataBucketPool) memoryManager.getMemoryComponent(Type.POOL);
- registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
- for (FeedFrameCollector reader : registeredCollectors.values()) {
- reader.start();
- }
- setMode(DistributionMode.SHARED);
- break;
- case SHARED:
- frameCollector.start();
- registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
- break;
- }
- evaluateIfSpillIsEnabled();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(
- "Switching to " + distributionMode + " mode from " + currentMode + " mode " + " Feed id " + feedId);
- }
- }
-
- public synchronized void deregisterFrameCollector(FeedFrameCollector frameCollector) {
- switch (distributionMode) {
- case INACTIVE:
- throw new IllegalStateException(
- "Invalid attempt to deregister frame collector in " + distributionMode + " mode.");
- case SHARED:
- frameCollector.closeCollector();
- registeredCollectors.remove(frameCollector.getFrameWriter());
- int nCollectors = registeredCollectors.size();
- if (nCollectors == 1) {
- FeedFrameCollector loneCollector = registeredCollectors.values().iterator().next();
- setMode(DistributionMode.SINGLE);
- loneCollector.setState(FeedFrameCollector.State.TRANSITION);
- loneCollector.closeCollector();
- memoryManager.releaseMemoryComponent(pool);
- evaluateIfSpillIsEnabled();
- } else {
- if (!spillToDiskRequired) {
- evaluateIfSpillIsEnabled();
- }
- }
- break;
- case SINGLE:
- frameCollector.closeCollector();
- setMode(DistributionMode.INACTIVE);
- spillToDiskRequired = false;
- break;
-
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Deregistered frame reader" + frameCollector + " from feed distributor for " + feedId);
- }
- }
-
- public void evaluateIfSpillIsEnabled() {
- if (!spillToDiskRequired) {
- for (FeedFrameCollector collector : registeredCollectors.values()) {
- spillToDiskRequired = spillToDiskRequired
- || collector.getFeedPolicyAccessor().spillToDiskOnCongestion();
- if (spillToDiskRequired) {
- break;
- }
- }
- }
- }
-
- public boolean deregisterFrameCollector(IFrameWriter frameWriter) {
- FeedFrameCollector collector = registeredCollectors.get(frameWriter);
- if (collector != null) {
- deregisterFrameCollector(collector);
- return true;
- }
- return false;
- }
-
- public synchronized void setMode(DistributionMode mode) {
- this.distributionMode = mode;
- }
-
- public boolean isRegistered(IFrameWriter writer) {
- return registeredCollectors.get(writer) != null;
- }
-
+ /*
+ * Fix. What should be done?:
+ * 0. mark failure so no one can subscribe or unsubscribe.
+ * 1. Throw the throwable.
+ * 2. when fail() is called, call fail on all subscribers
+ * 3. close all the subscribers.
+ * (non-Javadoc)
+ * @see org.apache.hyracks.api.comm.IFrameWriter#nextFrame(java.nio.ByteBuffer)
+ */
+ @Override
public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
- try {
- switch (distributionMode) {
- case INACTIVE:
- break;
- case SINGLE:
- FeedFrameCollector collector = registeredCollectors.values().iterator().next();
- switch (collector.getState()) {
- case HANDOVER:
- case ACTIVE:
- if (enableSynchronousTransfer) {
- collector.nextFrame(frame); // processing is synchronous
- } else {
- handleDataBucket(frame);
- }
- break;
- case TRANSITION:
- handleDataBucket(frame);
- break;
- case FINISHED:
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Discarding fetched tuples, feed has ended ["
- + registeredCollectors.get(0) + "]" + " Feed Id " + feedId
- + " frame distributor " + this.getFeedRuntimeType());
- }
- registeredCollectors.remove(0);
- break;
- }
- break;
- case SHARED:
- handleDataBucket(frame);
- break;
- }
- } catch (Exception e) {
- throw new HyracksDataException(e);
+ if (rootFailureCause != null) {
+ throw new HyracksDataException(rootFailureCause);
}
- }
-
- private void nextBucket(DataBucket bucket) throws InterruptedException {
for (FeedFrameCollector collector : registeredCollectors.values()) {
- collector.sendMessage(bucket); // asynchronous call
- }
- }
-
- private void handleDataBucket(ByteBuffer frame) throws HyracksDataException, InterruptedException {
- DataBucket bucket = getDataBucket();
- if (bucket == null) {
- handleFrameDuringMemoryCongestion(frame);
- } else {
- bucket.reset(frame);
- bucket.setDesiredReadCount(registeredCollectors.size());
- nextBucket(bucket);
- }
- }
-
- private void handleFrameDuringMemoryCongestion(ByteBuffer frame) throws HyracksDataException {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to allocate memory, will evaluate the need to spill");
- }
- // wait till memory is available
- }
-
- private DataBucket getDataBucket() {
- DataBucket bucket = null;
- if (pool != null) {
- bucket = pool.getDataBucket();
- if (bucket != null) {
- bucket.setDesiredReadCount(registeredCollectors.size());
- return bucket;
- } else {
- return null;
+ try {
+ collector.nextFrame(frame);
+ } catch (Throwable th) {
+ rootFailureCause = th;
+ throw th;
}
}
- return null;
}
- public DistributionMode getMode() {
- return distributionMode;
- }
-
- public void close() throws HyracksDataException {
- try {
- switch (distributionMode) {
- case INACTIVE:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("FrameDistributor is " + distributionMode);
+ @Override
+ public void fail() throws HyracksDataException {
+ Collection<FeedFrameCollector> collectors = registeredCollectors.values();
+ Iterator<FeedFrameCollector> it = collectors.iterator();
+ while (it.hasNext()) {
+ FeedFrameCollector collector = it.next();
+ try {
+ collector.fail();
+ } catch (Throwable th) {
+ while (it.hasNext()) {
+ FeedFrameCollector innerCollector = it.next();
+ try {
+ innerCollector.fail();
+ } catch (Throwable innerTh) {
+ th.addSuppressed(innerTh);
}
- break;
- case SINGLE:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disconnecting single frame reader in " + distributionMode + " mode "
- + " for feedId " + feedId + " " + this.feedRuntimeType);
- }
- setMode(DistributionMode.INACTIVE);
- if (!enableSynchronousTransfer) {
- notifyEndOfFeed(); // send EOD Data Bucket
- waitForCollectorsToFinish();
- }
- registeredCollectors.values().iterator().next().disconnect();
- break;
- case SHARED:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Signalling End Of Feed; currently operating in " + distributionMode + " mode");
- }
- notifyEndOfFeed(); // send EOD Data Bucket
- waitForCollectorsToFinish();
- break;
- }
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- }
-
- private void waitForCollectorsToFinish() {
- synchronized (registeredCollectors.values()) {
- while (!allCollectorsFinished()) {
- try {
- registeredCollectors.values().wait();
- } catch (InterruptedException e) {
- e.printStackTrace();
}
+ throw th;
}
}
}
- private boolean allCollectorsFinished() {
- boolean allFinished = true;
- for (FeedFrameCollector collector : registeredCollectors.values()) {
- allFinished = allFinished && collector.getState().equals(FeedFrameCollector.State.FINISHED);
+ @Override
+ public void close() throws HyracksDataException {
+ Collection<FeedFrameCollector> collectors = registeredCollectors.values();
+ Iterator<FeedFrameCollector> it = collectors.iterator();
+ while (it.hasNext()) {
+ FeedFrameCollector collector = it.next();
+ try {
+ collector.close();
+ } catch (Throwable th) {
+ while (it.hasNext()) {
+ FeedFrameCollector innerCollector = it.next();
+ try {
+ innerCollector.close();
+ } catch (Throwable innerTh) {
+ th.addSuppressed(innerTh);
+ } finally {
+ innerCollector.setState(State.FINISHED);
+ }
+ }
+ // resume here
+ throw th;
+ } finally {
+ collector.setState(State.FINISHED);
+ }
}
- return allFinished;
}
- public Collection<FeedFrameCollector> getRegisteredCollectors() {
- return registeredCollectors.values();
- }
-
- public Map<IFrameWriter, FeedFrameCollector> getRegisteredReaders() {
- return registeredCollectors;
- }
-
- public FeedId getFeedId() {
- return feedId;
- }
-
- public DistributionMode getDistributionMode() {
- return distributionMode;
- }
-
- public FeedRuntimeType getFeedRuntimeType() {
- return feedRuntimeType;
- }
-
- public int getPartition() {
- return partition;
- }
-
- public FrameTupleAccessor getFta() {
- return fta;
- }
-
+ @Override
public void flush() throws HyracksDataException {
- switch (distributionMode) {
- case SINGLE:
- FeedFrameCollector collector = registeredCollectors.values().iterator().next();
+ if (rootFailureCause != null) {
+ throw new HyracksDataException(rootFailureCause);
+ }
+ for (FeedFrameCollector collector : registeredCollectors.values()) {
+ try {
collector.flush();
- default:
- break;
+ } catch (Throwable th) {
+ rootFailureCause = th;
+ throw th;
+ }
}
}
+ @Override
+ public void open() throws HyracksDataException {
+ // Nothing to do here :)
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameEventCallback.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameEventCallback.java
deleted file mode 100644
index f1499fb..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameEventCallback.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class FrameEventCallback implements IFrameEventCallback {
-
- private static final Logger LOGGER = Logger.getLogger(FrameEventCallback.class.getName());
-
- private final FeedPolicyAccessor fpa;
- private final FeedRuntimeInputHandler inputSideHandler;
- private IFrameWriter coreOperator;
-
- public FrameEventCallback(FeedPolicyAccessor fpa, FeedRuntimeInputHandler inputSideHandler,
- IFrameWriter coreOperator) {
- this.fpa = fpa;
- this.inputSideHandler = inputSideHandler;
- this.coreOperator = coreOperator;
- }
-
- @Override
- public void frameEvent(FrameEvent event) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Frame Event for " + inputSideHandler.getRuntimeId() + " " + event);
- }
- if (!event.equals(FrameEvent.FINISHED_PROCESSING_SPILLAGE)
- && inputSideHandler.getMode().equals(Mode.PROCESS_SPILL)) {
- return;
- }
- switch (event) {
- case PENDING_WORK_THRESHOLD_REACHED:
- if (fpa.spillToDiskOnCongestion()) {
- inputSideHandler.setMode(Mode.SPILL);
- } else if (fpa.discardOnCongestion()) {
- inputSideHandler.setMode(Mode.DISCARD);
- } else if (fpa.throttlingEnabled()) {
- inputSideHandler.setThrottlingEnabled(true);
- } else {
- try {
- inputSideHandler.reportUnresolvableCongestion();
- } catch (HyracksDataException e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to report congestion!!!");
- }
- }
- }
- break;
- case FINISHED_PROCESSING:
- inputSideHandler.setFinished(true);
- synchronized (coreOperator) {
- coreOperator.notifyAll();
- }
- break;
- case PENDING_WORK_DONE:
- switch (inputSideHandler.getMode()) {
- case SPILL:
- case DISCARD:
- case POST_SPILL_DISCARD:
- inputSideHandler.setMode(Mode.PROCESS);
- break;
- default:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Received " + event + " ignoring as operating in " + inputSideHandler.getMode());
- }
- }
- break;
- case FINISHED_PROCESSING_SPILLAGE:
- inputSideHandler.setMode(Mode.PROCESS);
- break;
- default:
- break;
- }
- }
-
- public void setCoreOperator(IFrameWriter coreOperator) {
- this.coreOperator = coreOperator;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameSpiller.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameSpiller.java
new file mode 100644
index 0000000..f0d226a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FrameSpiller.java
@@ -0,0 +1,217 @@
+/*
+ * 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.asterix.external.feed.dataflow;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.util.ArrayDeque;
+
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+/**
+ * A {@link FrameSpiller} is used with feeds when "spill.to.disk.on.congestion" is set to true. The spiller spills
+ * excess tuples to disk if an operator
+ * cannot process incoming data at its arrival rate. The maximum size of data (tuples) that can be spilled to disk is
+ * configured using the property
+ * "max.spill.size.on.disk"
+ */
+public class FrameSpiller {
+ private static final Logger LOGGER = Logger.getLogger(FrameSpiller.class.getName());
+ private static final int FRAMES_PER_FILE = 1024;
+
+ private final String fileNamePrefix;
+ private final ArrayDeque<File> files = new ArrayDeque<>();
+ private final VSizeFrame frame;
+ private final int budget; // Max current frames in disk allowed
+ private BufferedOutputStream bos; // Current output stream
+ private BufferedInputStream bis; // Current input stream
+ private File currentWriteFile; // Current write file
+ private File currentReadFile; // Current read file
+ private int currentWriteCount = 0; // Current file write count
+ private int currentReadCount = 0; // Current file read count
+ private int totalWriteCount = 0; // Total frames spilled
+ private int totalReadCount = 0; // Total frames read
+ private int fileCount = 0; // How many spill files?
+
+ public FrameSpiller(IHyracksTaskContext ctx, String fileNamePrefix, long budgetInBytes)
+ throws HyracksDataException {
+ this.frame = new VSizeFrame(ctx);
+ this.fileNamePrefix = fileNamePrefix;
+ this.budget = (int) (budgetInBytes / ctx.getInitialFrameSize());
+
+ }
+
+ public void open() throws HyracksDataException {
+ try {
+ this.currentWriteFile = createFile();
+ this.currentReadFile = currentWriteFile;
+ this.bos = new BufferedOutputStream(new FileOutputStream(currentWriteFile));
+ this.bis = new BufferedInputStream(new FileInputStream(currentReadFile));
+ } catch (Exception e) {
+ LOGGER.fatal("Unable to create spill file", e);
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public boolean switchToMemory() {
+ return totalWriteCount == totalReadCount;
+ }
+
+ public int remaining() {
+ return totalWriteCount - totalReadCount;
+ }
+
+ public synchronized ByteBuffer next() throws HyracksDataException {
+ try {
+ frame.reset();
+ if (totalReadCount == totalWriteCount) {
+ return null;
+ }
+ if (currentReadFile == null) {
+ if (!files.isEmpty()) {
+ currentReadFile = files.pop();
+ bis = new BufferedInputStream(new FileInputStream(currentReadFile));
+ } else {
+ return null;
+ }
+ }
+ // read first frame
+ bis.read(frame.getBuffer().array(), 0, frame.getFrameSize());
+ byte frameCount = frame.getBuffer().array()[0];
+ if (frameCount > 1) {
+ // expand the frame keeping existing data
+ frame.ensureFrameSize(frame.getMinSize() * frameCount);
+ bis.read(frame.getBuffer().array(), frame.getMinSize(), frame.getFrameSize() - frame.getMinSize());
+ }
+ currentReadCount++;
+ totalReadCount++;
+ if (currentReadCount >= FRAMES_PER_FILE) {
+ currentReadCount = 0;
+ // done with the file
+ bis.close();
+ Files.delete(currentReadFile.toPath());
+ if (!files.isEmpty()) {
+ currentReadFile = files.pop();
+ bis = new BufferedInputStream(new FileInputStream(currentReadFile));
+ } else {
+ currentReadFile = null;
+ }
+ }
+ return frame.getBuffer();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public double usedBudget() {
+ return ((double) (totalWriteCount - totalReadCount) / (double) budget);
+ }
+
+ public synchronized boolean spill(ByteBuffer frame) throws HyracksDataException {
+ try {
+ if (totalWriteCount - totalReadCount >= budget) {
+ return false;
+ }
+ currentWriteCount++;
+ totalWriteCount++;
+ bos.write(frame.array());
+ bos.flush();
+ if (currentWriteCount >= FRAMES_PER_FILE) {
+ bos.close();
+ currentWriteCount = 0;
+ currentWriteFile = createFile();
+ files.add(currentWriteFile);
+ bos = new BufferedOutputStream(new FileOutputStream(currentWriteFile));
+ }
+ return true;
+ } catch (IOException e) {
+ close();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private File createFile() throws HyracksDataException {
+ try {
+ String fileName = fileNamePrefix + "_" + fileCount++;
+ File file = new File(fileName);
+ if (!file.exists()) {
+ boolean success = file.createNewFile();
+ if (!success) {
+ throw new HyracksDataException("Unable to create spill file " + fileName);
+ } else {
+ if (LOGGER.isEnabledFor(Level.INFO)) {
+ LOGGER.info("Created spill file " + file.getAbsolutePath());
+ }
+ }
+ }
+ return file;
+ } catch (Throwable th) {
+ throw new HyracksDataException(th);
+ }
+ }
+
+ public synchronized void close() {
+ // Do proper cleanup
+ if (bos != null) {
+ try {
+ bos.flush();
+ bos.close();
+ } catch (IOException e) {
+ LOGGER.warn(e.getMessage(), e);
+ }
+ }
+ if (bis != null) {
+ try {
+ bis.close();
+ } catch (IOException e) {
+ LOGGER.warn(e.getMessage(), e);
+ }
+ }
+ if (currentReadFile != null) {
+ try {
+ Files.deleteIfExists(currentReadFile.toPath());
+ } catch (Exception e) {
+ LOGGER.warn(e.getMessage(), e);
+ }
+ currentReadFile = null;
+ }
+ while (!files.isEmpty()) {
+ File file = files.pop();
+ try {
+ Files.deleteIfExists(file.toPath());
+ } catch (Exception e) {
+ LOGGER.warn(e.getMessage(), e);
+ }
+ }
+ currentWriteCount = 0;
+ currentReadCount = 0;
+ totalWriteCount = 0;
+ totalReadCount = 0;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/StorageFrameHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/StorageFrameHandler.java
deleted file mode 100644
index 22dcfac..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/StorageFrameHandler.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.asterix.external.feed.dataflow;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.asterix.external.feed.watch.IntakePartitionStatistics;
-import org.apache.asterix.external.util.FeedConstants.StatisticsConstants;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class StorageFrameHandler {
-
- private final Map<Integer, Map<Integer, IntakePartitionStatistics>> intakeStatistics;
- private long avgDelayPersistence;
-
- public StorageFrameHandler() {
- intakeStatistics = new HashMap<Integer, Map<Integer, IntakePartitionStatistics>>();
- avgDelayPersistence = 0L;
- }
-
- public synchronized void updateTrackingInformation(ByteBuffer frame, FrameTupleAccessor frameAccessor) {
- int nTuples = frameAccessor.getTupleCount();
- long delay = 0;
- long intakeTimestamp;
- long currentTime = System.currentTimeMillis();
- int partition = 0;
- int recordId = 0;
- for (int i = 0; i < nTuples; i++) {
- int recordStart = frameAccessor.getTupleStartOffset(i) + frameAccessor.getFieldSlotsLength();
- int openPartOffsetOrig = frame.getInt(recordStart + 6);
- int numOpenFields = frame.getInt(recordStart + openPartOffsetOrig);
-
- int recordIdOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
- + (StatisticsConstants.INTAKE_TUPLEID.length() + 2) + 1;
- recordId = frame.getInt(recordStart + recordIdOffset);
-
- int partitionOffset = recordIdOffset + 4 + (StatisticsConstants.INTAKE_PARTITION.length() + 2) + 1;
- partition = frame.getInt(recordStart + partitionOffset);
-
- ackRecordId(partition, recordId);
- int intakeTimestampValueOffset = partitionOffset + 4 + (StatisticsConstants.INTAKE_TIMESTAMP.length() + 2)
- + 1;
- intakeTimestamp = frame.getLong(recordStart + intakeTimestampValueOffset);
-
- int storeTimestampValueOffset = intakeTimestampValueOffset + 8
- + (StatisticsConstants.STORE_TIMESTAMP.length() + 2) + 1;
- frame.putLong(recordStart + storeTimestampValueOffset, System.currentTimeMillis());
- delay += currentTime - intakeTimestamp;
- }
- avgDelayPersistence = delay / nTuples;
- }
-
- private void ackRecordId(int partition, int recordId) {
- Map<Integer, IntakePartitionStatistics> map = intakeStatistics.get(partition);
- if (map == null) {
- map = new HashMap<Integer, IntakePartitionStatistics>();
- intakeStatistics.put(partition, map);
- }
- int base = (int) Math.ceil(recordId * 1.0 / IntakePartitionStatistics.ACK_WINDOW_SIZE);
- IntakePartitionStatistics intakeStatsForBaseOfPartition = map.get(base);
- if (intakeStatsForBaseOfPartition == null) {
- intakeStatsForBaseOfPartition = new IntakePartitionStatistics(partition, base);
- map.put(base, intakeStatsForBaseOfPartition);
- }
- intakeStatsForBaseOfPartition.ackRecordId(recordId);
- }
-
- public byte[] getAckData(int partition, int base) {
- Map<Integer, IntakePartitionStatistics> intakeStats = intakeStatistics.get(partition);
- if (intakeStats != null) {
- IntakePartitionStatistics intakePartitionStats = intakeStats.get(base);
- if (intakePartitionStats != null) {
- return intakePartitionStats.getAckInfo();
- }
- }
- return null;
- }
-
- public synchronized Map<Integer, IntakePartitionStatistics> getBaseAcksForPartition(int partition) {
- Map<Integer, IntakePartitionStatistics> intakeStatsForPartition = intakeStatistics.get(partition);
- Map<Integer, IntakePartitionStatistics> clone = new HashMap<Integer, IntakePartitionStatistics>();
- for (Entry<Integer, IntakePartitionStatistics> entry : intakeStatsForPartition.entrySet()) {
- clone.put(entry.getKey(), entry.getValue());
- }
- return intakeStatsForPartition;
- }
-
- public long getAvgDelayPersistence() {
- return avgDelayPersistence;
- }
-
- public void setAvgDelayPersistence(long avgDelayPersistence) {
- this.avgDelayPersistence = avgDelayPersistence;
- }
-
- public Set<Integer> getPartitionsWithStats() {
- return intakeStatistics.keySet();
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/SyncFeedRuntimeInputHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/SyncFeedRuntimeInputHandler.java
new file mode 100644
index 0000000..1bdc7e1
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/SyncFeedRuntimeInputHandler.java
@@ -0,0 +1,72 @@
+/*
+ * 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.asterix.external.feed.dataflow;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class SyncFeedRuntimeInputHandler extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+ private final FeedExceptionHandler exceptionHandler;
+
+ public SyncFeedRuntimeInputHandler(IHyracksTaskContext ctx, IFrameWriter writer, FrameTupleAccessor fta) {
+ this.writer = writer;
+ this.exceptionHandler = new FeedExceptionHandler(ctx, fta);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ while (frame != null) {
+ try {
+ writer.nextFrame(frame);
+ return;
+ } catch (HyracksDataException e) {
+ frame = exceptionHandler.handle(e, frame);
+ if (frame == null) {
+ throw e;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ writer.close();
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ writer.flush();
+ }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java
new file mode 100644
index 0000000..25aa86a
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/ConcurrentFramePool.java
@@ -0,0 +1,204 @@
+/*
+ * 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.asterix.external.feed.management;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.asterix.external.feed.dataflow.FrameAction;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ConcurrentFramePool {
+ private static final String ERROR_INVALID_FRAME_SIZE =
+ "The size should be an integral multiple of the default frame size";
+ private final String nodeId;
+ private final int budget;
+ private final int defaultFrameSize;
+ private final ArrayDeque<ByteBuffer> pool;
+ private final ArrayDeque<FrameAction> subscribers = new ArrayDeque<>();
+ private final Map<Integer, ArrayDeque<ByteBuffer>> largeFramesPools;
+ private int handedOut;
+ private int created;
+
+ public ConcurrentFramePool(String nodeId, long budgetInBytes, int frameSize) {
+ this.nodeId = nodeId;
+ this.defaultFrameSize = frameSize;
+ this.budget = (int) (budgetInBytes / frameSize);
+ this.pool = new ArrayDeque<>(budget);
+ this.largeFramesPools = new HashMap<>();
+ }
+
+ public synchronized ByteBuffer get() {
+ if (handedOut < budget) {
+ handedOut++;
+ return allocate();
+ }
+ return null;
+ }
+
+ public int remaining() {
+ return budget - handedOut;
+ }
+
+ public synchronized ByteBuffer get(int bufferSize) throws HyracksDataException {
+ if (bufferSize % defaultFrameSize != 0) {
+ throw new HyracksDataException(ERROR_INVALID_FRAME_SIZE);
+ }
+ int multiplier = bufferSize / defaultFrameSize;
+ if (handedOut + multiplier <= budget) {
+ handedOut += multiplier;
+ ArrayDeque<ByteBuffer> largeFramesPool = largeFramesPools.get(multiplier);
+ if (largeFramesPool == null || largeFramesPool.isEmpty()) {
+ if (created + multiplier > budget) {
+ freeup(multiplier);
+ }
+ created += multiplier;
+ return ByteBuffer.allocate(bufferSize);
+ }
+ return largeFramesPool.poll();
+ }
+ // Not enough budget
+ return null;
+ }
+
+ private int freeup(int desiredNumberOfFreePages) {
+ int needToFree = desiredNumberOfFreePages - (budget - created);
+ int freed = 0;
+ // start by large frames
+ for (Iterator<Entry<Integer, ArrayDeque<ByteBuffer>>> it = largeFramesPools.entrySet().iterator(); it
+ .hasNext();) {
+ Entry<Integer, ArrayDeque<ByteBuffer>> entry = it.next();
+ if (entry.getKey() != desiredNumberOfFreePages) {
+ while (!entry.getValue().isEmpty()) {
+ entry.getValue().pop();
+ freed += entry.getKey();
+ if (freed >= needToFree) {
+ // created is handled here
+ created -= freed;
+ return freed;
+ }
+ }
+ it.remove();
+ }
+ }
+ // freed all large pages. need to free small pages as well
+ needToFree -= freed;
+ while (needToFree > 0) {
+ pool.pop();
+ needToFree--;
+ freed++;
+ }
+ created -= freed;
+ return freed;
+ }
+
+ private ByteBuffer allocate() {
+ if (pool.isEmpty()) {
+ if (created == budget) {
+ freeup(1);
+ }
+ created++;
+ return ByteBuffer.allocate(defaultFrameSize);
+ } else {
+ return pool.pop();
+ }
+ }
+
+ public synchronized boolean get(Collection<ByteBuffer> buffers, int count) {
+ if (handedOut + count <= budget) {
+ handedOut += count;
+ for (int i = 0; i < count; i++) {
+ buffers.add(allocate());
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "ConcurrentFramePool [" + nodeId + "]" + "(consumed:" + handedOut + "/" + budget + ")";
+ }
+
+ public synchronized void release(Collection<ByteBuffer> buffers) throws HyracksDataException {
+ for (ByteBuffer buffer : buffers) {
+ release(buffer);
+ }
+ }
+
+ public synchronized void release(ByteBuffer buffer) throws HyracksDataException {
+ int multiples = buffer.capacity() / defaultFrameSize;
+ handedOut -= multiples;
+ if (multiples == 1) {
+ pool.add(buffer);
+ } else {
+ ArrayDeque<ByteBuffer> largeFramesPool = largeFramesPools.get(multiples);
+ if (largeFramesPool == null) {
+ largeFramesPool = new ArrayDeque<>();
+ largeFramesPools.put(multiples, largeFramesPool);
+ }
+ largeFramesPool.push(buffer);
+ }
+ // check subscribers
+ while (!subscribers.isEmpty()) {
+ FrameAction frameAction = subscribers.peek();
+ // check if we have enough and answer immediately.
+ if (frameAction.getSize() == defaultFrameSize) {
+ buffer = get();
+ } else {
+ buffer = get(frameAction.getSize());
+ }
+ if (buffer != null) {
+ try {
+ frameAction.call(buffer);
+ } finally {
+ subscribers.remove();
+ }
+ } else {
+ break;
+ }
+ }
+ }
+
+ public synchronized boolean subscribe(FrameAction frameAction) throws HyracksDataException {
+ // check if subscribers are empty?
+ if (subscribers.isEmpty()) {
+ ByteBuffer buffer;
+ // check if we have enough and answer immediately.
+ if (frameAction.getSize() == defaultFrameSize) {
+ buffer = get();
+ } else {
+ buffer = get(frameAction.getSize());
+ }
+ if (buffer != null) {
+ frameAction.call(buffer);
+ // There is no need to subscribe. perform action and return false
+ return false;
+ }
+ }
+ // none of the above, add to subscribers and return true
+ subscribers.add(frameAction);
+ return true;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
index 1af7153..13f19b8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionId.java
@@ -29,16 +29,17 @@
private static final long serialVersionUID = 1L;
private final FeedId feedId; // Dataverse - Feed
- private final String datasetName; // Dataset
+ private final String datasetName; // Dataset <Dataset is empty in case of no target dataset>
+ private final int hash;
public FeedConnectionId(FeedId feedId, String datasetName) {
this.feedId = feedId;
this.datasetName = datasetName;
+ this.hash = toString().hashCode();
}
public FeedConnectionId(String dataverse, String feedName, String datasetName) {
- this.feedId = new FeedId(dataverse, feedName);
- this.datasetName = datasetName;
+ this(new FeedId(dataverse, feedName), datasetName);
}
public FeedId getFeedId() {
@@ -64,7 +65,7 @@
@Override
public int hashCode() {
- return toString().hashCode();
+ return hash;
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java
index dd2fc60..a746ef4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedConnectionManager.java
@@ -46,6 +46,7 @@
this.nodeId = nodeId;
}
+ @Override
public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
return feedRuntimeManagers.get(feedId);
}
@@ -67,8 +68,7 @@
}
@Override
- public synchronized void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime)
- throws Exception {
+ public synchronized void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime) {
FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
if (runtimeMgr == null) {
runtimeMgr = new FeedRuntimeManager(connectionId, this);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java
index 5095e7d..62f4c6c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedManager.java
@@ -18,22 +18,14 @@
*/
package org.apache.asterix.external.feed.management;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
import org.apache.asterix.common.config.AsterixFeedProperties;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.feed.api.IFeedConnectionManager;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
-import org.apache.asterix.external.feed.api.IFeedMessageService;
-import org.apache.asterix.external.feed.api.IFeedMetadataManager;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFeedSubscriptionManager;
-import org.apache.asterix.external.feed.message.FeedMessageService;
-import org.apache.asterix.external.feed.watch.FeedMetricCollector;
-import org.apache.asterix.external.feed.watch.NodeLoadReportService;
-import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.external.feed.api.ISubscribableRuntime;
+import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
@@ -41,23 +33,13 @@
* Provider necessary central repository for registering/retrieving
* artifacts/services associated with a feed.
*/
-public class FeedManager implements IFeedManager {
+public class FeedManager {
- private static final Logger LOGGER = Logger.getLogger(FeedManager.class.getName());
-
- private final IFeedSubscriptionManager feedSubscriptionManager;
+ private final Map<FeedRuntimeId, ISubscribableRuntime> subscribableRuntimes;
private final IFeedConnectionManager feedConnectionManager;
- private final IFeedMemoryManager feedMemoryManager;
-
- private final IFeedMetricCollector feedMetricCollector;
-
- private final IFeedMetadataManager feedMetadataManager;
-
- private final IFeedMessageService feedMessageService;
-
- private final NodeLoadReportService nodeLoadReportService;
+ private final ConcurrentFramePool feedMemoryManager;
private final AsterixFeedProperties asterixFeedProperties;
@@ -68,60 +50,39 @@
public FeedManager(String nodeId, AsterixFeedProperties feedProperties, int frameSize)
throws AsterixException, HyracksDataException {
this.nodeId = nodeId;
- this.feedSubscriptionManager = new FeedSubscriptionManager(nodeId);
this.feedConnectionManager = new FeedConnectionManager(nodeId);
- this.feedMetadataManager = new FeedMetadataManager(nodeId);
- this.feedMemoryManager = new FeedMemoryManager(nodeId, feedProperties, frameSize);
- String ccClusterIp = AsterixClusterProperties.INSTANCE.getCluster() != null
- ? AsterixClusterProperties.INSTANCE.getCluster().getMasterNode().getClusterIp() : "localhost";
- this.feedMessageService = new FeedMessageService(feedProperties, nodeId, ccClusterIp);
- this.nodeLoadReportService = new NodeLoadReportService(nodeId, this);
- try {
- this.feedMessageService.start();
- this.nodeLoadReportService.start();
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start feed services " + e.getMessage());
- }
- e.printStackTrace();
- }
- this.feedMetricCollector = new FeedMetricCollector(nodeId);
+ this.feedMemoryManager =
+ new ConcurrentFramePool(nodeId, feedProperties.getMemoryComponentGlobalBudget(), frameSize);
this.frameSize = frameSize;
this.asterixFeedProperties = feedProperties;
+ this.subscribableRuntimes = new ConcurrentHashMap<FeedRuntimeId, ISubscribableRuntime>();
}
- @Override
- public IFeedSubscriptionManager getFeedSubscriptionManager() {
- return feedSubscriptionManager;
- }
-
- @Override
public IFeedConnectionManager getFeedConnectionManager() {
return feedConnectionManager;
}
- @Override
- public IFeedMemoryManager getFeedMemoryManager() {
+ public ConcurrentFramePool getFeedMemoryManager() {
return feedMemoryManager;
}
- @Override
- public IFeedMetricCollector getFeedMetricCollector() {
- return feedMetricCollector;
- }
-
public int getFrameSize() {
return frameSize;
}
- @Override
- public IFeedMetadataManager getFeedMetadataManager() {
- return feedMetadataManager;
+ public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime) {
+ FeedRuntimeId sid = subscribableRuntime.getRuntimeId();
+ if (!subscribableRuntimes.containsKey(sid)) {
+ subscribableRuntimes.put(sid, subscribableRuntime);
+ }
}
- @Override
- public IFeedMessageService getFeedMessageService() {
- return feedMessageService;
+ public void deregisterFeedSubscribableRuntime(FeedRuntimeId subscribableRuntimeId) {
+ subscribableRuntimes.remove(subscribableRuntimeId);
+ }
+
+ public ISubscribableRuntime getSubscribableRuntime(FeedRuntimeId subscribableRuntimeId) {
+ return subscribableRuntimes.get(subscribableRuntimeId);
}
@Override
@@ -129,7 +90,6 @@
return "FeedManager " + "[" + nodeId + "]";
}
- @Override
public AsterixFeedProperties getAsterixFeedProperties() {
return asterixFeedProperties;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMemoryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMemoryManager.java
deleted file mode 100644
index de9d22c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMemoryManager.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.asterix.external.feed.management;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent;
-import org.apache.asterix.external.feed.api.IFeedMemoryManager;
-import org.apache.asterix.external.feed.api.IFeedMemoryComponent.Type;
-import org.apache.asterix.external.feed.dataflow.DataBucketPool;
-import org.apache.asterix.external.feed.dataflow.FrameCollection;
-
-public class FeedMemoryManager implements IFeedMemoryManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedMemoryManager.class.getName());
- private static final int ALLOCATION_INCREMENT = 10;
-
- private final AtomicInteger componentId = new AtomicInteger(0);
- private final String nodeId;
- private final int budget;
- private final int frameSize;
-
- private int committed;
-
- public FeedMemoryManager(String nodeId, AsterixFeedProperties feedProperties, int frameSize) {
- this.nodeId = nodeId;
- this.frameSize = frameSize;
- budget = (int) feedProperties.getMemoryComponentGlobalBudget() / frameSize;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Feed Memory budget " + budget + " frames (frame size=" + frameSize + ")");
- }
- }
-
- @Override
- public synchronized IFeedMemoryComponent getMemoryComponent(Type type) {
- IFeedMemoryComponent memoryComponent = null;
- boolean valid = false;
- switch (type) {
- case COLLECTION:
- valid = committed + START_COLLECTION_SIZE <= budget;
- if (valid) {
- memoryComponent = new FrameCollection(componentId.incrementAndGet(), this, START_COLLECTION_SIZE);
- }
- break;
- case POOL:
- valid = committed + START_POOL_SIZE <= budget;
- if (valid) {
- memoryComponent = new DataBucketPool(componentId.incrementAndGet(), this, START_POOL_SIZE,
- frameSize);
- }
- committed += START_POOL_SIZE;
- break;
- }
- if (!valid) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to allocate memory component of type" + type);
- }
- }
- return valid ? memoryComponent : null;
- }
-
- @Override
- public synchronized boolean expandMemoryComponent(IFeedMemoryComponent memoryComponent) {
- if (committed + ALLOCATION_INCREMENT > budget) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Memory budget " + budget + " is exhausted. Space left: " + (budget - committed)
- + " frames.");
- }
- return false;
- } else {
- memoryComponent.expand(ALLOCATION_INCREMENT);
- committed += ALLOCATION_INCREMENT;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Expanded memory component " + memoryComponent + " by " + ALLOCATION_INCREMENT + " " + this);
- }
- return true;
- }
- }
-
- @Override
- public synchronized void releaseMemoryComponent(IFeedMemoryComponent memoryComponent) {
- int delta = memoryComponent.getTotalAllocation();
- committed -= delta;
- memoryComponent.reset();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Reset " + memoryComponent + " and reclaimed " + delta + " frames " + this);
- }
- }
-
- @Override
- public String toString() {
- return "FeedMemoryManager [" + nodeId + "]" + "(" + committed + "/" + budget + ")";
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMetadataManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMetadataManager.java
deleted file mode 100644
index 34ae461..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedMetadataManager.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.asterix.external.feed.management;
-
-import java.util.Date;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedMetadataManager;
-import org.apache.asterix.external.feed.message.XAQLFeedMessage;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.om.base.ARecord;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class FeedMetadataManager implements IFeedMetadataManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedMetadataManager.class.getName());
-
- private final String nodeId;
- private ARecordType recordType;
-
- public FeedMetadataManager(String nodeId) throws AsterixException, HyracksDataException {
- this.nodeId = nodeId;
- String[] fieldNames = new String[] { "id", "dataverseName", "feedName", "targetDataset", "tuple", "message",
- "timestamp" };
- IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
-
- recordType = new ARecordType(FeedConstants.FAILED_TUPLE_DATASET_TYPE, fieldNames, fieldTypes, true);
- }
-
- @Override
- public void logTuple(FeedConnectionId connectionId, String tuple, String message, IFeedManager feedManager)
- throws AsterixException {
- try {
- AString id = new AString("1");
- AString dataverseValue = new AString(connectionId.getFeedId().getDataverse());
- AString feedValue = new AString(connectionId.getFeedId().getFeedName());
- AString targetDatasetValue = new AString(connectionId.getDatasetName());
- AString tupleValue = new AString(tuple);
- AString messageValue = new AString(message);
- AString dateTime = new AString(new Date().toString());
-
- IAObject[] fields = new IAObject[] { id, dataverseValue, feedValue, targetDatasetValue, tupleValue,
- messageValue, dateTime };
- ARecord record = new ARecord(recordType, fields);
- StringBuilder builder = new StringBuilder();
- builder.append("use dataverse " + FeedConstants.FEEDS_METADATA_DV + ";" + "\n");
- builder.append("insert into dataset " + FeedConstants.FAILED_TUPLE_DATASET + " ");
- builder.append(" (" + recordToString(record) + ")");
- builder.append(";");
-
- XAQLFeedMessage xAqlMessage = new XAQLFeedMessage(connectionId, builder.toString());
- feedManager.getFeedMessageService().sendMessage(xAqlMessage);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Sent " + xAqlMessage.toJSON());
- }
- } catch (Exception pe) {
- throw new AsterixException(pe);
- }
- }
-
- @Override
- public String toString() {
- return "FeedMetadataManager [" + nodeId + "]";
- }
-
- private String recordToString(ARecord record) {
- String[] fieldNames = record.getType().getFieldNames();
- StringBuilder sb = new StringBuilder();
- sb.append("{ ");
- for (int i = 0; i < fieldNames.length; i++) {
- if (i > 0) {
- sb.append(", ");
- }
- sb.append("\"" + fieldNames[i] + "\"");
- sb.append(": ");
- switch (record.getType().getFieldTypes()[i].getTypeTag()) {
- case STRING:
- sb.append("\"" + ((AString) record.getValueByPos(i)).getStringValue() + "\"");
- break;
- default:
- break;
- }
- }
- sb.append(" }");
- return sb.toString();
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedSubscriptionManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedSubscriptionManager.java
deleted file mode 100644
index e402f92..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/management/FeedSubscriptionManager.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.asterix.external.feed.management;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedSubscriptionManager;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
-
-public class FeedSubscriptionManager implements IFeedSubscriptionManager {
-
- private static Logger LOGGER = Logger.getLogger(FeedSubscriptionManager.class.getName());
-
- private final String nodeId;
-
- private final Map<SubscribableFeedRuntimeId, ISubscribableRuntime> subscribableRuntimes;
-
- public FeedSubscriptionManager(String nodeId) {
- this.nodeId = nodeId;
- this.subscribableRuntimes = new HashMap<SubscribableFeedRuntimeId, ISubscribableRuntime>();
- }
-
- @Override
- public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime) {
- SubscribableFeedRuntimeId sid = (SubscribableFeedRuntimeId) subscribableRuntime.getRuntimeId();
- if (!subscribableRuntimes.containsKey(sid)) {
- subscribableRuntimes.put(sid, subscribableRuntime);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered feed subscribable runtime " + subscribableRuntime);
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Feed ingestion runtime " + subscribableRuntime + " already registered.");
- }
- }
- }
-
- @Override
- public ISubscribableRuntime getSubscribableRuntime(SubscribableFeedRuntimeId subscribableFeedRuntimeId) {
- return subscribableRuntimes.get(subscribableFeedRuntimeId);
- }
-
- @Override
- public void deregisterFeedSubscribableRuntime(SubscribableFeedRuntimeId ingestionId) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("De-registered feed subscribable runtime " + ingestionId);
- }
- subscribableRuntimes.remove(ingestionId);
- }
-
- @Override
- public String toString() {
- return "IngestionManager [" + nodeId + "]";
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedCongestionMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedCongestionMessage.java
deleted file mode 100644
index 6c924d2..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedCongestionMessage.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.external.util.FeedConstants.MessageConstants;
-
-public class FeedCongestionMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
- private final FeedRuntimeId runtimeId;
- private int inflowRate;
- private int outflowRate;
- private Mode mode;
-
- public FeedCongestionMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId, int inflowRate,
- int outflowRate, Mode mode) {
- super(MessageType.CONGESTION);
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- this.inflowRate = inflowRate;
- this.outflowRate = outflowRate;
- this.mode = mode;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
- obj.put(FeedConstants.MessageConstants.OPERAND_ID, runtimeId.getOperandId());
- obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
- obj.put(FeedConstants.MessageConstants.INFLOW_RATE, inflowRate);
- obj.put(FeedConstants.MessageConstants.OUTFLOW_RATE, outflowRate);
- obj.put(FeedConstants.MessageConstants.MODE, mode);
- return obj;
- }
-
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- public int getInflowRate() {
- return inflowRate;
- }
-
- public int getOutflowRate() {
- return outflowRate;
- }
-
- public static FeedCongestionMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
- .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
- obj.getInt(FeedConstants.MessageConstants.PARTITION),
- obj.getString(FeedConstants.MessageConstants.OPERAND_ID));
- Mode mode = Mode.valueOf(obj.getString(MessageConstants.MODE));
- return new FeedCongestionMessage(connectionId, runtimeId,
- obj.getInt(FeedConstants.MessageConstants.INFLOW_RATE),
- obj.getInt(FeedConstants.MessageConstants.OUTFLOW_RATE), mode);
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public Mode getMode() {
- return mode;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessageService.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessageService.java
deleted file mode 100644
index 13d6622..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedMessageService.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import java.net.Socket;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-
-import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.external.feed.api.IFeedMessage;
-import org.apache.asterix.external.feed.api.IFeedMessageService;
-import org.apache.asterix.external.util.FeedConstants;
-
-/**
- * Sends feed report messages on behalf of an operator instance
- * to the SuperFeedManager associated with the feed.
- */
-public class FeedMessageService implements IFeedMessageService {
-
- private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
-
- private final LinkedBlockingQueue<String> inbox;
- private final FeedMessageHandler mesgHandler;
- private final String nodeId;
- private ExecutorService executor;
-
- public FeedMessageService(AsterixFeedProperties feedProperties, String nodeId, String ccClusterIp) {
- this.inbox = new LinkedBlockingQueue<String>();
- this.mesgHandler = new FeedMessageHandler(inbox, ccClusterIp, feedProperties.getFeedCentralManagerPort());
- this.nodeId = nodeId;
- this.executor = Executors.newSingleThreadExecutor();
- }
-
- public void start() throws Exception {
-
- executor.execute(mesgHandler);
- }
-
- public void stop() {
- synchronized (mesgHandler.getLock()) {
- executor.shutdownNow();
- }
- mesgHandler.stop();
- }
-
- @Override
- public void sendMessage(IFeedMessage message) {
- try {
- JSONObject obj = message.toJSON();
- obj.put(FeedConstants.MessageConstants.NODE_ID, nodeId);
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, message.getMessageType().name());
- inbox.add(obj.toString());
- } catch (JSONException jse) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("JSON exception in parsing message " + message + " exception [" + jse.getMessage() + "]");
- }
- }
- }
-
- private static class FeedMessageHandler implements Runnable {
-
- private final LinkedBlockingQueue<String> inbox;
- private final String host;
- private final int port;
- private final Object lock;
-
- private Socket cfmSocket;
-
- private static final byte[] EOL = "\n".getBytes();
-
- public FeedMessageHandler(LinkedBlockingQueue<String> inbox, String host, int port) {
- this.inbox = inbox;
- this.host = host;
- this.port = port;
- this.lock = new Object();
- }
-
- public void run() {
- try {
- cfmSocket = new Socket(host, port);
- if (cfmSocket != null) {
- while (true) {
- String message = inbox.take();
- synchronized (lock) { // lock prevents message handler from sending incomplete message midst shutdown attempt
- cfmSocket.getOutputStream().write(message.getBytes());
- cfmSocket.getOutputStream().write(EOL);
- }
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start feed message service");
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception in handling incoming feed messages" + e.getMessage());
- }
- } finally {
- stop();
- }
-
- }
-
- public void stop() {
- if (cfmSocket != null) {
- try {
- cfmSocket.close();
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception in closing socket " + e.getMessage());
- }
- }
- }
- }
-
- public Object getLock() {
- return lock;
- }
-
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedReportMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedReportMessage.java
deleted file mode 100644
index 1b8c45d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedReportMessage.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.ValueType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.external.util.FeedConstants.MessageConstants;
-
-public class FeedReportMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
- private final FeedRuntimeId runtimeId;
- private final ValueType valueType;
- private int value;
-
- public FeedReportMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType, int value) {
- super(MessageType.FEED_REPORT);
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- this.valueType = valueType;
- this.value = value;
- }
-
- public void reset(int value) {
- this.value = value;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
- obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
- obj.put(FeedConstants.MessageConstants.VALUE_TYPE, valueType);
- obj.put(FeedConstants.MessageConstants.VALUE, value);
- return obj;
- }
-
- public static FeedReportMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
- .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
- obj.getInt(FeedConstants.MessageConstants.PARTITION), FeedConstants.MessageConstants.NOT_APPLICABLE);
- ValueType type = ValueType.valueOf(obj.getString(MessageConstants.VALUE_TYPE));
- int value = Integer.parseInt(obj.getString(MessageConstants.VALUE));
- return new FeedReportMessage(connectionId, runtimeId, type, value);
- }
-
- public int getValue() {
- return value;
- }
-
- public void setValue(int value) {
- this.value = value;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- public ValueType getValueType() {
- return valueType;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitAckMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitAckMessage.java
deleted file mode 100644
index 61e26de..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitAckMessage.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import javax.xml.bind.DatatypeConverter;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.util.FeedConstants;
-
-public class FeedTupleCommitAckMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
- private int intakePartition;
- private int base;
- private byte[] commitAcks;
-
- public FeedTupleCommitAckMessage(FeedConnectionId connectionId, int intakePartition, int base, byte[] commitAcks) {
- super(MessageType.COMMIT_ACK);
- this.connectionId = connectionId;
- this.intakePartition = intakePartition;
- this.base = base;
- this.commitAcks = commitAcks;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
- obj.put(FeedConstants.MessageConstants.BASE, base);
- String commitAcksString = DatatypeConverter.printBase64Binary(commitAcks);
- obj.put(FeedConstants.MessageConstants.COMMIT_ACKS, commitAcksString);
- return obj;
- }
-
- public static FeedTupleCommitAckMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
- int base = obj.getInt(FeedConstants.MessageConstants.BASE);
- String commitAcksString = obj.getString(FeedConstants.MessageConstants.COMMIT_ACKS);
- byte[] commitAcks = DatatypeConverter.parseBase64Binary(commitAcksString);
- return new FeedTupleCommitAckMessage(connectionId, intakePartition, base, commitAcks);
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public int getIntakePartition() {
- return intakePartition;
- }
-
- public byte[] getCommitAcks() {
- return commitAcks;
- }
-
- public void reset(int intakePartition, int base, byte[] commitAcks) {
- this.intakePartition = intakePartition;
- this.base = base;
- this.commitAcks = commitAcks;
- }
-
- public int getBase() {
- return base;
- }
-
- public void setBase(int base) {
- this.base = base;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitResponseMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitResponseMessage.java
deleted file mode 100644
index a61dc06..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/FeedTupleCommitResponseMessage.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.util.FeedConstants;
-
-public class FeedTupleCommitResponseMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
- private final int intakePartition;
- private final int maxWindowAcked;
-
- public FeedTupleCommitResponseMessage(FeedConnectionId connectionId, int intakePartition, int maxWindowAcked) {
- super(MessageType.COMMIT_ACK_RESPONSE);
- this.connectionId = connectionId;
- this.intakePartition = intakePartition;
- this.maxWindowAcked = maxWindowAcked;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
- obj.put(FeedConstants.MessageConstants.MAX_WINDOW_ACKED, maxWindowAcked);
- return obj;
- }
-
- @Override
- public String toString() {
- return connectionId + "[" + intakePartition + "]" + "(" + maxWindowAcked + ")";
- }
-
- public static FeedTupleCommitResponseMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
- int maxWindowAcked = obj.getInt(FeedConstants.MessageConstants.MAX_WINDOW_ACKED);
- return new FeedTupleCommitResponseMessage(connectionId, intakePartition, maxWindowAcked);
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public int getMaxWindowAcked() {
- return maxWindowAcked;
- }
-
- public int getIntakePartition() {
- return intakePartition;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageListener.java
deleted file mode 100644
index 67f2884..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageListener.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.nio.CharBuffer;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class MessageListener {
-
- private static final Logger LOGGER = Logger.getLogger(MessageListener.class.getName());
-
- private int port;
- private final LinkedBlockingQueue<String> outbox;
-
- private ExecutorService executorService = Executors.newFixedThreadPool(10);
-
- private MessageListenerServer listenerServer;
-
- public MessageListener(int port, LinkedBlockingQueue<String> outbox) {
- this.port = port;
- this.outbox = outbox;
- }
-
- public void stop() {
- listenerServer.stop();
- if (!executorService.isShutdown()) {
- executorService.shutdownNow();
- }
- }
-
- public void start() throws IOException {
- listenerServer = new MessageListenerServer(port, outbox);
- executorService.execute(listenerServer);
- }
-
- private static class MessageListenerServer implements Runnable {
-
- private final int port;
- private final LinkedBlockingQueue<String> outbox;
- private ServerSocket server;
-
- private static final char EOL = (char) "\n".getBytes()[0];
-
- public MessageListenerServer(int port, LinkedBlockingQueue<String> outbox) {
- this.port = port;
- this.outbox = outbox;
- }
-
- public void stop() {
- try {
- server.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public void run() {
- Socket client = null;
- try {
- server = new ServerSocket(port);
- client = server.accept();
- InputStream in = client.getInputStream();
- CharBuffer buffer = CharBuffer.allocate(5000);
- char ch;
- while (true) {
- ch = (char) in.read();
- if (((int) ch) == -1) {
- break;
- }
- while (ch != EOL) {
- buffer.put(ch);
- ch = (char) in.read();
- }
- buffer.flip();
- String s = new String(buffer.array());
- synchronized (outbox) {
- outbox.add(s + "\n");
- }
- buffer.position(0);
- buffer.limit(5000);
- }
-
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start Message listener" + server);
- }
- } finally {
- if (server != null) {
- try {
- server.close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-
- }
-
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
deleted file mode 100644
index 6ed176a..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/MessageReceiver.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IMessageReceiver;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public abstract class MessageReceiver<T> implements IMessageReceiver<T> {
-
- protected static final Logger LOGGER = Logger.getLogger(MessageReceiver.class.getName());
-
- protected final ArrayBlockingQueue<T> inbox;
- protected ExecutorService executor;
-
- public MessageReceiver() {
- inbox = new ArrayBlockingQueue<T>(2);
- }
-
- public abstract void processMessage(T message) throws Exception;
-
- @Override
- public void start() {
- executor = Executors.newSingleThreadExecutor();
- executor.execute(new MessageReceiverRunnable<T>(this));
- }
-
- @Override
- public synchronized void sendMessage(T message) throws InterruptedException {
- inbox.put(message);
- }
-
- @Override
- public void close(boolean processPending) {
- if (executor != null) {
- executor.shutdown();
- executor = null;
- if (processPending) {
- flushPendingMessages();
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Will discard the pending frames " + inbox.size());
- }
- }
- }
- }
-
- private static class MessageReceiverRunnable<T> implements Runnable {
-
- private final ArrayBlockingQueue<T> inbox;
- private final MessageReceiver<T> messageReceiver;
-
- public MessageReceiverRunnable(MessageReceiver<T> messageReceiver) {
- this.inbox = messageReceiver.inbox;
- this.messageReceiver = messageReceiver;
- }
- // TODO: this should handle exceptions better
-
- @Override
- public void run() {
- while (true) {
- try {
- T message = inbox.poll();
- if (message == null) {
- messageReceiver.emptyInbox();
- message = inbox.take();
- }
- messageReceiver.processMessage(message);
- } catch (InterruptedException e) {
- e.printStackTrace();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
-
- protected void flushPendingMessages() {
- while (!inbox.isEmpty()) {
- T message = null;
- try {
- message = inbox.take();
- processMessage(message);
- } catch (InterruptedException ie) {
- // ignore exception but break from the loop
- break;
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception " + e + " in processing message " + message);
- }
- }
- }
- }
-
- public abstract void emptyInbox() throws HyracksDataException;
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/NodeReportMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/NodeReportMessage.java
deleted file mode 100644
index 1548d6d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/NodeReportMessage.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedMessage;
-import org.apache.asterix.external.util.FeedConstants;
-
-public class NodeReportMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private double cpuLoad;
- private double usedHeap;
- private int nRuntimes;
-
- public NodeReportMessage(float cpuLoad, long usedHeap, int nRuntimes) {
- super(IFeedMessage.MessageType.NODE_REPORT);
- this.usedHeap = usedHeap;
- this.cpuLoad = cpuLoad;
- this.nRuntimes = nRuntimes;
- }
-
- public void reset(double cpuLoad, double usedHeap, int nRuntimes) {
- this.cpuLoad = cpuLoad;
- this.usedHeap = usedHeap;
- this.nRuntimes = nRuntimes;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.CPU_LOAD, cpuLoad);
- obj.put(FeedConstants.MessageConstants.HEAP_USAGE, usedHeap);
- obj.put(FeedConstants.MessageConstants.N_RUNTIMES, nRuntimes);
- return obj;
- }
-
- public double getCpuLoad() {
- return cpuLoad;
- }
-
- public double getUsedHeap() {
- return usedHeap;
- }
-
- public int getnRuntimes() {
- return nRuntimes;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/PrepareStallMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/PrepareStallMessage.java
deleted file mode 100644
index 76fe0c2..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/PrepareStallMessage.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.util.FeedConstants;
-
-/**
- * A feed control message indicating the need to end the feed. This message is dispatched
- * to all locations that host an operator involved in the feed pipeline.
- */
-public class PrepareStallMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
-
- private final int computePartitionsRetainLimit;
-
- public PrepareStallMessage(FeedConnectionId connectionId, int computePartitionsRetainLimit) {
- super(MessageType.PREPARE_STALL);
- this.connectionId = connectionId;
- this.computePartitionsRetainLimit = computePartitionsRetainLimit;
- }
-
- @Override
- public String toString() {
- return MessageType.PREPARE_STALL.name() + " " + connectionId;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.COMPUTE_PARTITION_RETAIN_LIMIT, computePartitionsRetainLimit);
- return obj;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public int getComputePartitionsRetainLimit() {
- return computePartitionsRetainLimit;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/RemoteSocketMessageListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/RemoteSocketMessageListener.java
deleted file mode 100644
index 0749f82..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/RemoteSocketMessageListener.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Socket;
-import java.nio.CharBuffer;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-public class RemoteSocketMessageListener {
-
- private static final Logger LOGGER = Logger.getLogger(RemoteSocketMessageListener.class.getName());
-
- private final String host;
- private final int port;
- private final LinkedBlockingQueue<String> outbox;
- private final ExecutorService executorService = Executors.newFixedThreadPool(10);
-
- private RemoteMessageListenerServer listenerServer;
-
- public RemoteSocketMessageListener(String host, int port, LinkedBlockingQueue<String> outbox) {
- this.host = host;
- this.port = port;
- this.outbox = outbox;
- }
-
- public void stop() {
- if (!executorService.isShutdown()) {
- executorService.shutdownNow();
- }
- listenerServer.stop();
-
- }
-
- public void start() throws IOException {
- listenerServer = new RemoteMessageListenerServer(host, port, outbox);
- executorService.execute(listenerServer);
- }
-
- private static class RemoteMessageListenerServer implements Runnable {
-
- private final String host;
- private final int port;
- private final LinkedBlockingQueue<String> outbox;
- private Socket client;
-
- public RemoteMessageListenerServer(String host, int port, LinkedBlockingQueue<String> outbox) {
- this.host = host;
- this.port = port;
- this.outbox = outbox;
- }
-
- public void stop() {
- try {
- client.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public void run() {
- char EOL = (char) "\n".getBytes()[0];
- Socket client = null;
- try {
- client = new Socket(host, port);
- InputStream in = client.getInputStream();
- CharBuffer buffer = CharBuffer.allocate(5000);
- char ch;
- while (true) {
- ch = (char) in.read();
- if ((ch) == -1) {
- break;
- }
- while (ch != EOL) {
- buffer.put(ch);
- ch = (char) in.read();
- }
- buffer.flip();
- String s = new String(buffer.array());
- synchronized (outbox) {
- outbox.add(s + "\n");
- }
- buffer.position(0);
- buffer.limit(5000);
- }
-
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start Remote Message listener" + client);
- }
- } finally {
- if (client != null && !client.isClosed()) {
- try {
- client.close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
- }
-
- public static interface IMessageAnalyzer {
-
- /**
- * @return
- */
- public LinkedBlockingQueue<String> getMessageQueue();
-
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ScaleInReportMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ScaleInReportMessage.java
deleted file mode 100644
index 62aba04..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ScaleInReportMessage.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.util.FeedConstants;
-
-/**
- * A feed control message indicating the need to scale in a stage of the feed ingestion pipeline.
- * Currently, scaling-in of the compute stage is supported.
- **/
-public class ScaleInReportMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
-
- private final FeedRuntimeType runtimeType;
-
- private int currentCardinality;
-
- private int reducedCardinaliy;
-
- public ScaleInReportMessage(FeedConnectionId connectionId, FeedRuntimeType runtimeType, int currentCardinality,
- int reducedCardinaliy) {
- super(MessageType.SCALE_IN_REQUEST);
- this.connectionId = connectionId;
- this.runtimeType = runtimeType;
- this.currentCardinality = currentCardinality;
- this.reducedCardinaliy = reducedCardinaliy;
- }
-
- @Override
- public String toString() {
- return MessageType.SCALE_IN_REQUEST.name() + " " + connectionId + " [" + runtimeType + "] "
- + " currentCardinality " + currentCardinality + " reducedCardinality " + reducedCardinaliy;
- }
-
- public FeedRuntimeType getRuntimeType() {
- return runtimeType;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeType);
- obj.put(FeedConstants.MessageConstants.CURRENT_CARDINALITY, currentCardinality);
- obj.put(FeedConstants.MessageConstants.REDUCED_CARDINALITY, reducedCardinaliy);
- return obj;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public static ScaleInReportMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- FeedRuntimeType runtimeType = FeedRuntimeType.valueOf(obj
- .getString(FeedConstants.MessageConstants.RUNTIME_TYPE));
- return new ScaleInReportMessage(connectionId, runtimeType,
- obj.getInt(FeedConstants.MessageConstants.CURRENT_CARDINALITY),
- obj.getInt(FeedConstants.MessageConstants.REDUCED_CARDINALITY));
- }
-
- public void reset(int currentCardinality, int reducedCardinaliy) {
- this.currentCardinality = currentCardinality;
- this.reducedCardinaliy = reducedCardinaliy;
- }
-
- public int getCurrentCardinality() {
- return currentCardinality;
- }
-
- public void setCurrentCardinality(int currentCardinality) {
- this.currentCardinality = currentCardinality;
- }
-
- public int getReducedCardinaliy() {
- return reducedCardinaliy;
- }
-
- public void setReducedCardinaliy(int reducedCardinaliy) {
- this.reducedCardinaliy = reducedCardinaliy;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/SocketMessageListener.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/SocketMessageListener.java
deleted file mode 100644
index 3a6709e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/SocketMessageListener.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.nio.CharBuffer;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IMessageReceiver;
-
-/**
- * Listens for messages at a configured port and redirects them to a
- * an instance of {@code IMessageReceiver}.
- * Messages may arrive in parallel from multiple senders. Each sender is handled by
- * a respective instance of {@code ClientHandler}.
- */
-public class SocketMessageListener {
-
- private static final Logger LOGGER = Logger.getLogger(SocketMessageListener.class.getName());
-
- private final IMessageReceiver<String> messageReceiver;
- private final MessageListenerServer listenerServer;
-
- private ExecutorService executorService = Executors.newFixedThreadPool(10);
-
- public SocketMessageListener(int port, IMessageReceiver<String> messageReceiver) {
- this.messageReceiver = messageReceiver;
- this.listenerServer = new MessageListenerServer(port, messageReceiver);
- }
-
- public void stop() throws IOException {
- listenerServer.stop();
- messageReceiver.close(false);
- if (!executorService.isShutdown()) {
- executorService.shutdownNow();
- }
- }
-
- public void start() {
- messageReceiver.start();
- executorService.execute(listenerServer);
- }
-
- private static class MessageListenerServer implements Runnable {
-
- private final int port;
- private final IMessageReceiver<String> messageReceiver;
- private ServerSocket server;
- private final Executor executor;
-
- public MessageListenerServer(int port, IMessageReceiver<String> messageReceiver) {
- this.port = port;
- this.messageReceiver = messageReceiver;
- this.executor = Executors.newCachedThreadPool();
- }
-
- public void stop() throws IOException {
- server.close();
- }
-
- @Override
- public void run() {
- Socket client = null;
- try {
- server = new ServerSocket(port);
- while (true) {
- client = server.accept();
- ClientHandler handler = new ClientHandler(client, messageReceiver);
- executor.execute(handler);
- }
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start Message listener" + server);
- }
- } finally {
- if (server != null) {
- try {
- server.close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
-
- private static class ClientHandler implements Runnable {
-
- private static final char EOL = (char) "\n".getBytes()[0];
-
- private final Socket client;
- private final IMessageReceiver<String> messageReceiver;
-
- public ClientHandler(Socket client, IMessageReceiver<String> messageReceiver) {
- this.client = client;
- this.messageReceiver = messageReceiver;
- }
-
- @Override
- public void run() {
- try {
- InputStream in = client.getInputStream();
- CharBuffer buffer = CharBuffer.allocate(5000);
- char ch;
- while (true) {
- ch = (char) in.read();
- if ((ch) == -1) {
- break;
- }
- while (ch != EOL) {
- buffer.put(ch);
- ch = (char) in.read();
- }
- buffer.flip();
- String s = new String(buffer.array(), 0, buffer.limit());
- messageReceiver.sendMessage(s + "\n");
- buffer.position(0);
- buffer.limit(5000);
- }
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to process mesages from client" + client);
- }
- } finally {
- if (client != null) {
- try {
- client.close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
- }
-
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/StorageReportFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/StorageReportFeedMessage.java
deleted file mode 100644
index 68ce74d..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/StorageReportFeedMessage.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.util.FeedConstants;
-import org.apache.asterix.external.util.FeedConstants.MessageConstants;
-
-/**
- * A feed control message sent from a storage runtime of a feed pipeline to report the intake timestamp corresponding
- * to the last persisted tuple.
- */
-public class StorageReportFeedMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
- private final int partition;
- private long lastPersistedTupleIntakeTimestamp;
- private boolean persistenceDelayWithinLimit;
- private long averageDelay;
- private int intakePartition;
-
- public StorageReportFeedMessage(FeedConnectionId connectionId, int partition,
- long lastPersistedTupleIntakeTimestamp, boolean persistenceDelayWithinLimit, long averageDelay,
- int intakePartition) {
- super(MessageType.STORAGE_REPORT);
- this.connectionId = connectionId;
- this.partition = partition;
- this.lastPersistedTupleIntakeTimestamp = lastPersistedTupleIntakeTimestamp;
- this.persistenceDelayWithinLimit = persistenceDelayWithinLimit;
- this.averageDelay = averageDelay;
- this.intakePartition = intakePartition;
- }
-
- @Override
- public String toString() {
- return messageType.name() + " " + connectionId + " [" + lastPersistedTupleIntakeTimestamp + "] ";
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public long getLastPersistedTupleIntakeTimestamp() {
- return lastPersistedTupleIntakeTimestamp;
- }
-
- public int getPartition() {
- return partition;
- }
-
- public boolean isPersistenceDelayWithinLimit() {
- return persistenceDelayWithinLimit;
- }
-
- public void setPersistenceDelayWithinLimit(boolean persistenceDelayWithinLimit) {
- this.persistenceDelayWithinLimit = persistenceDelayWithinLimit;
- }
-
- public long getAverageDelay() {
- return averageDelay;
- }
-
- public void setAverageDelay(long averageDelay) {
- this.averageDelay = averageDelay;
- }
-
- public int getIntakePartition() {
- return intakePartition;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.LAST_PERSISTED_TUPLE_INTAKE_TIMESTAMP, lastPersistedTupleIntakeTimestamp);
- obj.put(MessageConstants.PERSISTENCE_DELAY_WITHIN_LIMIT, persistenceDelayWithinLimit);
- obj.put(MessageConstants.AVERAGE_PERSISTENCE_DELAY, averageDelay);
- obj.put(FeedConstants.MessageConstants.PARTITION, partition);
- obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
-
- return obj;
- }
-
- public static StorageReportFeedMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- int partition = obj.getInt(FeedConstants.MessageConstants.PARTITION);
- long timestamp = obj.getLong(FeedConstants.MessageConstants.LAST_PERSISTED_TUPLE_INTAKE_TIMESTAMP);
- boolean persistenceDelayWithinLimit = obj.getBoolean(MessageConstants.PERSISTENCE_DELAY_WITHIN_LIMIT);
- long averageDelay = obj.getLong(MessageConstants.AVERAGE_PERSISTENCE_DELAY);
- int intakePartition = obj.getInt(MessageConstants.INTAKE_PARTITION);
- return new StorageReportFeedMessage(connectionId, partition, timestamp, persistenceDelayWithinLimit,
- averageDelay, intakePartition);
- }
-
- public void reset(long lastPersistedTupleIntakeTimestamp, boolean delayWithinLimit, long averageDelay) {
- this.lastPersistedTupleIntakeTimestamp = lastPersistedTupleIntakeTimestamp;
- this.persistenceDelayWithinLimit = delayWithinLimit;
- this.averageDelay = averageDelay;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/TerminateDataFlowMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/TerminateDataFlowMessage.java
deleted file mode 100644
index ab77840..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/TerminateDataFlowMessage.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.util.FeedConstants;
-
-public class TerminateDataFlowMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
-
- public TerminateDataFlowMessage(FeedConnectionId connectionId) {
- super(MessageType.TERMINATE_FLOW);
- this.connectionId = connectionId;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- return obj;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ThrottlingEnabledFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ThrottlingEnabledFeedMessage.java
deleted file mode 100644
index 0459310..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/ThrottlingEnabledFeedMessage.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.util.FeedConstants;
-
-/**
- * A feed control message indicating the need to end the feed. This message is dispatched
- * to all locations that host an operator involved in the feed pipeline.
- */
-public class ThrottlingEnabledFeedMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final FeedConnectionId connectionId;
-
- private final FeedRuntimeId runtimeId;
-
- public ThrottlingEnabledFeedMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId) {
- super(MessageType.THROTTLING_ENABLED);
- this.connectionId = connectionId;
- this.runtimeId = runtimeId;
- }
-
- @Override
- public String toString() {
- return MessageType.END.name() + " " + connectionId + " [" + runtimeId + "] ";
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
- obj.put(FeedConstants.MessageConstants.OPERAND_ID, runtimeId.getOperandId());
- obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
- return obj;
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public FeedRuntimeId getFeedRuntimeId() {
- return runtimeId;
- }
-
- public static ThrottlingEnabledFeedMessage read(JSONObject obj) throws JSONException {
- FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
- obj.getString(FeedConstants.MessageConstants.FEED));
- FeedConnectionId connectionId = new FeedConnectionId(feedId,
- obj.getString(FeedConstants.MessageConstants.DATASET));
- FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
- .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
- obj.getInt(FeedConstants.MessageConstants.PARTITION),
- obj.getString(FeedConstants.MessageConstants.OPERAND_ID));
- return new ThrottlingEnabledFeedMessage(connectionId, runtimeId);
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/XAQLFeedMessage.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/XAQLFeedMessage.java
deleted file mode 100644
index cef3fa9..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/message/XAQLFeedMessage.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.asterix.external.feed.message;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.util.FeedConstants;
-
-/**
- * A feed control message indicating the need to execute a give AQL.
- */
-public class XAQLFeedMessage extends FeedMessage {
-
- private static final long serialVersionUID = 1L;
-
- private final String aql;
- private final FeedConnectionId connectionId;
-
- public XAQLFeedMessage(FeedConnectionId connectionId, String aql) {
- super(MessageType.XAQL);
- this.connectionId = connectionId;
- this.aql = aql;
- }
-
- @Override
- public String toString() {
- return messageType.name() + " " + connectionId + " [" + aql + "] ";
- }
-
- public FeedConnectionId getConnectionId() {
- return connectionId;
- }
-
- public String getAql() {
- return aql;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject obj = new JSONObject();
- obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
- obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
- obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
- obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
- obj.put(FeedConstants.MessageConstants.AQL, aql);
- return obj;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
index 43d5bce..9c7a319 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterExecutor.java
@@ -18,8 +18,6 @@
*/
package org.apache.asterix.external.feed.runtime;
-import org.apache.asterix.external.api.IAdapterRuntimeManager;
-import org.apache.asterix.external.api.IAdapterRuntimeManager.State;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.util.ExternalDataExceptionUtils;
@@ -33,12 +31,12 @@
private static final Logger LOGGER = Logger.getLogger(AdapterExecutor.class.getName());
private final DistributeFeedFrameWriter writer; // A writer that sends frames to multiple receivers (that can
- // increase or decrease at any time)
+ // increase or decrease at any time)
private final FeedAdapter adapter; // The adapter
- private final IAdapterRuntimeManager adapterManager;// The runtime manager <-- two way visibility -->
+ private final AdapterRuntimeManager adapterManager;// The runtime manager <-- two way visibility -->
public AdapterExecutor(int partition, DistributeFeedFrameWriter writer, FeedAdapter adapter,
- IAdapterRuntimeManager adapterManager) {
+ AdapterRuntimeManager adapterManager) {
this.writer = writer;
this.adapter = adapter;
this.adapterManager = adapterManager;
@@ -72,7 +70,8 @@
}
// Done with the adapter. about to close, setting the stage based on the failed ingestion flag and notifying the
// runtime manager
- adapterManager.setState(failedIngestion ? State.FAILED_INGESTION : State.FINISHED_INGESTION);
+ adapterManager.setFailed(failedIngestion);
+ adapterManager.setDone(true);
synchronized (adapterManager) {
adapterManager.notifyAll();
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
index b0f2517..b604db5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/AdapterRuntimeManager.java
@@ -22,9 +22,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
-import org.apache.asterix.external.api.IAdapterRuntimeManager;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.log4j.Logger;
@@ -32,7 +30,7 @@
/**
* This class manages the execution of an adapter within a feed
*/
-public class AdapterRuntimeManager implements IAdapterRuntimeManager {
+public class AdapterRuntimeManager {
private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
@@ -40,46 +38,37 @@
private final FeedAdapter feedAdapter; // The adapter
- private final IIntakeProgressTracker tracker; // Not used. needs to be fixed soon.
-
- private final AdapterExecutor adapterExecutor; // The executor for the adapter <-- two way visibility -->
+ private final AdapterExecutor adapterExecutor; // The executor for the adapter
private final int partition; // The partition number
private final ExecutorService executorService; // Executor service to run/shutdown the adapter executor
- private IngestionRuntime ingestionRuntime; // Runtime representing the ingestion stage of a feed <-- two way
- // visibility -->
+ private IngestionRuntime ingestionRuntime; // Runtime representing the ingestion stage of a feed
- private State state; // One of {ACTIVE_INGESTION, NACTIVE_INGESTION, FINISHED_INGESTION,
- // FAILED_INGESTION}
+ private volatile boolean done = false;
+ private volatile boolean failed = false;
- public AdapterRuntimeManager(FeedId feedId, FeedAdapter feedAdapter, IIntakeProgressTracker tracker,
- DistributeFeedFrameWriter writer, int partition) {
+ public AdapterRuntimeManager(FeedId feedId, FeedAdapter feedAdapter, DistributeFeedFrameWriter writer,
+ int partition) {
this.feedId = feedId;
this.feedAdapter = feedAdapter;
- this.tracker = tracker;
this.partition = partition;
this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
this.executorService = Executors.newSingleThreadExecutor();
- this.state = State.INACTIVE_INGESTION;
}
- @Override
- public void start() throws Exception {
- state = State.ACTIVE_INGESTION;
+ public void start() {
executorService.execute(adapterExecutor);
}
- @Override
- public void stop() {
+ public void stop() throws InterruptedException {
boolean stopped = false;
try {
stopped = feedAdapter.stop();
} catch (Exception exception) {
LOGGER.error("Unable to stop adapter " + feedAdapter, exception);
} finally {
- state = State.FINISHED_INGESTION;
if (stopped) {
// stop() returned true, we wait for the process termination
executorService.shutdown();
@@ -87,6 +76,7 @@
executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
} catch (InterruptedException e) {
LOGGER.error("Interrupted while waiting for feed adapter to finish its work", e);
+ throw e;
}
} else {
// stop() returned false, we try to force shutdown
@@ -96,7 +86,6 @@
}
}
- @Override
public FeedId getFeedId() {
return feedId;
}
@@ -106,30 +95,14 @@
return feedId + "[" + partition + "]";
}
- @Override
public FeedAdapter getFeedAdapter() {
return feedAdapter;
}
- public IIntakeProgressTracker getTracker() {
- return tracker;
- }
-
- @Override
- public synchronized State getState() {
- return state;
- }
-
- @Override
- public synchronized void setState(State state) {
- this.state = state;
- }
-
public AdapterExecutor getAdapterExecutor() {
return adapterExecutor;
}
- @Override
public int getPartition() {
return partition;
}
@@ -138,9 +111,19 @@
return ingestionRuntime;
}
- @Override
- public IIntakeProgressTracker getProgressTracker() {
- return tracker;
+ public boolean isFailed() {
+ return failed;
}
+ public void setFailed(boolean failed) {
+ this.failed = failed;
+ }
+
+ public boolean isDone() {
+ return done;
+ }
+
+ public void setDone(boolean done) {
+ this.done = done;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
index 8249fa6..6b1c5b8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/CollectionRuntime.java
@@ -23,10 +23,7 @@
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector.State;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
/**
@@ -40,20 +37,20 @@
private final FeedConnectionId connectionId; // [Dataverse - Feed - Dataset]
private final ISubscribableRuntime sourceRuntime; // Runtime that provides the data
private final Map<String, String> feedPolicy; // Policy associated with the feed
- private FeedFrameCollector frameCollector; // Collector that can be plugged into a frame distributor
+ private final FeedFrameCollector frameCollector; // Collector that can be plugged into a frame distributor
private final IHyracksTaskContext ctx;
- public CollectionRuntime(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- FeedRuntimeInputHandler inputSideHandler, IFrameWriter outputSideWriter, ISubscribableRuntime sourceRuntime,
- Map<String, String> feedPolicy, IHyracksTaskContext ctx) {
- super(runtimeId, inputSideHandler, outputSideWriter);
+ public CollectionRuntime(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ISubscribableRuntime sourceRuntime,
+ Map<String, String> feedPolicy, IHyracksTaskContext ctx, FeedFrameCollector frameCollector) {
+ super(runtimeId);
this.connectionId = connectionId;
this.sourceRuntime = sourceRuntime;
this.feedPolicy = feedPolicy;
this.ctx = ctx;
+ this.frameCollector = frameCollector;
}
- public State waitTillCollectionOver() throws InterruptedException {
+ public void waitTillCollectionOver() throws InterruptedException {
if (!(isCollectionOver())) {
synchronized (frameCollector) {
while (!isCollectionOver()) {
@@ -61,7 +58,6 @@
}
}
}
- return frameCollector.getState();
}
private boolean isCollectionOver() {
@@ -70,11 +66,6 @@
}
@Override
- public void setMode(Mode mode) {
- getInputHandler().setMode(mode);
- }
-
- @Override
public Map<String, String> getFeedPolicy() {
return feedPolicy;
}
@@ -87,11 +78,6 @@
return sourceRuntime;
}
- public void setFrameCollector(FeedFrameCollector frameCollector) {
- this.frameCollector = frameCollector;
- }
-
- @Override
public FeedFrameCollector getFrameCollector() {
return frameCollector;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
index 76b1b19..8c86d86 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntime.java
@@ -18,30 +18,15 @@
*/
package org.apache.asterix.external.feed.runtime;
-import org.apache.asterix.external.feed.api.IFeedOperatorOutputSideHandler;
import org.apache.asterix.external.feed.api.IFeedRuntime;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.hyracks.api.comm.IFrameWriter;
public class FeedRuntime implements IFeedRuntime {
/** A unique identifier for the runtime **/
protected final FeedRuntimeId runtimeId;
- /** The output frame writer associated with the runtime **/
- protected IFrameWriter frameWriter;
-
- /** The pre-processor associated with the runtime **/
- protected FeedRuntimeInputHandler inputHandler;
-
- public FeedRuntime(FeedRuntimeId runtimeId, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter) {
- this.runtimeId = runtimeId;
- this.frameWriter = frameWriter;
- this.inputHandler = inputHandler;
- }
-
- public void setFrameWriter(IFeedOperatorOutputSideHandler frameWriter) {
- this.frameWriter = frameWriter;
+ public FeedRuntime(FeedRuntimeId runtimeId) {
+ this.runtimeId = runtimeId;;
}
@Override
@@ -50,26 +35,7 @@
}
@Override
- public IFrameWriter getFeedFrameWriter() {
- return frameWriter;
- }
-
- @Override
public String toString() {
return runtimeId.toString();
}
-
- @Override
- public FeedRuntimeInputHandler getInputHandler() {
- return inputHandler;
- }
-
- public Mode getMode() {
- return inputHandler != null ? inputHandler.getMode() : Mode.PROCESS;
- }
-
- public void setMode(Mode mode) {
- this.inputHandler.setMode(mode);
- }
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java
index f269a06..f888542 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/FeedRuntimeId.java
@@ -21,26 +21,31 @@
import java.io.Serializable;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
+import org.apache.asterix.external.feed.management.FeedId;
public class FeedRuntimeId implements Serializable {
private static final long serialVersionUID = 1L;
- public static final String DEFAULT_OPERAND_ID = "N/A";
+ public static final String DEFAULT_TARGET_ID = "N/A";
+ private final FeedId feedId;
private final FeedRuntimeType runtimeType;
private final int partition;
- private final String operandId;
+ private final String targetId;
+ private final int hashCode;
- public FeedRuntimeId(FeedRuntimeType runtimeType, int partition, String operandId) {
+ public FeedRuntimeId(FeedId feedId, FeedRuntimeType runtimeType, int partition, String operandId) {
+ this.feedId = feedId;
this.runtimeType = runtimeType;
this.partition = partition;
- this.operandId = operandId;
+ this.targetId = operandId;
+ this.hashCode = toString().hashCode();
}
@Override
public String toString() {
- return runtimeType + "[" + partition + "]" + "{" + operandId + "}";
+ return runtimeType + "(" + feedId + ")" + "[" + partition + "]" + "==>" + "{" + targetId + "}";
}
@Override
@@ -52,13 +57,13 @@
return false;
}
FeedRuntimeId other = (FeedRuntimeId) o;
- return (other.getFeedRuntimeType().equals(runtimeType) && other.getOperandId().equals(operandId) && other
- .getPartition() == partition);
+ return (other.feedId.equals(feedId) && other.getFeedRuntimeType().equals(runtimeType)
+ && other.getTargetId().equals(targetId) && other.getPartition() == partition);
}
@Override
public int hashCode() {
- return toString().hashCode();
+ return hashCode;
}
public FeedRuntimeType getFeedRuntimeType() {
@@ -73,8 +78,11 @@
return runtimeType;
}
- public String getOperandId() {
- return operandId;
+ public String getTargetId() {
+ return targetId;
}
+ public FeedId getFeedId() {
+ return feedId;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
index 021ad36..5e8c022 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/IngestionRuntime.java
@@ -21,70 +21,66 @@
import java.nio.ByteBuffer;
import java.util.logging.Level;
-import org.apache.asterix.external.api.IAdapterRuntimeManager;
+import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.dataflow.FrameDistributor;
import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.io.MessagingFrameTupleAppender;
public class IngestionRuntime extends SubscribableRuntime {
- private final IAdapterRuntimeManager adapterRuntimeManager;
+ private final AdapterRuntimeManager adapterRuntimeManager;
private final IHyracksTaskContext ctx;
+ private int numSubscribers = 0;
public IngestionRuntime(FeedId feedId, FeedRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
- RecordDescriptor recordDesc, IAdapterRuntimeManager adaptorRuntimeManager, IHyracksTaskContext ctx) {
- super(feedId, runtimeId, null, feedWriter, recordDesc);
+ AdapterRuntimeManager adaptorRuntimeManager, IHyracksTaskContext ctx) {
+ super(feedId, runtimeId, feedWriter);
this.adapterRuntimeManager = adaptorRuntimeManager;
this.ctx = ctx;
}
@Override
- public void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime) throws Exception {
- FeedFrameCollector reader = dWriter.subscribeFeed(fpa, collectionRuntime.getInputHandler(),
- collectionRuntime.getConnectionId());
- collectionRuntime.setFrameCollector(reader);
-
- if (dWriter.getDistributionMode().equals(FrameDistributor.DistributionMode.SINGLE)) {
+ public synchronized void subscribe(CollectionRuntime collectionRuntime) throws HyracksDataException {
+ FeedFrameCollector collector = collectionRuntime.getFrameCollector();
+ dWriter.subscribe(collector);
+ subscribers.add(collectionRuntime);
+ if (numSubscribers == 0) {
ctx.setSharedObject(ByteBuffer.allocate(MessagingFrameTupleAppender.MAX_MESSAGE_SIZE));
+ collectionRuntime.getCtx().setSharedObject(ctx.getSharedObject());
adapterRuntimeManager.start();
}
- subscribers.add(collectionRuntime);
+ numSubscribers++;
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Subscribed feed collection [" + collectionRuntime + "] to " + this);
}
- collectionRuntime.getCtx().setSharedObject(ctx.getSharedObject());
}
@Override
- public void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception {
- if (dWriter.getDistributionMode().equals(FrameDistributor.DistributionMode.SINGLE)) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Stopping adapter for " + this + " as no more registered collectors");
- }
+ public synchronized void unsubscribe(CollectionRuntime collectionRuntime) throws InterruptedException {
+ numSubscribers--;
+ if (numSubscribers == 0) {
adapterRuntimeManager.stop();
- } else {
- dWriter.unsubscribeFeed(collectionRuntime.getInputHandler());
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Unsubscribed feed collection [" + collectionRuntime + "] from " + this);
}
subscribers.remove(collectionRuntime);
}
- public void endOfFeed() throws InterruptedException {
- dWriter.notifyEndOfFeed();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Notified End Of Feed [" + this + "]");
- }
+ public AdapterRuntimeManager getAdapterRuntimeManager() {
+ return adapterRuntimeManager;
}
- public IAdapterRuntimeManager getAdapterRuntimeManager() {
- return adapterRuntimeManager;
+ public void terminate() {
+ for (ISubscriberRuntime subscriber : subscribers) {
+ try {
+ unsubscribe((CollectionRuntime) subscriber);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Excpetion in unsubscribing " + subscriber + " message " + e.getMessage());
+ }
+ }
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableFeedRuntimeId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableFeedRuntimeId.java
deleted file mode 100644
index f6db99c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableFeedRuntimeId.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.asterix.external.feed.runtime;
-
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.management.FeedId;
-
-public class SubscribableFeedRuntimeId extends FeedRuntimeId {
- private static final long serialVersionUID = 1L;
- private final FeedId feedId;
-
- public SubscribableFeedRuntimeId(FeedId feedId, FeedRuntimeType runtimeType, int partition) {
- super(runtimeType, partition, FeedRuntimeId.DEFAULT_OPERAND_ID);
- this.feedId = feedId;
- }
-
- public FeedId getFeedId() {
- return feedId;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof SubscribableFeedRuntimeId)) {
- return false;
- }
-
- return (super.equals(o) && this.feedId.equals(((SubscribableFeedRuntimeId) o).getFeedId()));
- }
-
- @Override
- public int hashCode() {
- return super.hashCode() + feedId.hashCode();
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
index 056875c..e060e27 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/runtime/SubscribableRuntime.java
@@ -25,26 +25,18 @@
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-public class SubscribableRuntime extends FeedRuntime implements ISubscribableRuntime {
+public abstract class SubscribableRuntime extends FeedRuntime implements ISubscribableRuntime {
protected static final Logger LOGGER = Logger.getLogger(SubscribableRuntime.class.getName());
-
protected final FeedId feedId;
protected final List<ISubscriberRuntime> subscribers;
- protected final RecordDescriptor recordDescriptor;
protected final DistributeFeedFrameWriter dWriter;
- public SubscribableRuntime(FeedId feedId, FeedRuntimeId runtimeId, FeedRuntimeInputHandler inputHandler,
- DistributeFeedFrameWriter dWriter, RecordDescriptor recordDescriptor) {
- super(runtimeId, inputHandler, dWriter);
+ public SubscribableRuntime(FeedId feedId, FeedRuntimeId runtimeId, DistributeFeedFrameWriter dWriter) {
+ super(runtimeId);
this.feedId = feedId;
- this.recordDescriptor = recordDescriptor;
this.dWriter = dWriter;
this.subscribers = new ArrayList<ISubscriberRuntime>();
}
@@ -58,38 +50,7 @@
return "SubscribableRuntime" + " [" + feedId + "]" + "(" + runtimeId + ")";
}
- @Override
- public synchronized void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime)
- throws Exception {
- FeedFrameCollector collector = dWriter.subscribeFeed(new FeedPolicyAccessor(collectionRuntime.getFeedPolicy()),
- collectionRuntime.getInputHandler(), collectionRuntime.getConnectionId());
- collectionRuntime.setFrameCollector(collector);
- subscribers.add(collectionRuntime);
- }
-
- @Override
- public synchronized void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception {
- dWriter.unsubscribeFeed(collectionRuntime.getFeedFrameWriter());
- subscribers.remove(collectionRuntime);
- }
-
- @Override
- public synchronized List<ISubscriberRuntime> getSubscribers() {
- return subscribers;
- }
-
- @Override
- public DistributeFeedFrameWriter getFeedFrameWriter() {
- return dWriter;
- }
-
public FeedRuntimeType getFeedRuntimeType() {
return runtimeId.getFeedRuntimeType();
}
-
- @Override
- public RecordDescriptor getRecordDescriptor() {
- return recordDescriptor;
- }
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/BasicMonitoredBuffer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/BasicMonitoredBuffer.java
deleted file mode 100644
index ad40608..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/BasicMonitoredBuffer.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFramePostProcessor;
-import org.apache.asterix.external.feed.api.IFramePreprocessor;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class BasicMonitoredBuffer extends MonitoredBuffer {
-
- public BasicMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter, FrameTupleAccessor fta,
- RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
- FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
- IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
- super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
- exceptionHandler, callback, nPartitions, policyAccessor);
- }
-
- @Override
- protected boolean monitorProcessingRate() {
- return false;
- }
-
- @Override
- protected boolean logInflowOutflowRate() {
- return false;
- }
-
- @Override
- protected IFramePreprocessor getFramePreProcessor() {
- return null;
- }
-
- @Override
- protected IFramePostProcessor getFramePostProcessor() {
- return null;
- }
-
- @Override
- protected boolean monitorInputQueueLength() {
- return false;
- }
-
- @Override
- protected boolean reportInflowRate() {
- return false;
- }
-
- @Override
- protected boolean reportOutflowRate() {
- return false;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/ComputeSideMonitoredBuffer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/ComputeSideMonitoredBuffer.java
deleted file mode 100644
index 211fc7b..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/ComputeSideMonitoredBuffer.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFramePostProcessor;
-import org.apache.asterix.external.feed.api.IFramePreprocessor;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class ComputeSideMonitoredBuffer extends MonitoredBuffer {
-
- public ComputeSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
- FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
- FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
- IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
- super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
- exceptionHandler, callback, nPartitions, policyAccessor);
- }
-
- @Override
- protected boolean monitorProcessingRate() {
- return true;
- }
-
- protected boolean logInflowOutflowRate() {
- return true;
- }
-
- @Override
- protected boolean monitorInputQueueLength() {
- return true;
- }
-
- @Override
- protected IFramePreprocessor getFramePreProcessor() {
- return null;
- }
-
- @Override
- protected IFramePostProcessor getFramePostProcessor() {
- return null;
- }
-
- @Override
- protected boolean reportOutflowRate() {
- return false;
- }
-
- @Override
- protected boolean reportInflowRate() {
- return false;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedMetricCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedMetricCollector.java
deleted file mode 100644
index f0db639..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/FeedMetricCollector.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-public class FeedMetricCollector implements IFeedMetricCollector {
-
- private static final Logger LOGGER = Logger.getLogger(FeedMetricCollector.class.getName());
-
- private static final int UNKNOWN = -1;
-
- private final AtomicInteger globalSenderId = new AtomicInteger(1);
- private final Map<Integer, Sender> senders = new HashMap<Integer, Sender>();
- private final Map<Integer, Series> statHistory = new HashMap<Integer, Series>();
- private final Map<String, Sender> sendersByName = new HashMap<String, Sender>();
-
- public FeedMetricCollector(String nodeId) {
- }
-
- @Override
- public synchronized int createReportSender(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- ValueType valueType, MetricType metricType) {
- Sender sender = new Sender(globalSenderId.getAndIncrement(), connectionId, runtimeId, valueType, metricType);
- senders.put(sender.senderId, sender);
- sendersByName.put(sender.getDisplayName(), sender);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Sender id " + sender.getSenderId() + " created for " + sender);
- }
- return sender.senderId;
- }
-
- @Override
- public void removeReportSender(int senderId) {
- Sender sender = senders.get(senderId);
- if (sender != null) {
- statHistory.remove(senderId);
- senders.remove(senderId);
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to remove sender Id");
- }
- throw new IllegalStateException("Unable to remove sender Id " + senderId + " senders " + senders);
- }
- }
-
- @Override
- public boolean sendReport(int senderId, int value) {
- Sender sender = senders.get(senderId);
- if (sender != null) {
- Series series = statHistory.get(sender.senderId);
- if (series == null) {
- switch (sender.mType) {
- case AVG:
- series = new SeriesAvg();
- break;
- case RATE:
- series = new SeriesRate();
- break;
- }
- statHistory.put(sender.senderId, series);
- }
- series.addValue(value);
- return true;
- }
- throw new IllegalStateException("Unable to send report sender Id " + senderId + " senders " + senders);
- }
-
- @Override
- public void resetReportSender(int senderId) {
- Sender sender = senders.get(senderId);
- if (sender != null) {
- Series series = statHistory.get(sender.senderId);
- if (series != null) {
- series.reset();
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Sender with id " + senderId + " not found. Unable to reset!");
- }
- throw new IllegalStateException("Unable to reset sender Id " + senderId + " senders " + senders);
- }
- }
-
- private static class Sender {
-
- private final int senderId;
- private final MetricType mType;
- private final String displayName;
-
- public Sender(int senderId, FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType,
- MetricType mType) {
- this.senderId = senderId;
- this.mType = mType;
- this.displayName = createDisplayName(connectionId, runtimeId, valueType);
- }
-
- @Override
- public String toString() {
- return displayName + "[" + senderId + "]" + "(" + mType + ")";
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof Sender)) {
- return false;
- }
- return ((Sender) o).senderId == senderId;
- }
-
- @Override
- public int hashCode() {
- return senderId;
- }
-
- public static String createDisplayName(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- ValueType valueType) {
- return connectionId + " (" + runtimeId.getFeedRuntimeType() + " )" + "[" + runtimeId.getPartition() + "]"
- + "{" + valueType + "}";
- }
-
- public String getDisplayName() {
- return displayName;
- }
-
- public int getSenderId() {
- return senderId;
- }
- }
-
- @Override
- public int getMetric(int senderId) {
- Sender sender = senders.get(senderId);
- return getMetric(sender);
- }
-
- @Override
- public int getMetric(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType) {
- String displayName = Sender.createDisplayName(connectionId, runtimeId, valueType);
- Sender sender = sendersByName.get(displayName);
- return getMetric(sender);
- }
-
- private int getMetric(Sender sender) {
- if (sender == null || statHistory.get(sender.getSenderId()) == null) {
- return UNKNOWN;
- }
-
- float result = -1;
- Series series = statHistory.get(sender.getSenderId());
- switch (sender.mType) {
- case AVG:
- result = ((SeriesAvg) series).getAvg();
- break;
- case RATE:
- result = ((SeriesRate) series).getRate();
- break;
- }
- return (int) result;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakePartitionStatistics.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakePartitionStatistics.java
deleted file mode 100644
index acfd1fb..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakePartitionStatistics.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.util.BitSet;
-
-public class IntakePartitionStatistics {
-
- public static int ACK_WINDOW_SIZE = 1024;
- private BitSet bitSet;
-
- public IntakePartitionStatistics(int partition, int base) {
- this.bitSet = new BitSet(ACK_WINDOW_SIZE);
- }
-
- public void ackRecordId(int recordId) {
- int posIndexWithinBase = recordId % ACK_WINDOW_SIZE;
- this.bitSet.set(posIndexWithinBase);
- }
-
- public byte[] getAckInfo() {
- return bitSet.toByteArray();
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakeSideMonitoredBuffer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakeSideMonitoredBuffer.java
deleted file mode 100644
index 7a79e23..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/IntakeSideMonitoredBuffer.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFramePostProcessor;
-import org.apache.asterix.external.feed.api.IFramePreprocessor;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class IntakeSideMonitoredBuffer extends MonitoredBuffer {
-
- public IntakeSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
- FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
- FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
- IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
- super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
- exceptionHandler, callback, nPartitions, policyAccessor);
- }
-
- @Override
- protected boolean monitorProcessingRate() {
- return false;
- }
-
- @Override
- protected boolean logInflowOutflowRate() {
- return false;
- }
-
- @Override
- protected IFramePreprocessor getFramePreProcessor() {
- return null;
- }
-
- @Override
- protected IFramePostProcessor getFramePostProcessor() {
- return null;
- }
-
- @Override
- protected boolean monitorInputQueueLength() {
- return false;
- }
-
- @Override
- protected boolean reportOutflowRate() {
- return false;
- }
-
- @Override
- protected boolean reportInflowRate() {
- return true;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
deleted file mode 100644
index 67e6295..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBuffer.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.nio.ByteBuffer;
-import java.util.Map;
-import java.util.Timer;
-import java.util.TimerTask;
-import java.util.logging.Level;
-
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.MetricType;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.ValueType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFrameEventCallback.FrameEvent;
-import org.apache.asterix.external.feed.api.IFramePostProcessor;
-import org.apache.asterix.external.feed.api.IFramePreprocessor;
-import org.apache.asterix.external.feed.dataflow.DataBucket;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.dataflow.StorageFrameHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.MessageReceiver;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.feed.watch.MonitoredBufferTimerTasks.LogInputOutputRateTask;
-import org.apache.asterix.external.feed.watch.MonitoredBufferTimerTasks.MonitorInputQueueLengthTimerTask;
-import org.apache.asterix.external.feed.watch.MonitoredBufferTimerTasks.MonitoreProcessRateTimerTask;
-import org.apache.asterix.external.feed.watch.MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask;
-import org.apache.asterix.external.util.FeedFrameUtil;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public abstract class MonitoredBuffer extends MessageReceiver<DataBucket> {
-
- protected static final long LOG_INPUT_OUTPUT_RATE_FREQUENCY = 5000; // 5 seconds
- protected static final long INPUT_QUEUE_MEASURE_FREQUENCY = 1000; // 1 second
- protected static final long PROCESSING_RATE_MEASURE_FREQUENCY = 10000; // 10 seconds
-
- protected static final int PROCESS_RATE_REFRESH = 2; // refresh processing rate every 10th frame
-
- protected final IHyracksTaskContext ctx;
- protected final FeedConnectionId connectionId;
- protected final FeedRuntimeId runtimeId;
- protected final FrameTupleAccessor inflowFta;
- protected final FrameTupleAccessor outflowFta;
- protected final FeedRuntimeInputHandler inputHandler;
- protected final IFrameEventCallback callback;
- protected final Timer timer;
- private final IExceptionHandler exceptionHandler;
- protected final FeedPolicyAccessor policyAccessor;
- protected int nPartitions;
-
- private IFrameWriter frameWriter;
- protected IFeedMetricCollector metricCollector;
- protected boolean monitorProcessingRate = false;
- protected boolean monitorInputQueueLength = false;
- protected boolean logInflowOutflowRate = false;
- protected boolean reportOutflowRate = false;
- protected boolean reportInflowRate = false;
-
- protected int inflowReportSenderId = -1;
- protected int outflowReportSenderId = -1;
- protected TimerTask monitorInputQueueLengthTask;
- protected TimerTask processingRateTask;
- protected TimerTask logInflowOutflowRateTask;
- protected MonitoredBufferStorageTimerTask storageTimeTrackingRateTask;
- protected StorageFrameHandler storageFromeHandler;
-
- protected int processingRate = -1;
- protected int frameCount = 0;
- private long avgDelayPersistence = 0;
- private boolean active;
- private Map<Integer, Long> tupleTimeStats;
- IFramePostProcessor postProcessor = null;
- IFramePreprocessor preProcessor = null;
-
- public static MonitoredBuffer getMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler,
- IFrameWriter frameWriter, FrameTupleAccessor fta, RecordDescriptor recordDesc,
- IFeedMetricCollector metricCollector, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- IExceptionHandler exceptionHandler, IFrameEventCallback callback, int nPartitions,
- FeedPolicyAccessor policyAccessor) {
- switch (runtimeId.getFeedRuntimeType()) {
- case COMPUTE:
- return new ComputeSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
- connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
- case STORE:
- return new StorageSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
- connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
- case COLLECT:
- return new IntakeSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
- connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
- default:
- return new BasicMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
- connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
- }
- }
-
- protected MonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
- FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
- FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
- IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
- this.ctx = ctx;
- this.connectionId = connectionId;
- this.frameWriter = frameWriter;
- this.inflowFta = new FrameTupleAccessor(recordDesc);
- this.outflowFta = new FrameTupleAccessor(recordDesc);
- this.runtimeId = runtimeId;
- this.metricCollector = metricCollector;
- this.exceptionHandler = exceptionHandler;
- this.callback = callback;
- this.inputHandler = inputHandler;
- this.timer = new Timer();
- this.policyAccessor = policyAccessor;
- this.nPartitions = nPartitions;
- this.active = true;
- initializeMonitoring();
- }
-
- protected abstract boolean monitorProcessingRate();
-
- protected abstract boolean logInflowOutflowRate();
-
- protected abstract boolean reportOutflowRate();
-
- protected abstract boolean reportInflowRate();
-
- protected abstract boolean monitorInputQueueLength();
-
- protected abstract IFramePreprocessor getFramePreProcessor();
-
- protected abstract IFramePostProcessor getFramePostProcessor();
-
- protected void initializeMonitoring() {
- monitorProcessingRate = monitorProcessingRate();
- monitorInputQueueLength = monitorInputQueueLength();
- reportInflowRate = reportInflowRate();
- reportOutflowRate = reportOutflowRate();
- logInflowOutflowRate = policyAccessor.isLoggingStatisticsEnabled() || logInflowOutflowRate();
-
- if (monitorProcessingRate && policyAccessor.isElastic()) { // check possibility to scale in
- this.processingRateTask = new MonitoreProcessRateTimerTask(this, inputHandler.getFeedManager(),
- connectionId, nPartitions);
- this.timer.scheduleAtFixedRate(processingRateTask, 0, PROCESSING_RATE_MEASURE_FREQUENCY);
- }
-
- if (monitorInputQueueLength && (policyAccessor.isElastic() || policyAccessor.throttlingEnabled()
- || policyAccessor.spillToDiskOnCongestion() || policyAccessor.discardOnCongestion())) {
- this.monitorInputQueueLengthTask = new MonitorInputQueueLengthTimerTask(this, callback);
- this.timer.scheduleAtFixedRate(monitorInputQueueLengthTask, 0, INPUT_QUEUE_MEASURE_FREQUENCY);
- }
-
- if (reportInflowRate || reportOutflowRate) {
- this.logInflowOutflowRateTask = new LogInputOutputRateTask(this, logInflowOutflowRate, reportInflowRate,
- reportOutflowRate);
- this.timer.scheduleAtFixedRate(logInflowOutflowRateTask, 0, LOG_INPUT_OUTPUT_RATE_FREQUENCY);
- this.inflowReportSenderId = metricCollector.createReportSender(connectionId, runtimeId,
- ValueType.INFLOW_RATE, MetricType.RATE);
- this.outflowReportSenderId = metricCollector.createReportSender(connectionId, runtimeId,
- ValueType.OUTFLOW_RATE, MetricType.RATE);
- }
- }
-
- protected void deinitializeMonitoring() {
- if (monitorInputQueueLengthTask != null) {
- monitorInputQueueLengthTask.cancel();
- }
- if (processingRateTask != null) {
- processingRateTask.cancel();
- }
- if (reportInflowRate || reportOutflowRate) {
- metricCollector.removeReportSender(inflowReportSenderId);
- metricCollector.removeReportSender(outflowReportSenderId);
- logInflowOutflowRateTask.cancel();
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disabled monitoring for " + this.runtimeId);
- }
- }
-
- protected void postProcessFrame(long startTime, ByteBuffer frame) throws Exception {
- if (monitorProcessingRate) {
- frameCount++;
- if (frameCount % PROCESS_RATE_REFRESH == 0) {
- long endTime = System.currentTimeMillis();
- processingRate = (int) ((double) outflowFta.getTupleCount() * 1000 / (endTime - startTime));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Processing Rate :" + processingRate + " tuples/sec");
- }
- frameCount = 0;
- }
- }
-
- if (logInflowOutflowRate || reportOutflowRate) {
- metricCollector.sendReport(outflowReportSenderId, outflowFta.getTupleCount());
- }
-
- postProcessFrame(frame);
-
- }
-
- protected void preProcessFrame(ByteBuffer frame) throws Exception {
- if (preProcessor == null) {
- preProcessor = getFramePreProcessor();
- }
- if (preProcessor != null) {
- preProcessor.preProcess(frame);
- }
- }
-
- protected void postProcessFrame(ByteBuffer frame) throws Exception {
- if (postProcessor == null) {
- postProcessor = getFramePostProcessor();
- }
- if (postProcessor != null) {
- outflowFta.reset(frame);
- postProcessor.postProcessFrame(frame, outflowFta);
- }
- }
-
- @Override
- public void sendMessage(DataBucket message) {
- inbox.add(message);
- }
-
- public void sendReport(ByteBuffer frame) {
- if ((reportInflowRate) && !(inputHandler.getMode().equals(Mode.PROCESS_BACKLOG)
- || inputHandler.getMode().equals(Mode.PROCESS_SPILL))) {
- inflowFta.reset(frame);
- metricCollector.sendReport(inflowReportSenderId, inflowFta.getTupleCount());
- }
- }
-
- /** return rate in terms of tuples/sec **/
- public int getInflowRate() {
- return metricCollector.getMetric(inflowReportSenderId);
- }
-
- /** return rate in terms of tuples/sec **/
- public int getOutflowRate() {
- return metricCollector.getMetric(outflowReportSenderId);
- }
-
- /** return the number of pending frames from the input queue **/
- public int getWorkSize() {
- return inbox.size();
- }
-
- /** reset the number of partitions (cardinality) for the runtime **/
- public void setNumberOfPartitions(int nPartitions) {
- if (processingRateTask != null) {
- int currentPartitions = ((MonitoreProcessRateTimerTask) processingRateTask).getNumberOfPartitions();
- if (currentPartitions != nPartitions) {
- ((MonitoreProcessRateTimerTask) processingRateTask).setNumberOfPartitions(nPartitions);
- }
- }
- }
-
- public FeedRuntimeInputHandler getInputHandler() {
- return inputHandler;
- }
-
- public synchronized void close(boolean processPending, boolean disableMonitoring) {
- super.close(processPending);
- if (disableMonitoring) {
- deinitializeMonitoring();
- }
- active = false;
- }
-
- @Override
- public synchronized void processMessage(DataBucket message) throws Exception {
- if (!active) {
- message.doneReading();
- return;
- }
- switch (message.getContentType()) {
- case DATA:
- boolean finishedProcessing = false;
- ByteBuffer frameReceived = message.getContent();
- ByteBuffer frameToProcess = null;
- if (inputHandler.isThrottlingEnabled()) {
- inflowFta.reset(frameReceived);
- int pRate = getProcessingRate();
- int inflowRate = getInflowRate();
- if (inflowRate > pRate) {
- double retainFraction = (pRate * 0.8 / inflowRate);
- frameToProcess = throttleFrame(inflowFta, retainFraction);
- inflowFta.reset(frameToProcess);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Throttling at fraction " + retainFraction + "inflow rate " + inflowRate
- + " no of tuples remaining " + inflowFta.getTupleCount());
-
- }
- } else {
- frameToProcess = frameReceived;
- }
- } else {
- frameToProcess = frameReceived;
- }
- outflowFta.reset(frameToProcess);
- long startTime = 0;
- while (!finishedProcessing) {
- try {
- inflowFta.reset(frameToProcess);
- startTime = System.currentTimeMillis();
- preProcessFrame(frameToProcess);
- frameWriter.nextFrame(frameToProcess);
- postProcessFrame(startTime, frameToProcess);
- finishedProcessing = true;
- } catch (Exception e) {
- e.printStackTrace();
- frameToProcess = exceptionHandler.handleException(e, frameToProcess);
- finishedProcessing = true;
- }
- }
- message.doneReading();
- break;
- case EOD:
- message.doneReading();
- timer.cancel();
- callback.frameEvent(FrameEvent.FINISHED_PROCESSING);
- break;
- case EOSD:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Done processing spillage");
- }
- message.doneReading();
- callback.frameEvent(FrameEvent.FINISHED_PROCESSING_SPILLAGE);
- break;
-
- }
- }
-
- private ByteBuffer throttleFrame(FrameTupleAccessor fta, double retainFraction) throws HyracksDataException {
- int desiredTuples = (int) (fta.getTupleCount() * retainFraction);
- return FeedFrameUtil.getSampledFrame(ctx, fta, desiredTuples);
- }
-
- public Mode getMode() {
- return inputHandler.getMode();
- }
-
- public FeedRuntimeId getRuntimeId() {
- return runtimeId;
- }
-
- public void setFrameWriter(IFrameWriter frameWriter) {
- this.frameWriter = frameWriter;
- }
-
- public void reset() {
- active = true;
- if (logInflowOutflowRate) {
- metricCollector.resetReportSender(inflowReportSenderId);
- metricCollector.resetReportSender(outflowReportSenderId);
- }
- }
-
- public int getProcessingRate() {
- return processingRate;
- }
-
- public Map<Integer, Long> getTupleTimeStats() {
- return tupleTimeStats;
- }
-
- public long getAvgDelayRecordPersistence() {
- return avgDelayPersistence;
- }
-
- public MonitoredBufferStorageTimerTask getStorageTimeTrackingRateTask() {
- return storageTimeTrackingRateTask;
- }
-
- @Override
- public void emptyInbox() throws HyracksDataException {
- inputHandler.flush();
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBufferTimerTasks.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBufferTimerTasks.java
deleted file mode 100644
index 86c6bca..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/MonitoredBufferTimerTasks.java
+++ /dev/null
@@ -1,299 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimerTask;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedMessageService;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.ValueType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.api.IFrameEventCallback.FrameEvent;
-import org.apache.asterix.external.feed.dataflow.StorageFrameHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.message.FeedReportMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitAckMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitResponseMessage;
-import org.apache.asterix.external.feed.message.ScaleInReportMessage;
-import org.apache.asterix.external.feed.message.StorageReportFeedMessage;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-
-public class MonitoredBufferTimerTasks {
-
- private static final Logger LOGGER = Logger.getLogger(MonitorInputQueueLengthTimerTask.class.getName());
-
- public static class MonitoredBufferStorageTimerTask extends TimerTask {
-
- private static final int PERSISTENCE_DELAY_VIOLATION_MAX = 5;
-
- private final StorageSideMonitoredBuffer mBuffer;
- private final IFeedManager feedManager;
- private final int partition;
- private final FeedConnectionId connectionId;
- private final FeedPolicyAccessor policyAccessor;
- private final StorageFrameHandler storageFromeHandler;
- private final StorageReportFeedMessage storageReportMessage;
- private final FeedTupleCommitAckMessage tupleCommitAckMessage;
-
- private Map<Integer, Integer> maxIntakeBaseCovered;
- private int countDelayExceeded = 0;
-
- public MonitoredBufferStorageTimerTask(StorageSideMonitoredBuffer mBuffer, IFeedManager feedManager,
- FeedConnectionId connectionId, int partition, FeedPolicyAccessor policyAccessor,
- StorageFrameHandler storageFromeHandler) {
- this.mBuffer = mBuffer;
- this.feedManager = feedManager;
- this.connectionId = connectionId;
- this.partition = partition;
- this.policyAccessor = policyAccessor;
- this.storageFromeHandler = storageFromeHandler;
- this.storageReportMessage = new StorageReportFeedMessage(this.connectionId, this.partition, 0, false, 0, 0);
- this.tupleCommitAckMessage = new FeedTupleCommitAckMessage(this.connectionId, 0, 0, null);
- this.maxIntakeBaseCovered = new HashMap<Integer, Integer>();
- }
-
- @Override
- public void run() {
- if (mBuffer.isAckingEnabled() && !mBuffer.getInputHandler().isThrottlingEnabled()) {
- ackRecords();
- }
- if (mBuffer.isTimeTrackingEnabled()) {
- checkLatencyViolation();
- }
- }
-
- private void ackRecords() {
- Set<Integer> partitions = storageFromeHandler.getPartitionsWithStats();
- List<Integer> basesCovered = new ArrayList<Integer>();
- for (int intakePartition : partitions) {
- Map<Integer, IntakePartitionStatistics> baseAcks = storageFromeHandler
- .getBaseAcksForPartition(intakePartition);
- for (Entry<Integer, IntakePartitionStatistics> entry : baseAcks.entrySet()) {
- int base = entry.getKey();
- IntakePartitionStatistics stats = entry.getValue();
- Integer maxIntakeBaseForPartition = maxIntakeBaseCovered.get(intakePartition);
- if (maxIntakeBaseForPartition == null || maxIntakeBaseForPartition < base) {
- tupleCommitAckMessage.reset(intakePartition, base, stats.getAckInfo());
- feedManager.getFeedMessageService().sendMessage(tupleCommitAckMessage);
- } else {
- basesCovered.add(base);
- }
- }
- for (Integer b : basesCovered) {
- baseAcks.remove(b);
- }
- basesCovered.clear();
- }
- }
-
- private void checkLatencyViolation() {
- long avgDelayPersistence = storageFromeHandler.getAvgDelayPersistence();
- if (avgDelayPersistence > policyAccessor.getMaxDelayRecordPersistence()) {
- countDelayExceeded++;
- if (countDelayExceeded > PERSISTENCE_DELAY_VIOLATION_MAX) {
- storageReportMessage.reset(0, false, mBuffer.getAvgDelayRecordPersistence());
- feedManager.getFeedMessageService().sendMessage(storageReportMessage);
- }
- } else {
- countDelayExceeded = 0;
- }
- }
-
- public void receiveCommitAckResponse(FeedTupleCommitResponseMessage message) {
- maxIntakeBaseCovered.put(message.getIntakePartition(), message.getMaxWindowAcked());
- }
- }
-
- public static class LogInputOutputRateTask extends TimerTask {
-
- private final MonitoredBuffer mBuffer;
- private final boolean log;
- private final boolean reportInflow;
- private final boolean reportOutflow;
-
- private final IFeedMessageService messageService;
- private final FeedReportMessage message;
-
- public LogInputOutputRateTask(MonitoredBuffer mBuffer, boolean log, boolean reportInflow, boolean reportOutflow) {
- this.mBuffer = mBuffer;
- this.log = log;
- this.reportInflow = reportInflow;
- this.reportOutflow = reportOutflow;
- if (reportInflow || reportOutflow) {
- ValueType vType = reportInflow ? ValueType.INFLOW_RATE : ValueType.OUTFLOW_RATE;
- messageService = mBuffer.getInputHandler().getFeedManager().getFeedMessageService();
- message = new FeedReportMessage(mBuffer.getInputHandler().getConnectionId(), mBuffer.getRuntimeId(),
- vType, 0);
- } else {
- messageService = null;
- message = null;
- }
-
- }
-
- @Override
- public void run() {
- int pendingWork = mBuffer.getWorkSize();
- int outflowRate = mBuffer.getOutflowRate();
- int inflowRate = mBuffer.getInflowRate();
- if (log) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(mBuffer.getRuntimeId() + " " + "Inflow rate:" + inflowRate + " Outflow Rate:"
- + outflowRate + " Pending Work " + pendingWork);
- }
- }
- if (reportInflow) {
- message.reset(inflowRate);
- } else if (reportOutflow) {
- message.reset(outflowRate);
- }
- messageService.sendMessage(message);
- }
- }
-
- public static class MonitorInputQueueLengthTimerTask extends TimerTask {
-
- private final MonitoredBuffer mBuffer;
- private final IFrameEventCallback callback;
- private final int pendingWorkThreshold;
- private final int maxSuccessiveThresholdPeriods;
- private FrameEvent lastEvent = FrameEvent.NO_OP;
- private int pendingWorkExceedCount = 0;
-
- public MonitorInputQueueLengthTimerTask(MonitoredBuffer mBuffer, IFrameEventCallback callback) {
- this.mBuffer = mBuffer;
- this.callback = callback;
- AsterixFeedProperties props = mBuffer.getInputHandler().getFeedManager().getAsterixFeedProperties();
- pendingWorkThreshold = props.getPendingWorkThreshold();
- maxSuccessiveThresholdPeriods = props.getMaxSuccessiveThresholdPeriod();
- }
-
- @Override
- public void run() {
- int pendingWork = mBuffer.getWorkSize();
- if (mBuffer.getMode().equals(Mode.PROCESS_SPILL) || mBuffer.getMode().equals(Mode.PROCESS_BACKLOG)) {
- return;
- }
-
- switch (lastEvent) {
- case NO_OP:
- case PENDING_WORK_DONE:
- case FINISHED_PROCESSING_SPILLAGE:
- if (pendingWork > pendingWorkThreshold) {
- pendingWorkExceedCount++;
- if (pendingWorkExceedCount > maxSuccessiveThresholdPeriods) {
- pendingWorkExceedCount = 0;
- lastEvent = FrameEvent.PENDING_WORK_THRESHOLD_REACHED;
- callback.frameEvent(lastEvent);
- }
- } else if (pendingWork == 0 && mBuffer.getMode().equals(Mode.SPILL)) {
- lastEvent = FrameEvent.PENDING_WORK_DONE;
- callback.frameEvent(lastEvent);
- }
- break;
- case PENDING_WORK_THRESHOLD_REACHED:
- if (((pendingWork * 1.0) / pendingWorkThreshold) <= 0.5) {
- lastEvent = FrameEvent.PENDING_WORK_DONE;
- callback.frameEvent(lastEvent);
- }
- break;
- case FINISHED_PROCESSING:
- break;
-
- }
- }
- }
-
- /**
- * A timer task to measure and compare the processing rate and inflow rate
- * to look for possibility to scale-in, that is reduce the degree of cardinality
- * of the compute operator.
- */
- public static class MonitoreProcessRateTimerTask extends TimerTask {
-
- private final MonitoredBuffer mBuffer;
- private final IFeedManager feedManager;
- private int nPartitions;
- private ScaleInReportMessage sMessage;
- private boolean proposedChange;
-
- public MonitoreProcessRateTimerTask(MonitoredBuffer mBuffer, IFeedManager feedManager,
- FeedConnectionId connectionId, int nPartitions) {
- this.mBuffer = mBuffer;
- this.feedManager = feedManager;
- this.nPartitions = nPartitions;
- this.sMessage = new ScaleInReportMessage(connectionId, FeedRuntimeType.COMPUTE, 0, 0);
- this.proposedChange = false;
- }
-
- public int getNumberOfPartitions() {
- return nPartitions;
- }
-
- public void setNumberOfPartitions(int nPartitions) {
- this.nPartitions = nPartitions;
- proposedChange = false;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Reset the number of partitions for " + mBuffer.getRuntimeId() + " to " + nPartitions);
- }
- }
-
- @Override
- public void run() {
- if (!proposedChange) {
- int inflowRate = mBuffer.getInflowRate();
- int procRate = mBuffer.getProcessingRate();
- if (inflowRate > 0 && procRate > 0) {
- if (inflowRate < procRate) {
- int possibleCardinality = (int) Math.ceil(nPartitions * inflowRate / (double) procRate);
- if (possibleCardinality < nPartitions
- && ((((nPartitions - possibleCardinality) * 1.0) / nPartitions) >= 0.25)) {
- sMessage.reset(nPartitions, possibleCardinality);
- feedManager.getFeedMessageService().sendMessage(sMessage);
- proposedChange = true;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Proposed scale-in " + sMessage);
- }
- }
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Inflow Rate (" + inflowRate + ") exceeds Processing Rate" + " (" + procRate
- + ")");
- }
- }
- }
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Waiting for earlier proposal to scale in to be applied");
- }
- }
- }
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoad.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoad.java
deleted file mode 100644
index d3919b5..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoad.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-
-public class NodeLoad implements Comparable<NodeLoad> {
-
- private final String nodeId;
-
- private int nRuntimes;
-
- public NodeLoad(String nodeId) {
- this.nodeId = nodeId;
- this.nRuntimes = 0;
- }
-
- public void addLoad() {
- nRuntimes++;
- }
-
- public void removeLoad(FeedRuntimeType runtimeType) {
- nRuntimes--;
- }
-
- @Override
- public int compareTo(NodeLoad o) {
- if (this == o) {
- return 0;
- }
- return nRuntimes - o.getnRuntimes();
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public int getnRuntimes() {
- return nRuntimes;
- }
-
- public void setnRuntimes(int nRuntimes) {
- this.nRuntimes = nRuntimes;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReport.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReport.java
deleted file mode 100644
index bfddcf6..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReport.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.util.FeedConstants;
-import org.json.JSONException;
-import org.json.JSONObject;
-
-public class NodeLoadReport implements Comparable<NodeLoadReport> {
-
- private final String nodeId;
- private float cpuLoad;
- private double usedHeap;
- private int nRuntimes;
-
- public NodeLoadReport(String nodeId, float cpuLoad, float usedHeap, int nRuntimes) {
- this.nodeId = nodeId;
- this.cpuLoad = cpuLoad;
- this.usedHeap = usedHeap;
- this.nRuntimes = nRuntimes;
- }
-
- public static NodeLoadReport read(JSONObject obj) throws JSONException {
- NodeLoadReport r = new NodeLoadReport(obj.getString(FeedConstants.MessageConstants.NODE_ID),
- (float) obj.getDouble(FeedConstants.MessageConstants.CPU_LOAD),
- (float) obj.getDouble(FeedConstants.MessageConstants.HEAP_USAGE),
- obj.getInt(FeedConstants.MessageConstants.N_RUNTIMES));
- return r;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (!(o instanceof NodeLoadReport)) {
- return false;
- }
- return ((NodeLoadReport) o).nodeId.equals(nodeId);
- }
-
- @Override
- public int hashCode() {
- return nodeId.hashCode();
- }
-
- @Override
- public int compareTo(NodeLoadReport o) {
- if (nRuntimes != o.getnRuntimes()) {
- return nRuntimes - o.getnRuntimes();
- } else {
- return (int) (this.cpuLoad - ((NodeLoadReport) o).cpuLoad);
- }
- }
-
- public float getCpuLoad() {
- return cpuLoad;
- }
-
- public void setCpuLoad(float cpuLoad) {
- this.cpuLoad = cpuLoad;
- }
-
- public double getUsedHeap() {
- return usedHeap;
- }
-
- public void setUsedHeap(double usedHeap) {
- this.usedHeap = usedHeap;
- }
-
- public int getnRuntimes() {
- return nRuntimes;
- }
-
- public void setnRuntimes(int nRuntimes) {
- this.nRuntimes = nRuntimes;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReportService.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReportService.java
deleted file mode 100644
index f651935..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/NodeLoadReportService.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.lang.management.OperatingSystemMXBean;
-import java.util.List;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedMessageService;
-import org.apache.asterix.external.feed.api.IFeedService;
-import org.apache.asterix.external.feed.message.NodeReportMessage;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-
-public class NodeLoadReportService implements IFeedService {
-
- private static final int NODE_LOAD_REPORT_FREQUENCY = 2000;
- private static final float CPU_CHANGE_THRESHOLD = 0.2f;
- private static final float HEAP_CHANGE_THRESHOLD = 0.4f;
-
- private final NodeLoadReportTask task;
- private final Timer timer;
-
- public NodeLoadReportService(String nodeId, IFeedManager feedManager) {
- this.task = new NodeLoadReportTask(nodeId, feedManager);
- this.timer = new Timer();
- }
-
- @Override
- public void start() throws Exception {
- timer.schedule(task, 0, NODE_LOAD_REPORT_FREQUENCY);
- }
-
- @Override
- public void stop() {
- timer.cancel();
- }
-
- private static class NodeLoadReportTask extends TimerTask {
-
- private final IFeedManager feedManager;
- private final NodeReportMessage message;
- private final IFeedMessageService messageService;
-
- private static OperatingSystemMXBean osBean = ManagementFactory.getOperatingSystemMXBean();
- private static MemoryMXBean memBean = ManagementFactory.getMemoryMXBean();
-
- public NodeLoadReportTask(String nodeId, IFeedManager feedManager) {
- this.feedManager = feedManager;
- this.message = new NodeReportMessage(0.0f, 0L, 0);
- this.messageService = feedManager.getFeedMessageService();
- }
-
- @Override
- public void run() {
- List<FeedRuntimeId> runtimeIds = feedManager.getFeedConnectionManager().getRegisteredRuntimes();
- int nRuntimes = runtimeIds.size();
- double cpuLoad = getCpuLoad();
- double usedHeap = getUsedHeap();
- if (sendMessage(nRuntimes, cpuLoad, usedHeap)) {
- message.reset(cpuLoad, usedHeap, nRuntimes);
- messageService.sendMessage(message);
- }
- }
-
- private boolean sendMessage(int nRuntimes, double cpuLoad, double usedHeap) {
- if (message == null) {
- return true;
- }
-
- boolean changeInCpu = (Math.abs(cpuLoad - message.getCpuLoad())
- / message.getCpuLoad()) > CPU_CHANGE_THRESHOLD;
- boolean changeInUsedHeap = (Math.abs(usedHeap - message.getUsedHeap())
- / message.getUsedHeap()) > HEAP_CHANGE_THRESHOLD;
- boolean changeInRuntimeSize = nRuntimes != message.getnRuntimes();
- return changeInCpu || changeInUsedHeap || changeInRuntimeSize;
- }
-
- private double getCpuLoad() {
- return osBean.getSystemLoadAverage();
- }
-
- private double getUsedHeap() {
- return ((double) memBean.getHeapMemoryUsage().getUsed()) / memBean.getHeapMemoryUsage().getMax();
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/Series.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/Series.java
deleted file mode 100644
index ec95371..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/Series.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.MetricType;
-
-public abstract class Series {
-
- protected final MetricType type;
- protected int runningSum;
-
- public Series(MetricType type) {
- this.type = type;
- }
-
- public abstract void addValue(int value);
-
- public int getRunningSum() {
- return runningSum;
- }
-
- public MetricType getType() {
- return type;
- }
-
- public abstract void reset();
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesAvg.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesAvg.java
deleted file mode 100644
index 6182753..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesAvg.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.MetricType;
-
-public class SeriesAvg extends Series {
-
- private int count;
-
- public SeriesAvg() {
- super(MetricType.AVG);
- }
-
- public int getAvg() {
- return runningSum / count;
- }
-
- public synchronized void addValue(int value) {
- if (value < 0) {
- return;
- }
- runningSum += value;
- count++;
- }
-
- public void reset(){
- count = 0;
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesRate.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesRate.java
deleted file mode 100644
index 91eea87..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/SeriesRate.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.asterix.external.feed.api.IFeedMetricCollector.MetricType;
-
-public class SeriesRate extends Series {
-
- private static final long REFRESH_MEASUREMENT = 5000; // 5 seconds
-
- private int rate;
- private Timer timer;
- private RateComputingTask task;
-
- public SeriesRate() {
- super(MetricType.RATE);
- begin();
- }
-
- public int getRate() {
- return rate;
- }
-
- public synchronized void addValue(int value) {
- if (value < 0) {
- return;
- }
- runningSum += value;
- }
-
- public void begin() {
- if (timer == null) {
- timer = new Timer();
- task = new RateComputingTask(this);
- timer.scheduleAtFixedRate(task, 0, REFRESH_MEASUREMENT);
- }
- }
-
- public void end() {
- if (timer != null) {
- timer.cancel();
- }
- }
-
- public void reset() {
- rate = 0;
- if (task != null) {
- task.reset();
- }
- }
-
- private class RateComputingTask extends TimerTask {
-
- private int lastMeasured = 0;
- private final SeriesRate series;
-
- public RateComputingTask(SeriesRate series) {
- this.series = series;
- }
-
- @Override
- public void run() {
- int currentValue = series.getRunningSum();
- rate = (int) (((currentValue - lastMeasured) * 1000) / REFRESH_MEASUREMENT);
- lastMeasured = currentValue;
- }
-
- public void reset() {
- lastMeasured = 0;
- }
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
deleted file mode 100644
index 9db930e..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/StorageSideMonitoredBuffer.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/*
- * 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.asterix.external.feed.watch;
-
-import java.nio.ByteBuffer;
-
-import org.apache.asterix.external.feed.api.IExceptionHandler;
-import org.apache.asterix.external.feed.api.IFeedMetricCollector;
-import org.apache.asterix.external.feed.api.IFrameEventCallback;
-import org.apache.asterix.external.feed.api.IFramePostProcessor;
-import org.apache.asterix.external.feed.api.IFramePreprocessor;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.dataflow.StorageFrameHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.util.FeedConstants.StatisticsConstants;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-public class StorageSideMonitoredBuffer extends MonitoredBuffer {
-
- private static final long STORAGE_TIME_TRACKING_FREQUENCY = 5000;
-
- private boolean ackingEnabled;
- private final boolean timeTrackingEnabled;
-
- public StorageSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler,
- IFrameWriter frameWriter, FrameTupleAccessor fta, RecordDescriptor recordDesc,
- IFeedMetricCollector metricCollector, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
- IExceptionHandler exceptionHandler, IFrameEventCallback callback, int nPartitions,
- FeedPolicyAccessor policyAccessor) {
- super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
- exceptionHandler, callback, nPartitions, policyAccessor);
- timeTrackingEnabled = policyAccessor.isTimeTrackingEnabled();
- ackingEnabled = policyAccessor.atleastOnceSemantics();
- if (ackingEnabled || timeTrackingEnabled) {
- storageFromeHandler = new StorageFrameHandler();
- this.storageTimeTrackingRateTask = new MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask(this,
- inputHandler.getFeedManager(), connectionId, runtimeId.getPartition(), policyAccessor,
- storageFromeHandler);
- this.timer.scheduleAtFixedRate(storageTimeTrackingRateTask, 0, STORAGE_TIME_TRACKING_FREQUENCY);
- }
- }
-
- @Override
- protected boolean monitorProcessingRate() {
- return false;
- }
-
- @Override
- protected boolean logInflowOutflowRate() {
- return true;
- }
-
- @Override
- public IFramePreprocessor getFramePreProcessor() {
- return new IFramePreprocessor() {
-
- @Override
- public void preProcess(ByteBuffer frame) {
- try {
- if (ackingEnabled) {
- storageFromeHandler.updateTrackingInformation(frame, inflowFta);
- }
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- };
- }
-
- @Override
- protected IFramePostProcessor getFramePostProcessor() {
- return new IFramePostProcessor() {
-
- private static final long NORMAL_WINDOW_LIMIT = 400 * 1000;
- private static final long HIGH_WINDOW_LIMIT = 800 * 1000;
-
- private long delayNormalWindow = 0;
- private long delayHighWindow = 0;
- private long delayLowWindow = 0;
-
- private int countNormalWindow;
- private int countHighWindow;
- private int countLowWindow;
-
- private long beginIntakeTimestamp = 0;
-
- @Override
- public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor) {
- if (ackingEnabled || timeTrackingEnabled) {
- int nTuples = frameAccessor.getTupleCount();
- long intakeTimestamp;
- long currentTime = System.currentTimeMillis();
- for (int i = 0; i < nTuples; i++) {
- int recordStart = frameAccessor.getTupleStartOffset(i) + frameAccessor.getFieldSlotsLength();
- int openPartOffsetOrig = frame.getInt(recordStart + 6);
- int numOpenFields = frame.getInt(recordStart + openPartOffsetOrig);
-
- int recordIdOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
- + (StatisticsConstants.INTAKE_TUPLEID.length() + 2) + 1;
-
- int partitionOffset = recordIdOffset + 4 + (StatisticsConstants.INTAKE_PARTITION.length() + 2)
- + 1;
-
- int intakeTimestampValueOffset = partitionOffset + 4
- + (StatisticsConstants.INTAKE_TIMESTAMP.length() + 2) + 1;
- intakeTimestamp = frame.getLong(recordStart + intakeTimestampValueOffset);
- if (beginIntakeTimestamp == 0) {
- beginIntakeTimestamp = intakeTimestamp;
- LOGGER.warning("Begin Timestamp: " + beginIntakeTimestamp);
- }
-
- updateRunningAvg(intakeTimestamp, currentTime);
-
- int storeTimestampValueOffset = intakeTimestampValueOffset + 8
- + (StatisticsConstants.STORE_TIMESTAMP.length() + 2) + 1;
- frame.putLong(recordStart + storeTimestampValueOffset, System.currentTimeMillis());
- }
- logRunningAvg();
- resetRunningAvg();
- }
- }
-
- private void updateRunningAvg(long intakeTimestamp, long currentTime) {
- long diffTimestamp = intakeTimestamp - beginIntakeTimestamp;
- long delay = (currentTime - intakeTimestamp);
- if (diffTimestamp < NORMAL_WINDOW_LIMIT) {
- delayNormalWindow += delay;
- countNormalWindow++;
- } else if (diffTimestamp < HIGH_WINDOW_LIMIT) {
- delayHighWindow += delay;
- countHighWindow++;
- } else {
- delayLowWindow += delay;
- countLowWindow++;
- }
- }
-
- private void resetRunningAvg() {
- delayNormalWindow = 0;
- countNormalWindow = 0;
- delayHighWindow = 0;
- countHighWindow = 0;
- delayLowWindow = 0;
- countLowWindow = 0;
- }
-
- private void logRunningAvg() {
- if (countNormalWindow != 0 && delayNormalWindow != 0) {
- LOGGER.warning("Window:" + 0 + ":" + "Avg Travel_Time:" + (delayNormalWindow / countNormalWindow));
- }
- if (countHighWindow != 0 && delayHighWindow != 0) {
- LOGGER.warning("Window:" + 1 + ":" + "Avg Travel_Time:" + (delayHighWindow / countHighWindow));
- }
- if (countLowWindow != 0 && delayLowWindow != 0) {
- LOGGER.warning("Window:" + 2 + ":" + "Avg Travel_Time:" + (delayLowWindow / countLowWindow));
- }
- }
-
- };
- }
-
- public boolean isAckingEnabled() {
- return ackingEnabled;
- }
-
- public void setAcking(boolean ackingEnabled) {
- this.ackingEnabled = ackingEnabled;
- }
-
- public boolean isTimeTrackingEnabled() {
- return timeTrackingEnabled;
- }
-
- @Override
- protected boolean monitorInputQueueLength() {
- return true;
- }
-
- @Override
- protected boolean reportOutflowRate() {
- return true;
- }
-
- @Override
- protected boolean reportInflowRate() {
- return false;
- }
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
index ce1d893..26f8654 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
@@ -116,14 +116,14 @@
}
socket = null;
} catch (IOException e) {
- hde = ExternalDataExceptionUtils.suppress(hde, e);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, e);
}
try {
if (server != null) {
server.close();
}
} catch (IOException e) {
- hde = ExternalDataExceptionUtils.suppress(hde, e);
+ hde = ExternalDataExceptionUtils.suppressIntoHyracksDataException(hde, e);
} finally {
server = null;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
index 9485b77..36098ee 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorDescriptor.java
@@ -19,17 +19,14 @@
package org.apache.asterix.external.operators;
import java.util.Map;
-import java.util.logging.Logger;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedSubscriptionManager;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
+import org.apache.asterix.external.feed.management.FeedManager;
+import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -48,7 +45,6 @@
public class FeedCollectOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
private static final long serialVersionUID = 1L;
- private static final Logger LOGGER = Logger.getLogger(FeedCollectOperatorDescriptor.class.getName());
/** The type associated with the ADM data output from (the feed adapter OR the compute operator) */
private final IAType outputType;
@@ -59,9 +55,6 @@
/** Map representation of policy parameters */
private final Map<String, String> feedPolicyProperties;
- /** The (singleton) instance of {@code IFeedIngestionManager} **/
- private IFeedSubscriptionManager subscriptionManager;
-
/** The source feed from which the feed derives its data from. **/
private final FeedId sourceFeedId;
@@ -72,7 +65,7 @@
ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
FeedRuntimeType subscriptionLocation) {
super(spec, 0, 1);
- recordDescriptors[0] = rDesc;
+ this.recordDescriptors[0] = rDesc;
this.outputType = atype;
this.connectionId = feedConnectionId;
this.feedPolicyProperties = feedPolicyProperties;
@@ -84,22 +77,11 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- this.subscriptionManager = ((IFeedManager) runtimeCtx.getFeedManager()).getFeedSubscriptionManager();
- ISubscribableRuntime sourceRuntime = null;
- SubscribableFeedRuntimeId feedSubscribableRuntimeId = new SubscribableFeedRuntimeId(sourceFeedId,
- subscriptionLocation, partition);
- switch (subscriptionLocation) {
- case INTAKE:
- sourceRuntime = getIntakeRuntime(feedSubscribableRuntimeId);
- break;
- case COMPUTE:
- sourceRuntime = subscriptionManager.getSubscribableRuntime(feedSubscribableRuntimeId);
- break;
- default:
- throw new HyracksDataException("Can't subscirbe to FeedRuntime with Type: " + subscriptionLocation);
- }
+ FeedManager feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject()).getFeedManager();
+ FeedRuntimeId sourceRuntimeId =
+ new FeedRuntimeId(sourceFeedId, subscriptionLocation, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ISubscribableRuntime sourceRuntime = feedManager.getSubscribableRuntime(sourceRuntimeId);
return new FeedCollectOperatorNodePushable(ctx, sourceFeedId, connectionId, feedPolicyProperties, partition,
nPartitions, sourceRuntime);
}
@@ -124,10 +106,6 @@
return sourceFeedId;
}
- private IngestionRuntime getIntakeRuntime(SubscribableFeedRuntimeId subscribableRuntimeId) {
- return (IngestionRuntime) subscriptionManager.getSubscribableRuntime(subscribableRuntimeId);
- }
-
public FeedRuntimeType getSubscriptionLocation() {
return subscriptionLocation;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
index 178d2d5..aeea6ba 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedCollectOperatorNodePushable.java
@@ -19,29 +19,21 @@
package org.apache.asterix.external.operators;
import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedOperatorOutputSideHandler;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.CollectTransformFeedFrameWriter;
-import org.apache.asterix.external.feed.dataflow.FeedCollectRuntimeInputHandler;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector.State;
+import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
+import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.message.FeedPartitionStartMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
-import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -50,19 +42,14 @@
* The first operator in a collect job in a feed.
*/
public class FeedCollectOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
- private static Logger LOGGER = Logger.getLogger(FeedCollectOperatorNodePushable.class.getName());
private final int partition;
private final FeedConnectionId connectionId;
private final Map<String, String> feedPolicy;
private final FeedPolicyAccessor policyAccessor;
- private final IFeedManager feedManager;
+ private final FeedManager feedManager;
private final ISubscribableRuntime sourceRuntime;
private final IHyracksTaskContext ctx;
- private final int nPartitions;
-
- private RecordDescriptor outputRecordDescriptor;
- private FeedRuntimeInputHandler inputSideHandler;
private CollectionRuntime collectRuntime;
public FeedCollectOperatorNodePushable(IHyracksTaskContext ctx, FeedId sourceFeedId,
@@ -70,139 +57,38 @@
ISubscribableRuntime sourceRuntime) {
this.ctx = ctx;
this.partition = partition;
- this.nPartitions = nPartitions;
this.connectionId = feedConnectionId;
this.sourceRuntime = sourceRuntime;
this.feedPolicy = feedPolicy;
this.policyAccessor = new FeedPolicyAccessor(feedPolicy);
- this.feedManager = (IFeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
}
@Override
public void initialize() throws HyracksDataException {
try {
- outputRecordDescriptor = recordDesc;
- switch (((SubscribableFeedRuntimeId) sourceRuntime.getRuntimeId()).getFeedRuntimeType()) {
- case INTAKE:
- handleCompleteConnection();
- // Notify CC that Collection started
- ctx.sendApplicationMessageToCC(
- new FeedPartitionStartMessage(connectionId.getFeedId(), ctx.getJobletContext().getJobId()),
- null);
- break;
- case COMPUTE:
- handlePartialConnection();
- break;
- default:
- throw new IllegalStateException("Invalid source type "
- + ((SubscribableFeedRuntimeId) sourceRuntime.getRuntimeId()).getFeedRuntimeType());
+ FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), FeedRuntimeType.COLLECT, partition,
+ FeedRuntimeId.DEFAULT_TARGET_ID);
+ // Does this collector have a handler?
+ FrameTupleAccessor tAccessor = new FrameTupleAccessor(recordDesc);
+ if (policyAccessor.bufferingEnabled()) {
+ writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, writer, policyAccessor, tAccessor,
+ feedManager.getFeedMemoryManager());
+ } else {
+ writer = new SyncFeedRuntimeInputHandler(ctx, writer, tAccessor);
}
- State state = collectRuntime.waitTillCollectionOver();
- if (state.equals(State.FINISHED)) {
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId,
- collectRuntime.getRuntimeId());
- writer.close();
- inputSideHandler.close();
- } else if (state.equals(State.HANDOVER)) {
- inputSideHandler.setMode(Mode.STALL);
- writer.close();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ending Collect Operator, the input side handler is now in " + Mode.STALL
- + " and the output writer " + writer + " has been closed ");
- }
- }
- } catch (InterruptedException ie) {
- handleInterruptedException(ie);
+ collectRuntime = new CollectionRuntime(connectionId, runtimeId, sourceRuntime, feedPolicy, ctx,
+ new FeedFrameCollector(policyAccessor, writer, connectionId));
+ feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
+ sourceRuntime.subscribe(collectRuntime);
+ // Notify CC that Collection started
+ ctx.sendApplicationMessageToCC(
+ new FeedPartitionStartMessage(connectionId.getFeedId(), ctx.getJobletContext().getJobId()), null);
+ collectRuntime.waitTillCollectionOver();
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, collectRuntime.getRuntimeId());
} catch (Exception e) {
- e.printStackTrace();
throw new HyracksDataException(e);
}
}
-
- private void handleCompleteConnection() throws Exception {
- FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.COLLECT, partition,
- FeedRuntimeId.DEFAULT_OPERAND_ID);
- collectRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager().getFeedRuntime(connectionId,
- runtimeId);
- if (collectRuntime == null) {
- beginNewFeed(runtimeId);
- } else {
- reviveOldFeed();
- }
- }
-
- private void beginNewFeed(FeedRuntimeId runtimeId) throws Exception {
- writer.open();
- IFrameWriter outputSideWriter = writer;
- if (((SubscribableFeedRuntimeId) sourceRuntime.getRuntimeId()).getFeedRuntimeType()
- .equals(FeedRuntimeType.COMPUTE)) {
- outputSideWriter = new CollectTransformFeedFrameWriter(ctx, writer, sourceRuntime, outputRecordDescriptor,
- connectionId);
- this.recordDesc = sourceRuntime.getRecordDescriptor();
- }
-
- FrameTupleAccessor tupleAccessor = new FrameTupleAccessor(recordDesc);
- inputSideHandler = new FeedCollectRuntimeInputHandler(ctx, connectionId, runtimeId, outputSideWriter,
- policyAccessor, false, tupleAccessor, recordDesc, feedManager, nPartitions);
-
- collectRuntime = new CollectionRuntime(connectionId, runtimeId, inputSideHandler, outputSideWriter,
- sourceRuntime, feedPolicy, ctx);
- feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
- sourceRuntime.subscribeFeed(policyAccessor, collectRuntime);
- }
-
- private void reviveOldFeed() throws HyracksDataException {
- writer.open();
- collectRuntime.getFrameCollector().setState(State.ACTIVE);
- inputSideHandler = collectRuntime.getInputHandler();
-
- IFrameWriter innerWriter = inputSideHandler.getCoreOperator();
- if (innerWriter instanceof CollectTransformFeedFrameWriter) {
- ((CollectTransformFeedFrameWriter) innerWriter).reset(this.writer);
- } else {
- inputSideHandler.setCoreOperator(writer);
- }
-
- inputSideHandler.setMode(Mode.PROCESS);
- }
-
- private void handlePartialConnection() throws Exception {
- FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.COMPUTE_COLLECT, partition,
- FeedRuntimeId.DEFAULT_OPERAND_ID);
- writer.open();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Beginning new feed (from existing partial connection):" + connectionId);
- }
- IFeedOperatorOutputSideHandler wrapper = new CollectTransformFeedFrameWriter(ctx, writer, sourceRuntime,
- outputRecordDescriptor, connectionId);
-
- inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, wrapper, policyAccessor, false,
- new FrameTupleAccessor(recordDesc), recordDesc, feedManager, nPartitions);
-
- collectRuntime = new CollectionRuntime(connectionId, runtimeId, inputSideHandler, wrapper, sourceRuntime,
- feedPolicy, ctx);
- feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
- recordDesc = sourceRuntime.getRecordDescriptor();
- sourceRuntime.subscribeFeed(policyAccessor, collectRuntime);
- }
-
- private void handleInterruptedException(InterruptedException ie) throws HyracksDataException {
- if (policyAccessor.continueOnHardwareFailure()) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Continuing on failure as per feed policy, switching to " + Mode.STALL
- + " until failure is resolved");
- }
- inputSideHandler.setMode(Mode.STALL);
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Failure during feed ingestion. Deregistering feed runtime " + collectRuntime
- + " as feed is not configured to handle failures");
- }
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, collectRuntime.getRuntimeId());
- writer.close();
- throw new HyracksDataException(ie);
- }
- }
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
index db11caa..b1fd7a0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
@@ -21,16 +21,10 @@
import java.util.Map;
import java.util.logging.Logger;
-import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedSubscriptionManager;
import org.apache.asterix.external.feed.management.FeedId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
import org.apache.asterix.external.library.ExternalLibraryManager;
import org.apache.asterix.om.types.ARecordType;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -99,27 +93,18 @@
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- IFeedSubscriptionManager feedSubscriptionManager = ((IFeedManager) runtimeCtx.getFeedManager())
- .getFeedSubscriptionManager();
- SubscribableFeedRuntimeId feedIngestionId = new SubscribableFeedRuntimeId(feedId, FeedRuntimeType.INTAKE,
- partition);
- IngestionRuntime ingestionRuntime = (IngestionRuntime) feedSubscriptionManager
- .getSubscribableRuntime(feedIngestionId);
if (adaptorFactory == null) {
try {
- adaptorFactory = createExtenralAdapterFactory(ctx, partition);
+ adaptorFactory = createExternalAdapterFactory(ctx, partition);
} catch (Exception exception) {
throw new HyracksDataException(exception);
}
-
}
- return new FeedIntakeOperatorNodePushable(ctx, feedId, adaptorFactory, partition, ingestionRuntime,
- policyAccessor, recordDescProvider, this);
+ return new FeedIntakeOperatorNodePushable(ctx, feedId, adaptorFactory, partition, policyAccessor,
+ recordDescProvider, this);
}
- private IAdapterFactory createExtenralAdapterFactory(IHyracksTaskContext ctx, int partition) throws Exception {
+ private IAdapterFactory createExternalAdapterFactory(IHyracksTaskContext ctx, int partition) throws Exception {
IAdapterFactory adapterFactory = null;
ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(feedId.getDataverse(),
adaptorLibraryName);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
index cd20900..04ef016 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
@@ -18,30 +18,18 @@
*/
package org.apache.asterix.external.operators;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IAdapterRuntimeManager;
-import org.apache.asterix.external.api.IAdapterRuntimeManager.State;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedSubscriptionManager;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
-import org.apache.asterix.external.feed.api.ISubscriberRuntime;
import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.management.FeedId;
+import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.message.FeedPartitionStartMessage;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
-import org.apache.asterix.external.feed.runtime.CollectionRuntime;
+import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -55,176 +43,78 @@
*/
public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
- private static Logger LOGGER = Logger.getLogger(FeedIntakeOperatorNodePushable.class.getName());
-
private final FeedId feedId;
private final int partition;
- private final IFeedSubscriptionManager feedSubscriptionManager;
- private final IFeedManager feedManager;
private final IHyracksTaskContext ctx;
private final IAdapterFactory adapterFactory;
-
- private IngestionRuntime ingestionRuntime;
- private FeedAdapter adapter;
- private IIntakeProgressTracker tracker;
- private DistributeFeedFrameWriter feedFrameWriter;
-
private final FeedIntakeOperatorDescriptor opDesc;
- private final IRecordDescriptorProvider recordDescProvider;
-
public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IAdapterFactory adapterFactory,
- int partition, IngestionRuntime ingestionRuntime, FeedPolicyAccessor policyAccessor,
- IRecordDescriptorProvider recordDescProvider, FeedIntakeOperatorDescriptor feedIntakeOperatorDescriptor) {
+ int partition, FeedPolicyAccessor policyAccessor, IRecordDescriptorProvider recordDescProvider,
+ FeedIntakeOperatorDescriptor feedIntakeOperatorDescriptor) {
this.opDesc = feedIntakeOperatorDescriptor;
- this.recordDescProvider = recordDescProvider;
+ this.recordDesc = recordDescProvider.getOutputRecordDescriptor(opDesc.getActivityId(), 0);
this.ctx = ctx;
this.feedId = feedId;
this.partition = partition;
- this.ingestionRuntime = ingestionRuntime;
this.adapterFactory = adapterFactory;
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- this.feedManager = (IFeedManager) runtimeCtx.getFeedManager();
- this.feedSubscriptionManager = feedManager.getFeedSubscriptionManager();
}
@Override
public void initialize() throws HyracksDataException {
- IAdapterRuntimeManager adapterRuntimeManager = null;
+ FeedManager feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject()).getFeedManager();
+ AdapterRuntimeManager adapterRuntimeManager = null;
+ DistributeFeedFrameWriter frameDistributor = null;
+ IngestionRuntime ingestionRuntime = null;
+ boolean open = false;
try {
- if (ingestionRuntime == null) {
- try {
- adapter = (FeedAdapter) adapterFactory.createAdapter(ctx, partition);
- //TODO: Fix record tracking
- // if (adapterFactory.isRecordTrackingEnabled()) {
- // tracker = adapterFactory.createIntakeProgressTracker();
- // }
- } catch (Exception e) {
- LOGGER.severe("Unable to create adapter : " + adapterFactory.getAlias() + "[" + partition + "]"
- + " Exception " + e);
- throw new HyracksDataException(e);
- }
-
- recordDesc = recordDescProvider.getOutputRecordDescriptor(opDesc.getActivityId(), 0);
- FrameTupleAccessor fta = new FrameTupleAccessor(recordDesc);
- feedFrameWriter = new DistributeFeedFrameWriter(ctx, feedId, writer, FeedRuntimeType.INTAKE, partition,
- fta, feedManager);
- adapterRuntimeManager = new AdapterRuntimeManager(feedId, adapter, tracker, feedFrameWriter, partition);
- SubscribableFeedRuntimeId runtimeId = new SubscribableFeedRuntimeId(feedId, FeedRuntimeType.INTAKE,
- partition);
- ingestionRuntime = new IngestionRuntime(feedId, runtimeId, feedFrameWriter, recordDesc,
- adapterRuntimeManager, ctx);
- feedSubscriptionManager.registerFeedSubscribableRuntime(ingestionRuntime);
- // Notify FeedJobNotificationHandler that this provider is ready to receive subscription requests.
- ctx.sendApplicationMessageToCC(new FeedPartitionStartMessage(feedId, ctx.getJobletContext().getJobId()),
- null);
- feedFrameWriter.open();
- } else {
- if (ingestionRuntime.getAdapterRuntimeManager().getState().equals(State.INACTIVE_INGESTION)) {
- ingestionRuntime.getAdapterRuntimeManager().setState(State.ACTIVE_INGESTION);
- adapter = ingestionRuntime.getAdapterRuntimeManager().getFeedAdapter();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Switching to " + State.ACTIVE_INGESTION + " for ingestion runtime "
- + ingestionRuntime);
- LOGGER.info(" Adaptor " + adapter.getClass().getName() + "[" + partition + "]"
- + " connected to backend for feed " + feedId);
- }
- feedFrameWriter = ingestionRuntime.getFeedFrameWriter();
- } else {
- String message = "Feed Ingestion Runtime for feed " + feedId
- + " is already registered and is active!.";
- LOGGER.severe(message);
- throw new IllegalStateException(message);
+ Thread.currentThread().setName("Intake Thread");
+ // create the adapter
+ FeedAdapter adapter = (FeedAdapter) adapterFactory.createAdapter(ctx, partition);
+ // create the distributor
+ frameDistributor = new DistributeFeedFrameWriter(ctx, feedId, writer, FeedRuntimeType.INTAKE, partition,
+ new FrameTupleAccessor(recordDesc));
+ // create adapter runtime manager
+ adapterRuntimeManager = new AdapterRuntimeManager(feedId, adapter, frameDistributor, partition);
+ // create and register the runtime
+ FeedRuntimeId runtimeId =
+ new FeedRuntimeId(feedId, FeedRuntimeType.INTAKE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ingestionRuntime = new IngestionRuntime(feedId, runtimeId, frameDistributor, adapterRuntimeManager, ctx);
+ feedManager.registerFeedSubscribableRuntime(ingestionRuntime);
+ // Notify FeedJobNotificationHandler that this provider is ready to receive subscription requests.
+ ctx.sendApplicationMessageToCC(new FeedPartitionStartMessage(feedId, ctx.getJobletContext().getJobId()),
+ null);
+ // open the distributor
+ open = true;
+ frameDistributor.open();
+ // wait until ingestion is over
+ synchronized (adapterRuntimeManager) {
+ while (!adapterRuntimeManager.isDone()) {
+ adapterRuntimeManager.wait();
}
}
-
- waitTillIngestionIsOver(adapterRuntimeManager);
- feedSubscriptionManager
- .deregisterFeedSubscribableRuntime((SubscribableFeedRuntimeId) ingestionRuntime.getRuntimeId());
- if (adapterRuntimeManager.getState().equals(IAdapterRuntimeManager.State.FAILED_INGESTION)) {
+ // The ingestion is over. we need to remove the runtime from the manager
+ feedManager.deregisterFeedSubscribableRuntime(ingestionRuntime.getRuntimeId());
+ // If there was a failure, we need to throw an exception
+ if (adapterRuntimeManager.isFailed()) {
throw new HyracksDataException("Unable to ingest data");
}
-
- } catch (InterruptedException ie) {
+ } catch (Throwable ie) {
/*
* An Interrupted Exception is thrown if the Intake job cannot progress further due to failure of another node involved in the Hyracks job.
* As the Intake job involves only the intake operator, the exception is indicative of a failure at the sibling intake operator location.
* The surviving intake partitions must continue to live and receive data from the external source.
*/
- List<ISubscriberRuntime> subscribers = ingestionRuntime.getSubscribers();
- FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(new HashMap<String, String>());
- boolean needToHandleFailure = false;
- List<ISubscriberRuntime> failingSubscribers = new ArrayList<ISubscriberRuntime>();
- for (ISubscriberRuntime subscriber : subscribers) {
- policyAccessor.reset(subscriber.getFeedPolicy());
- if (!policyAccessor.continueOnHardwareFailure()) {
- failingSubscribers.add(subscriber);
- } else {
- needToHandleFailure = true;
- }
+ if (ingestionRuntime != null) {
+ ingestionRuntime.terminate();
+ feedManager.deregisterFeedSubscribableRuntime(ingestionRuntime.getRuntimeId());
}
-
- for (ISubscriberRuntime failingSubscriber : failingSubscribers) {
- try {
- ingestionRuntime.unsubscribeFeed((CollectionRuntime) failingSubscriber);
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning(
- "Excpetion in unsubscribing " + failingSubscriber + " message " + e.getMessage());
- }
- }
- }
-
- if (needToHandleFailure) {
- ingestionRuntime.getAdapterRuntimeManager().setState(State.INACTIVE_INGESTION);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Switching to " + State.INACTIVE_INGESTION + " on occurrence of failure.");
- }
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(
- "Interrupted Exception. None of the subscribers need to handle failures. Shutting down feed ingestion");
- }
- feedSubscriptionManager
- .deregisterFeedSubscribableRuntime((SubscribableFeedRuntimeId) ingestionRuntime.getRuntimeId());
- throw new HyracksDataException(ie);
- }
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
+ throw new HyracksDataException(ie);
} finally {
- if ((ingestionRuntime != null)
- && !ingestionRuntime.getAdapterRuntimeManager().getState().equals(State.INACTIVE_INGESTION)) {
- feedFrameWriter.close();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Closed Frame Writer " + feedFrameWriter + " adapter state "
- + ingestionRuntime.getAdapterRuntimeManager().getState());
- }
- } else {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ending intake operator node pushable in state " + State.INACTIVE_INGESTION
- + " Will resume after correcting failure");
- }
- }
-
- }
- }
-
- private void waitTillIngestionIsOver(IAdapterRuntimeManager adapterRuntimeManager) throws InterruptedException {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Waiting for adaptor [" + partition + "]" + "to be done with ingestion of feed " + feedId);
- }
- synchronized (adapterRuntimeManager) {
- while (!(adapterRuntimeManager.getState().equals(IAdapterRuntimeManager.State.FINISHED_INGESTION)
- || (adapterRuntimeManager.getState().equals(IAdapterRuntimeManager.State.FAILED_INGESTION)))) {
- adapterRuntimeManager.wait();
+ if (open) {
+ frameDistributor.close();
}
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Adaptor " + adapter.getClass().getName() + "[" + partition + "]"
- + " done with ingestion of feed " + feedId);
- }
}
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
index a92544b..82bf1da 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMessageOperatorNodePushable.java
@@ -18,42 +18,21 @@
*/
package org.apache.asterix.external.operators;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.api.IAdapterRuntimeManager;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.api.IFeedMessage;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.api.IIntakeProgressTracker;
import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
-import org.apache.asterix.external.feed.dataflow.FeedCollectRuntimeInputHandler;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector;
-import org.apache.asterix.external.feed.dataflow.FeedFrameCollector.State;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.management.FeedId;
-import org.apache.asterix.external.feed.management.FeedRuntimeManager;
+import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.message.EndFeedMessage;
-import org.apache.asterix.external.feed.message.FeedTupleCommitResponseMessage;
-import org.apache.asterix.external.feed.message.PrepareStallMessage;
-import org.apache.asterix.external.feed.message.TerminateDataFlowMessage;
-import org.apache.asterix.external.feed.message.ThrottlingEnabledFeedMessage;
+import org.apache.asterix.external.feed.runtime.AdapterRuntimeManager;
import org.apache.asterix.external.feed.runtime.CollectionRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.feed.runtime.IngestionRuntime;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
-import org.apache.asterix.external.feed.watch.IntakePartitionStatistics;
-import org.apache.asterix.external.feed.watch.MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask;
-import org.apache.asterix.external.feed.watch.StorageSideMonitoredBuffer;
-import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -71,7 +50,7 @@
private final FeedConnectionId connectionId;
private final IFeedMessage message;
- private final IFeedManager feedManager;
+ private final FeedManager feedManager;
private final int partition;
public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId connectionId,
@@ -79,9 +58,9 @@
this.connectionId = connectionId;
this.message = feedMessage;
this.partition = partition;
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- this.feedManager = (IFeedManager) runtimeCtx.getFeedManager();
+ IAsterixAppRuntimeContext runtimeCtx =
+ (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
+ this.feedManager = (FeedManager) runtimeCtx.getFeedManager();
}
@Override
@@ -100,28 +79,9 @@
break;
}
break;
- case PREPARE_STALL: {
- handlePrepareStallMessage((PrepareStallMessage) message);
- break;
- }
- case TERMINATE_FLOW: {
- FeedConnectionId connectionId = ((TerminateDataFlowMessage) message).getConnectionId();
- handleTerminateFlowMessage(connectionId);
- break;
- }
- case COMMIT_ACK_RESPONSE: {
- handleFeedTupleCommitResponseMessage((FeedTupleCommitResponseMessage) message);
- break;
- }
- case THROTTLING_ENABLED: {
- handleThrottlingEnabledMessage((ThrottlingEnabledFeedMessage) message);
- break;
- }
default:
break;
-
}
-
} catch (Exception e) {
throw new HyracksDataException(e);
} finally {
@@ -129,104 +89,12 @@
}
}
- private void handleThrottlingEnabledMessage(ThrottlingEnabledFeedMessage throttlingMessage) {
- FeedConnectionId connectionId = throttlingMessage.getConnectionId();
- FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
- Set<FeedRuntimeId> runtimes = runtimeManager.getFeedRuntimes();
- for (FeedRuntimeId runtimeId : runtimes) {
- if (runtimeId.getFeedRuntimeType().equals(FeedRuntimeType.STORE)) {
- FeedRuntime storeRuntime = runtimeManager.getFeedRuntime(runtimeId);
- ((StorageSideMonitoredBuffer) (storeRuntime.getInputHandler().getmBuffer())).setAcking(false);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Acking Disabled in view of throttling that has been activted upfron in the pipeline "
- + connectionId);
- }
- }
- }
- }
-
- private void handleFeedTupleCommitResponseMessage(FeedTupleCommitResponseMessage commitResponseMessage) {
- FeedConnectionId connectionId = commitResponseMessage.getConnectionId();
- FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
- Set<FeedRuntimeId> runtimes = runtimeManager.getFeedRuntimes();
- for (FeedRuntimeId runtimeId : runtimes) {
- FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
- switch (runtimeId.getFeedRuntimeType()) {
- case COLLECT:
- FeedCollectRuntimeInputHandler inputHandler = (FeedCollectRuntimeInputHandler) runtime
- .getInputHandler();
- int maxBasePersisted = commitResponseMessage.getMaxWindowAcked();
- inputHandler.dropTill(IntakePartitionStatistics.ACK_WINDOW_SIZE * (maxBasePersisted + 1));
- break;
- case STORE:
- MonitoredBufferStorageTimerTask sTask = runtime.getInputHandler().getmBuffer()
- .getStorageTimeTrackingRateTask();
- sTask.receiveCommitAckResponse(commitResponseMessage);
- break;
- default:
- break;
- }
- }
-
- SubscribableFeedRuntimeId sid = new SubscribableFeedRuntimeId(connectionId.getFeedId(), FeedRuntimeType.INTAKE,
- partition);
- IngestionRuntime ingestionRuntime = (IngestionRuntime) feedManager.getFeedSubscriptionManager()
- .getSubscribableRuntime(sid);
- if (ingestionRuntime != null) {
- IIntakeProgressTracker tracker = ingestionRuntime.getAdapterRuntimeManager().getProgressTracker();
- if (tracker != null) {
- tracker.notifyIngestedTupleTimestamp(System.currentTimeMillis());
- }
- }
- }
-
- private void handleTerminateFlowMessage(FeedConnectionId connectionId) throws HyracksDataException {
- FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
- Set<FeedRuntimeId> feedRuntimes = runtimeManager.getFeedRuntimes();
-
- boolean found = false;
- for (FeedRuntimeId runtimeId : feedRuntimes) {
- FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
- if (runtime.getRuntimeId().getRuntimeType().equals(FeedRuntimeType.COLLECT)) {
- ((CollectionRuntime) runtime).getFrameCollector().setState(State.HANDOVER);
- found = true;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Switched " + runtime + " to Hand Over stage");
- }
- }
- }
- if (!found) {
- throw new HyracksDataException("COLLECT Runtime not found!");
- }
- }
-
- private void handlePrepareStallMessage(PrepareStallMessage prepareStallMessage) throws HyracksDataException {
- FeedConnectionId connectionId = prepareStallMessage.getConnectionId();
- int computePartitionsRetainLimit = prepareStallMessage.getComputePartitionsRetainLimit();
- FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
- Set<FeedRuntimeId> feedRuntimes = runtimeManager.getFeedRuntimes();
- for (FeedRuntimeId runtimeId : feedRuntimes) {
- FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
- switch (runtimeId.getFeedRuntimeType()) {
- case COMPUTE:
- Mode requiredMode = runtimeId.getPartition() <= computePartitionsRetainLimit ? Mode.STALL
- : Mode.END;
- runtime.setMode(requiredMode);
- break;
- default:
- runtime.setMode(Mode.STALL);
- break;
- }
- }
- }
-
private void handleDiscontinueFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
FeedId sourceFeedId = endFeedMessage.getSourceFeedId();
- SubscribableFeedRuntimeId subscribableRuntimeId = new SubscribableFeedRuntimeId(sourceFeedId,
- FeedRuntimeType.INTAKE, partition);
- ISubscribableRuntime feedRuntime = feedManager.getFeedSubscriptionManager()
- .getSubscribableRuntime(subscribableRuntimeId);
- IAdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
+ FeedRuntimeId subscribableRuntimeId =
+ new FeedRuntimeId(sourceFeedId, FeedRuntimeType.INTAKE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ISubscribableRuntime feedRuntime = feedManager.getSubscribableRuntime(subscribableRuntimeId);
+ AdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
adapterRuntimeManager.stop();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Stopped Adapter " + adapterRuntimeManager);
@@ -254,11 +122,12 @@
throw new IllegalStateException("Invalid subscribable runtime type " + subscribableRuntimeType);
}
- runtimeId = new FeedRuntimeId(runtimeType, partition, FeedRuntimeId.DEFAULT_OPERAND_ID);
- CollectionRuntime feedRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager()
- .getFeedRuntime(connectionId, runtimeId);
+ runtimeId = new FeedRuntimeId(endFeedMessage.getSourceFeedId(), runtimeType, partition,
+ FeedRuntimeId.DEFAULT_TARGET_ID);
+ CollectionRuntime feedRuntime =
+ (CollectionRuntime) feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
if (feedRuntime != null) {
- feedRuntime.getSourceRuntime().unsubscribeFeed(feedRuntime);
+ feedRuntime.getSourceRuntime().unsubscribe(feedRuntime);
}
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Complete Unsubscription of " + endFeedMessage.getFeedConnectionId());
@@ -273,26 +142,12 @@
throw new IllegalStateException("Illegal State, invalid runtime type " + subscribableRuntimeType);
case COMPUTE:
// feed could be primary or secondary, doesn't matter
- SubscribableFeedRuntimeId feedSubscribableRuntimeId = new SubscribableFeedRuntimeId(
- connectionId.getFeedId(), FeedRuntimeType.COMPUTE, partition);
- ISubscribableRuntime feedRuntime = feedManager.getFeedSubscriptionManager()
- .getSubscribableRuntime(feedSubscribableRuntimeId);
- DistributeFeedFrameWriter dWriter = feedRuntime.getFeedFrameWriter();
- Map<IFrameWriter, FeedFrameCollector> registeredCollectors = dWriter.getRegisteredReaders();
-
- IFrameWriter unsubscribingWriter = null;
- for (Entry<IFrameWriter, FeedFrameCollector> entry : registeredCollectors.entrySet()) {
- IFrameWriter frameWriter = entry.getKey();
- FeedRuntimeInputHandler feedFrameWriter = (FeedRuntimeInputHandler) frameWriter;
- if (feedFrameWriter.getConnectionId().equals(endFeedMessage.getFeedConnectionId())) {
- unsubscribingWriter = feedFrameWriter;
- dWriter.unsubscribeFeed(unsubscribingWriter);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Partial Unsubscription of " + unsubscribingWriter);
- }
- break;
- }
- }
+ FeedRuntimeId feedSubscribableRuntimeId = new FeedRuntimeId(connectionId.getFeedId(),
+ FeedRuntimeType.COMPUTE, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
+ ISubscribableRuntime feedRuntime = feedManager.getSubscribableRuntime(feedSubscribableRuntimeId);
+ CollectionRuntime feedCollectionRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager()
+ .getFeedRuntime(connectionId, runtimeId);
+ feedRuntime.unsubscribe(feedCollectionRuntime);
break;
default:
break;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
index 6c9ef8d..716468e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaComputeNodePushable.java
@@ -24,18 +24,15 @@
import java.util.logging.Logger;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.api.ISubscribableRuntime;
-import org.apache.asterix.external.feed.dataflow.DistributeFeedFrameWriter;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
+import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.feed.management.FeedManager;
+import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
import org.apache.asterix.external.feed.runtime.FeedRuntime;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.asterix.external.feed.runtime.SubscribableFeedRuntimeId;
-import org.apache.asterix.external.feed.runtime.SubscribableRuntime;
import org.apache.asterix.external.util.FeedUtils;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IActivity;
@@ -80,10 +77,8 @@
**/
private int partition;
- private int nPartitions;
-
/** The (singleton) instance of IFeedManager **/
- private IFeedManager feedManager;
+ private FeedManager feedManager;
private FrameTupleAccessor fta;
@@ -91,14 +86,19 @@
private final FeedRuntimeType runtimeType = FeedRuntimeType.COMPUTE;
- private FeedRuntimeInputHandler inputSideHandler;
-
private ByteBuffer message = ByteBuffer.allocate(MessagingFrameTupleAppender.MAX_MESSAGE_SIZE);
private final FeedMetaOperatorDescriptor opDesc;
private final IRecordDescriptorProvider recordDescProvider;
+ private boolean opened;
+
+ /*
+ * In this operator:
+ * writer is the network partitioner
+ * coreOperator is the first operator
+ */
public FeedMetaComputeNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
Map<String, String> feedPolicyProperties, String operationId,
@@ -108,9 +108,8 @@
.createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
this.partition = partition;
- this.nPartitions = nPartitions;
this.connectionId = feedConnectionId;
- this.feedManager = (IFeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
ctx.setSharedObject(message);
this.opDesc = feedMetaOperatorDescriptor;
@@ -119,16 +118,12 @@
@Override
public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId = new SubscribableFeedRuntimeId(connectionId.getFeedId(), runtimeType, partition);
+ FeedRuntimeId runtimeId =
+ new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, FeedRuntimeId.DEFAULT_TARGET_ID);
try {
- feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
- if (feedRuntime == null) {
- initializeNewFeedRuntime(runtimeId);
- } else {
- reviveOldFeedRuntime(runtimeId);
- }
+ initializeNewFeedRuntime(runtimeId);
+ opened = true;
writer.open();
- coreOperator.open();
} catch (Exception e) {
e.printStackTrace();
throw new HyracksDataException(e);
@@ -136,113 +131,52 @@
}
private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- this.fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
- this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator,
- policyEnforcer.getFeedPolicyAccessor(), policyEnforcer.getFeedPolicyAccessor().bufferingEnabled(), fta,
- recordDesc, feedManager, nPartitions);
-
- DistributeFeedFrameWriter distributeWriter = new DistributeFeedFrameWriter(ctx, connectionId.getFeedId(),
- writer, runtimeType, partition, new FrameTupleAccessor(recordDesc), feedManager);
- coreOperator.setOutputFrameWriter(0, distributeWriter, recordDesc);
-
- feedRuntime = new SubscribableRuntime(connectionId.getFeedId(), runtimeId, inputSideHandler, distributeWriter,
- recordDesc);
- feedManager.getFeedSubscriptionManager().registerFeedSubscribableRuntime((ISubscribableRuntime) feedRuntime);
+ fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
+ FeedPolicyAccessor fpa = policyEnforcer.getFeedPolicyAccessor();
+ coreOperator.setOutputFrameWriter(0, writer, recordDesc);
+ if (fpa.bufferingEnabled()) {
+ writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator, fpa, fta,
+ feedManager.getFeedMemoryManager());
+ } else {
+ writer = new SyncFeedRuntimeInputHandler(ctx, coreOperator, fta);
+ }
+ feedRuntime = new FeedRuntime(runtimeId);
feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, feedRuntime);
-
- distributeWriter.subscribeFeed(policyEnforcer.getFeedPolicyAccessor(), writer, connectionId);
- }
-
- private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- this.fta = new FrameTupleAccessor(recordDesc);
- this.inputSideHandler = feedRuntime.getInputHandler();
- this.inputSideHandler.setCoreOperator(coreOperator);
-
- DistributeFeedFrameWriter distributeWriter = new DistributeFeedFrameWriter(ctx, connectionId.getFeedId(),
- writer, runtimeType, partition, new FrameTupleAccessor(recordDesc), feedManager);
- coreOperator.setOutputFrameWriter(0, distributeWriter, recordDesc);
- distributeWriter.subscribeFeed(policyEnforcer.getFeedPolicyAccessor(), writer, connectionId);
-
- inputSideHandler.reset(nPartitions);
- feedRuntime.setMode(Mode.PROCESS);
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
try {
FeedUtils.processFeedMessage(buffer, message, fta);
- inputSideHandler.nextFrame(buffer);
+ writer.nextFrame(buffer);
} catch (Exception e) {
- e.printStackTrace();
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
throw new HyracksDataException(e);
}
}
@Override
public void fail() throws HyracksDataException {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Core Op:" + coreOperator.getDisplayName() + " fail ");
- }
- feedRuntime.setMode(Mode.FAIL);
- coreOperator.fail();
+ writer.fail();
}
@Override
public void close() throws HyracksDataException {
- boolean stalled = inputSideHandler.getMode().equals(Mode.STALL);
- boolean end = inputSideHandler.getMode().equals(Mode.END);
try {
- if (!(stalled || end)) {
- inputSideHandler.nextFrame(null); // signal end of data
- while (!inputSideHandler.isFinished()) {
- synchronized (coreOperator) {
- if (inputSideHandler.isFinished()) {
- break;
- }
- coreOperator.wait();
- }
- }
- } else {
- inputSideHandler.setFinished(true);
- }
- coreOperator.close();
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("CLOSED " + coreOperator + " STALLED ?" + stalled + " ENDED " + end);
- }
- } catch (InterruptedException e) {
- throw new HyracksDataException(e);
+ deregister();
} finally {
- if (!stalled) {
- deregister();
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("DEREGISTERING " + this.feedRuntime.getRuntimeId());
- }
- } else {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("NOT DEREGISTERING " + this.feedRuntime.getRuntimeId());
- }
- }
- inputSideHandler.close();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
+ if (opened) {
+ writer.close();
}
}
}
private void deregister() {
- if (feedRuntime != null) {
- // deregister from subscription manager
- SubscribableFeedRuntimeId runtimeId = (SubscribableFeedRuntimeId) feedRuntime.getRuntimeId();
- feedManager.getFeedSubscriptionManager().deregisterFeedSubscribableRuntime(runtimeId);
-
- // deregister from connection manager
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
- }
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
}
@Override
public void flush() throws HyracksDataException {
- inputSideHandler.flush();
+ writer.flush();
}
-
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
deleted file mode 100644
index e99ae3a..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaNodePushable.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.asterix.external.operators;
-
-import java.nio.ByteBuffer;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
-import org.apache.asterix.external.feed.api.IFeedManager;
-import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
-import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
-import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
-import org.apache.asterix.external.feed.runtime.FeedRuntime;
-import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.IActivity;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-
-public class FeedMetaNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-
- private static final Logger LOGGER = Logger.getLogger(FeedMetaNodePushable.class.getName());
-
- /** Runtime node pushable corresponding to the core feed operator **/
- private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperator;
-
- /**
- * A policy enforcer that ensures dyanmic decisions for a feed are taken
- * in accordance with the associated ingestion policy
- **/
- private FeedPolicyEnforcer policyEnforcer;
-
- /**
- * The Feed Runtime instance associated with the operator. Feed Runtime
- * captures the state of the operator while the feed is active.
- */
- private FeedRuntime feedRuntime;
-
- /**
- * A unique identifier for the feed instance. A feed instance represents
- * the flow of data from a feed to a dataset.
- **/
- private FeedConnectionId connectionId;
-
- /**
- * Denotes the i'th operator instance in a setting where K operator
- * instances are scheduled to run in parallel
- **/
- private int partition;
-
- /** Total number of partitions available **/
- private int nPartitions;
-
- /** Type associated with the core feed operator **/
- private final FeedRuntimeType runtimeType = FeedRuntimeType.OTHER;
-
- /** The (singleton) instance of IFeedManager **/
- private IFeedManager feedManager;
-
- private FrameTupleAccessor fta;
-
- private final IHyracksTaskContext ctx;
-
- private final String operandId;
-
- /** The pre-processor associated with this runtime **/
- private FeedRuntimeInputHandler inputSideHandler;
-
- public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
- Map<String, String> feedPolicyProperties, String operationId) throws HyracksDataException {
- this.ctx = ctx;
- this.coreOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
- .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
- this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
- this.partition = partition;
- this.nPartitions = nPartitions;
- this.connectionId = feedConnectionId;
- this.feedManager = (IFeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
- .getApplicationObject()).getFeedManager();
- this.operandId = operationId;
- }
-
- @Override
- public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
- try {
- feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
- if (feedRuntime == null) {
- initializeNewFeedRuntime(runtimeId);
- } else {
- reviveOldFeedRuntime(runtimeId);
- }
- coreOperator.open();
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
- }
-
- private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- this.fta = new FrameTupleAccessor(recordDesc);
- this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator,
- policyEnforcer.getFeedPolicyAccessor(), false, fta, recordDesc, feedManager, nPartitions);
-
- setupBasicRuntime(inputSideHandler);
- }
-
- private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- this.inputSideHandler = feedRuntime.getInputHandler();
- this.fta = new FrameTupleAccessor(recordDesc);
- coreOperator.setOutputFrameWriter(0, writer, recordDesc);
- feedRuntime.setMode(Mode.PROCESS);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Retreived state from the zombie instance " + runtimeType + " node.");
- }
- }
-
- private void setupBasicRuntime(FeedRuntimeInputHandler inputHandler) throws Exception {
- coreOperator.setOutputFrameWriter(0, writer, recordDesc);
- FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
- feedRuntime = new FeedRuntime(runtimeId, inputHandler, writer);
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- try {
- inputSideHandler.nextFrame(buffer);
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.info("Core Op:" + coreOperator.getDisplayName() + " fail ");
- }
- feedRuntime.setMode(Mode.FAIL);
- coreOperator.fail();
- }
-
- @Override
- public void close() throws HyracksDataException {
- try {
- coreOperator.close();
- } catch (Exception e) {
- e.printStackTrace();
- // ignore
- } finally {
- if (inputSideHandler != null) {
- inputSideHandler.close();
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
- }
- }
- }
-
- @Override
- public void flush() throws HyracksDataException {
- inputSideHandler.flush();
- }
-
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
index 6591795..c7dd3d2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaOperatorDescriptor.java
@@ -22,7 +22,6 @@
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
@@ -101,16 +100,6 @@
nodePushable = new FeedMetaStoreNodePushable(ctx, recordDescProvider, partition, nPartitions,
coreOperator, feedConnectionId, feedPolicyProperties, operandId, this);
break;
- case OTHER:
- nodePushable = new FeedMetaNodePushable(ctx, recordDescProvider, partition, nPartitions, coreOperator,
- feedConnectionId, feedPolicyProperties, operandId);
- break;
- case ETS:
- nodePushable = ((AlgebricksMetaOperatorDescriptor) coreOperator).createPushRuntime(ctx,
- recordDescProvider, partition, nPartitions);
- break;
- case JOIN:
- break;
default:
throw new HyracksDataException(new IllegalArgumentException("Invalid feed runtime: " + runtimeType));
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
index f11b948..b79707b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedMetaStoreNodePushable.java
@@ -25,15 +25,16 @@
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.dataflow.AsterixLSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.external.feed.api.IFeedManager;
import org.apache.asterix.external.feed.api.IFeedRuntime.FeedRuntimeType;
-import org.apache.asterix.external.feed.api.IFeedRuntime.Mode;
import org.apache.asterix.external.feed.dataflow.FeedRuntimeInputHandler;
+import org.apache.asterix.external.feed.dataflow.SyncFeedRuntimeInputHandler;
import org.apache.asterix.external.feed.management.FeedConnectionId;
+import org.apache.asterix.external.feed.management.FeedManager;
import org.apache.asterix.external.feed.policy.FeedPolicyEnforcer;
import org.apache.asterix.external.feed.runtime.FeedRuntime;
import org.apache.asterix.external.feed.runtime.FeedRuntimeId;
import org.apache.asterix.external.util.FeedUtils;
+import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IActivity;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
@@ -48,7 +49,7 @@
private static final Logger LOGGER = Logger.getLogger(FeedMetaStoreNodePushable.class.getName());
/** Runtime node pushable corresponding to the core feed operator **/
- private final AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperator;
+ private AbstractUnaryInputUnaryOutputOperatorNodePushable insertOperator;
/**
* A policy enforcer that ensures dyanmic decisions for a feed are taken
@@ -74,21 +75,17 @@
**/
private final int partition;
- private final int nPartitions;
-
/** Type associated with the core feed operator **/
private final FeedRuntimeType runtimeType = FeedRuntimeType.STORE;
/** The (singleton) instance of IFeedManager **/
- private final IFeedManager feedManager;
+ private final FeedManager feedManager;
private FrameTupleAccessor fta;
private final IHyracksTaskContext ctx;
- private final String operandId;
-
- private FeedRuntimeInputHandler inputSideHandler;
+ private final String targetId;
private final ByteBuffer message = ByteBuffer.allocate(MessagingFrameTupleAppender.MAX_MESSAGE_SIZE);
@@ -98,18 +95,17 @@
public FeedMetaStoreNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
- Map<String, String> feedPolicyProperties, String operationId,
+ Map<String, String> feedPolicyProperties, String targetId,
FeedMetaOperatorDescriptor feedMetaOperatorDescriptor) throws HyracksDataException {
this.ctx = ctx;
- this.coreOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+ this.insertOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
.createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
this.partition = partition;
- this.nPartitions = nPartitions;
this.connectionId = feedConnectionId;
- this.feedManager = (IFeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+ this.feedManager = (FeedManager) ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
.getApplicationObject()).getFeedManager();
- this.operandId = operationId;
+ this.targetId = targetId;
ctx.setSharedObject(message);
this.recordDescProvider = recordDescProvider;
this.opDesc = feedMetaOperatorDescriptor;
@@ -117,15 +113,10 @@
@Override
public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
+ FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, targetId);
try {
- feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
- if (feedRuntime == null) {
- initializeNewFeedRuntime(runtimeId);
- } else {
- reviveOldFeedRuntime(runtimeId);
- }
- coreOperator.open();
+ initializeNewFeedRuntime(runtimeId);
+ insertOperator.open();
} catch (Exception e) {
LOGGER.log(Level.WARNING, "Failed to open feed store operator", e);
throw new HyracksDataException(e);
@@ -133,39 +124,29 @@
}
private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Runtime not found for " + runtimeId + " connection id " + connectionId);
- }
- this.fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
- this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator,
- policyEnforcer.getFeedPolicyAccessor(), policyEnforcer.getFeedPolicyAccessor().bufferingEnabled(), fta,
- recordDesc, feedManager, nPartitions);
- if (coreOperator instanceof AsterixLSMInsertDeleteOperatorNodePushable) {
- AsterixLSMInsertDeleteOperatorNodePushable indexOp = (AsterixLSMInsertDeleteOperatorNodePushable) coreOperator;
+ fta = new FrameTupleAccessor(recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0));
+ insertOperator.setOutputFrameWriter(0, writer, recordDesc);
+ if (insertOperator instanceof AsterixLSMInsertDeleteOperatorNodePushable) {
+ AsterixLSMInsertDeleteOperatorNodePushable indexOp =
+ (AsterixLSMInsertDeleteOperatorNodePushable) insertOperator;
if (!indexOp.isPrimary()) {
- inputSideHandler.setBufferingEnabled(false);
+ writer = insertOperator;
+ setupBasicRuntime(writer);
+ return;
}
}
- setupBasicRuntime(inputSideHandler);
- }
-
- private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
- this.inputSideHandler = feedRuntime.getInputHandler();
- this.fta = new FrameTupleAccessor(recordDesc);
- coreOperator.setOutputFrameWriter(0, writer, recordDesc);
- this.inputSideHandler.reset(nPartitions);
- this.inputSideHandler.setCoreOperator(coreOperator);
- feedRuntime.setMode(Mode.PROCESS);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning(
- "Retreived state from the zombie instance from previous execution for " + runtimeType + " node.");
+ if (policyEnforcer.getFeedPolicyAccessor().bufferingEnabled()) {
+ writer = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, insertOperator,
+ policyEnforcer.getFeedPolicyAccessor(), fta, feedManager.getFeedMemoryManager());
+ } else {
+ writer = new SyncFeedRuntimeInputHandler(ctx, insertOperator, fta);
}
+ setupBasicRuntime(writer);
}
- private void setupBasicRuntime(FeedRuntimeInputHandler inputHandler) throws Exception {
- coreOperator.setOutputFrameWriter(0, writer, recordDesc);
- FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
- feedRuntime = new FeedRuntime(runtimeId, inputHandler, writer);
+ private void setupBasicRuntime(IFrameWriter frameWriter) throws Exception {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(connectionId.getFeedId(), runtimeType, partition, targetId);
+ feedRuntime = new FeedRuntime(runtimeId);
feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, feedRuntime);
}
@@ -173,7 +154,7 @@
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
try {
FeedUtils.processFeedMessage(buffer, message, fta);
- inputSideHandler.nextFrame(buffer);
+ writer.nextFrame(buffer);
} catch (Exception e) {
e.printStackTrace();
throw new HyracksDataException(e);
@@ -182,68 +163,25 @@
@Override
public void fail() throws HyracksDataException {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.info("Core Op:" + coreOperator.getDisplayName() + " fail ");
- }
- feedRuntime.setMode(Mode.FAIL);
- coreOperator.fail();
+ writer.fail();
}
@Override
public void close() throws HyracksDataException {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("CLOSE CALLED FOR " + this.feedRuntime.getRuntimeId());
- }
- boolean stalled = inputSideHandler.getMode().equals(Mode.STALL);
try {
- if (!stalled) {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("SIGNALLING END OF DATA for " + this.feedRuntime.getRuntimeId() + " mode is "
- + inputSideHandler.getMode() + " WAITING ON " + coreOperator);
- }
- inputSideHandler.nextFrame(null); // signal end of data
- while (!inputSideHandler.isFinished()) {
- synchronized (coreOperator) {
- if (inputSideHandler.isFinished()) {
- break;
- }
- coreOperator.wait();
- }
- }
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("ABOUT TO CLOSE OPERATOR " + coreOperator);
- }
- }
- coreOperator.close();
- } catch (Exception e) {
- throw new HyracksDataException(e);
+ writer.close();
} finally {
- if (!stalled) {
- deregister();
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("DEREGISTERING " + this.feedRuntime.getRuntimeId());
- }
- } else {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("NOT DEREGISTERING " + this.feedRuntime.getRuntimeId());
- }
- }
- inputSideHandler.close();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
- }
+ deregister();
}
}
private void deregister() {
- if (feedRuntime != null) {
- feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
- }
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, feedRuntime.getRuntimeId());
}
@Override
public void flush() throws HyracksDataException {
- inputSideHandler.flush();
+ writer.flush();
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java
index f16e24b..d986648 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataExceptionUtils.java
@@ -43,7 +43,7 @@
return true;
}
- public static HyracksDataException suppress(HyracksDataException hde, Throwable th) {
+ public static HyracksDataException suppressIntoHyracksDataException(HyracksDataException hde, Throwable th) {
if (hde == null) {
return new HyracksDataException(th);
} else {
@@ -51,4 +51,20 @@
return hde;
}
}
+
+ public static Throwable suppress(Throwable suppressor, Throwable suppressed) {
+ if (suppressor == null) {
+ return suppressed;
+ } else if (suppressed != null) {
+ suppressor.addSuppressed(suppressed);
+ }
+ return suppressor;
+ }
+
+ public static HyracksDataException convertToHyracksDataException(Throwable throwable) {
+ if (throwable == null || throwable instanceof HyracksDataException) {
+ return (HyracksDataException) throwable;
+ }
+ return new HyracksDataException(throwable);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedMemoryManagerUnitTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedMemoryManagerUnitTest.java
new file mode 100644
index 0000000..8a6d1b6
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedMemoryManagerUnitTest.java
@@ -0,0 +1,482 @@
+/*
+ * 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.asterix.external.feed.test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.asterix.common.config.AsterixFeedProperties;
+import org.apache.asterix.external.feed.management.ConcurrentFramePool;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.mockito.Mockito;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import junit.framework.Test;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+
+@RunWith(PowerMockRunner.class)
+public class FeedMemoryManagerUnitTest extends TestCase {
+
+ private static final int DEFAULT_FRAME_SIZE = 32768;
+ private static final int NUM_FRAMES = 2048;
+ private static final long FEED_MEM_BUDGET = DEFAULT_FRAME_SIZE * NUM_FRAMES;
+ private static final int NUM_THREADS = 8;
+ private static final int MAX_SIZE = 52;
+ private static final double RELEASE_PROBABILITY = 0.20;
+
+ public FeedMemoryManagerUnitTest(String testName) {
+ super(testName);
+ }
+
+ /**
+ * @return the suite of tests being tested
+ */
+ public static Test suite() {
+ return new TestSuite(FeedMemoryManagerUnitTest.class);
+ }
+
+ @org.junit.Test
+ public void testMemoryManager() {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ int i = 0;
+ while (fmm.get() != null) {
+ i++;
+ }
+ Assert.assertEquals(i, NUM_FRAMES);
+ }
+
+ @org.junit.Test
+ public void testConcurrentMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ FixedSizeAllocator[] runners = new FixedSizeAllocator[NUM_THREADS];
+ Thread[] threads = new Thread[NUM_THREADS];
+ Arrays.parallelSetAll(runners, (int i) -> new FixedSizeAllocator(fmm));
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(runners[i]);
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+ int i = 0;
+ for (FixedSizeAllocator allocator : runners) {
+ i += allocator.getAllocated();
+ }
+ Assert.assertEquals(NUM_FRAMES, i);
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ @org.junit.Test
+ public void testVarSizeMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ Random random = new Random();
+ int i = 0;
+ int req;
+ while (true) {
+ req = random.nextInt(MAX_SIZE) + 1;
+ if (req == 1) {
+ if (fmm.get() != null) {
+ i += 1;
+ } else {
+ break;
+ }
+ } else if (fmm.get(req * DEFAULT_FRAME_SIZE) != null) {
+ i += req;
+ } else {
+ break;
+ }
+ }
+
+ Assert.assertEquals(i <= NUM_FRAMES, true);
+ Assert.assertEquals(i + req > NUM_FRAMES, true);
+ Assert.assertEquals(i + fmm.remaining(), NUM_FRAMES);
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ @org.junit.Test
+ public void testConcurrentVarSizeMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+
+ VarSizeAllocator[] runners = new VarSizeAllocator[NUM_THREADS];
+ Thread[] threads = new Thread[NUM_THREADS];
+ Arrays.parallelSetAll(runners, (int i) -> new VarSizeAllocator(fmm));
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(runners[i]);
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+ int allocated = 0;
+ for (int i = 0; i < threads.length; i++) {
+ if (runners[i].cause() != null) {
+ runners[i].cause().printStackTrace();
+ Assert.fail(runners[i].cause().getMessage());
+ }
+ allocated += runners[i].getAllocated();
+ }
+ Assert.assertEquals(allocated <= NUM_FRAMES, true);
+ for (int i = 0; i < threads.length; i++) {
+ Assert.assertEquals(allocated + runners[i].getLastReq() > NUM_FRAMES, true);
+ }
+ Assert.assertEquals(allocated + fmm.remaining(), NUM_FRAMES);
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ @org.junit.Test
+ public void testAcquireReleaseMemoryManager() throws HyracksDataException {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ Random random = new Random();
+ ArrayDeque<ByteBuffer> stack = new ArrayDeque<>();
+ while (true) {
+ if (random.nextDouble() < RELEASE_PROBABILITY) {
+ if (!stack.isEmpty()) {
+ fmm.release(stack.pop());
+ }
+ } else {
+ ByteBuffer buffer = fmm.get();
+ if (buffer == null) {
+ break;
+ } else {
+ stack.push(buffer);
+ }
+ }
+ }
+ Assert.assertEquals(stack.size(), NUM_FRAMES);
+ Assert.assertEquals(fmm.remaining(), 0);
+ for (ByteBuffer buffer : stack) {
+ fmm.release(buffer);
+ }
+ stack.clear();
+ Assert.assertEquals(fmm.remaining(), NUM_FRAMES);
+ }
+
+ @org.junit.Test
+ public void testConcurrentAcquireReleaseMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ FixedSizeGoodAllocator[] runners = new FixedSizeGoodAllocator[NUM_THREADS];
+ Thread[] threads = new Thread[NUM_THREADS];
+ Arrays.parallelSetAll(runners, (int i) -> new FixedSizeGoodAllocator(fmm));
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(runners[i]);
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+ int i = 0;
+ for (FixedSizeGoodAllocator allocator : runners) {
+ i += allocator.getAllocated();
+ }
+ Assert.assertEquals(NUM_FRAMES, i);
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ @org.junit.Test
+ public void testAcquireReleaseVarSizeMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ Random random = new Random();
+ ArrayDeque<ByteBuffer> stack = new ArrayDeque<>();
+ int i = 0;
+ int req;
+ while (true) {
+ // release
+ if (random.nextDouble() < RELEASE_PROBABILITY) {
+ if (!stack.isEmpty()) {
+ ByteBuffer buffer = stack.pop();
+ i -= (buffer.capacity() / DEFAULT_FRAME_SIZE);
+ fmm.release(buffer);
+ }
+ } else {
+ // acquire
+ req = random.nextInt(MAX_SIZE) + 1;
+ if (req == 1) {
+ ByteBuffer buffer = fmm.get();
+ if (buffer != null) {
+ stack.push(buffer);
+ i += 1;
+ } else {
+ break;
+ }
+ } else {
+ ByteBuffer buffer = fmm.get(req * DEFAULT_FRAME_SIZE);
+ if (buffer != null) {
+ stack.push(buffer);
+ i += req;
+ } else {
+ break;
+ }
+ }
+ }
+ }
+
+ Assert.assertEquals(i <= NUM_FRAMES, true);
+ Assert.assertEquals(i + req > NUM_FRAMES, true);
+ Assert.assertEquals(i + fmm.remaining(), NUM_FRAMES);
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ @org.junit.Test
+ public void testConcurrentAcquireReleaseVarSizeMemoryManager() {
+ try {
+ AsterixFeedProperties afp = Mockito.mock(AsterixFeedProperties.class);
+ Mockito.when(afp.getMemoryComponentGlobalBudget()).thenReturn(FEED_MEM_BUDGET);
+ ConcurrentFramePool fmm =
+ new ConcurrentFramePool("TestNode", afp.getMemoryComponentGlobalBudget(), DEFAULT_FRAME_SIZE);
+ VarSizeGoodAllocator[] runners = new VarSizeGoodAllocator[NUM_THREADS];
+ Thread[] threads = new Thread[NUM_THREADS];
+ Arrays.parallelSetAll(runners, (int i) -> new VarSizeGoodAllocator(fmm));
+ for (int i = 0; i < threads.length; i++) {
+ threads[i] = new Thread(runners[i]);
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].start();
+ }
+ for (int i = 0; i < threads.length; i++) {
+ threads[i].join();
+ }
+ int i = 0;
+ for (VarSizeGoodAllocator allocator : runners) {
+ if (allocator.cause() != null) {
+ allocator.cause().printStackTrace();
+ Assert.fail(allocator.cause().getMessage());
+ }
+ i += allocator.getAllocated();
+ }
+ Assert.assertEquals(NUM_FRAMES, i + fmm.remaining());
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ /*
+ * Runnables used for unit tests
+ */
+ private class FixedSizeAllocator implements Runnable {
+ private final ConcurrentFramePool fmm;
+ private int allocated = 0;
+
+ public FixedSizeAllocator(ConcurrentFramePool fmm) {
+ this.fmm = fmm;
+ }
+
+ public int getAllocated() {
+ return allocated;
+ }
+
+ @Override
+ public void run() {
+ while (fmm.get() != null) {
+ allocated++;
+ }
+ }
+ }
+
+ private class FixedSizeGoodAllocator implements Runnable {
+ private final ConcurrentFramePool fmm;
+ private final ArrayDeque<ByteBuffer> stack = new ArrayDeque<>();
+ private final Random random = new Random();
+
+ public FixedSizeGoodAllocator(ConcurrentFramePool fmm) {
+ this.fmm = fmm;
+ }
+
+ public int getAllocated() {
+ return stack.size();
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ if (random.nextDouble() < RELEASE_PROBABILITY) {
+ if (!stack.isEmpty()) {
+ try {
+ fmm.release(stack.pop());
+ } catch (HyracksDataException e) {
+ Assert.fail();
+ }
+ }
+ } else {
+ ByteBuffer buffer = fmm.get();
+ if (buffer == null) {
+ break;
+ } else {
+ stack.push(buffer);
+ }
+ }
+ }
+ }
+ }
+
+ private class VarSizeGoodAllocator implements Runnable {
+ private final ConcurrentFramePool fmm;
+ private int allocated = 0;
+ private int req = 0;
+ private final Random random = new Random();
+ private Throwable cause;
+ private final ArrayDeque<ByteBuffer> stack = new ArrayDeque<>();
+
+ public VarSizeGoodAllocator(ConcurrentFramePool fmm) {
+ this.fmm = fmm;
+ }
+
+ public int getAllocated() {
+ return allocated;
+ }
+
+ public Throwable cause() {
+ return cause;
+ }
+
+ @Override
+ public void run() {
+ try {
+ while (true) {
+ if (random.nextDouble() < RELEASE_PROBABILITY) {
+ if (!stack.isEmpty()) {
+ ByteBuffer buffer = stack.pop();
+ allocated -= (buffer.capacity() / DEFAULT_FRAME_SIZE);
+ fmm.release(buffer);
+ }
+ } else {
+ req = random.nextInt(MAX_SIZE) + 1;
+ if (req == 1) {
+ ByteBuffer buffer = fmm.get();
+ if (buffer != null) {
+ stack.push(buffer);
+ allocated += 1;
+ } else {
+ break;
+ }
+ } else {
+ ByteBuffer buffer = fmm.get(req * DEFAULT_FRAME_SIZE);
+ if (buffer != null) {
+ stack.push(buffer);
+ allocated += req;
+ } else {
+ break;
+ }
+ }
+ }
+ }
+ } catch (Throwable th) {
+ this.cause = th;
+ }
+ }
+ }
+
+ private class VarSizeAllocator implements Runnable {
+ private final ConcurrentFramePool fmm;
+ private int allocated = 0;
+ private int req = 0;
+ private final Random random = new Random();
+ private Throwable cause;
+
+ public VarSizeAllocator(ConcurrentFramePool fmm) {
+ this.fmm = fmm;
+ }
+
+ public int getAllocated() {
+ return allocated;
+ }
+
+ public int getLastReq() {
+ return req;
+ }
+
+ public Throwable cause() {
+ return cause;
+ }
+
+ @Override
+ public void run() {
+ try {
+ while (true) {
+ req = random.nextInt(MAX_SIZE) + 1;
+ if (req == 1) {
+ if (fmm.get() != null) {
+ allocated += 1;
+ } else {
+ break;
+ }
+ } else if (fmm.get(req * DEFAULT_FRAME_SIZE) != null) {
+ allocated += req;
+ } else {
+ break;
+ }
+ }
+ } catch (Throwable th) {
+ this.cause = th;
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedSpillerUnitTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedSpillerUnitTest.java
new file mode 100644
index 0000000..8751439
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/feed/test/FeedSpillerUnitTest.java
@@ -0,0 +1,178 @@
+/*
+ * 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.asterix.external.feed.test;
+
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import org.apache.asterix.external.feed.dataflow.FrameSpiller;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.test.support.TestUtils;
+import org.apache.wicket.util.file.File;
+import org.junit.Assert;
+import org.junit.runner.RunWith;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import junit.framework.Test;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+
+@RunWith(PowerMockRunner.class)
+public class FeedSpillerUnitTest extends TestCase {
+ private static final int DEFAULT_FRAME_SIZE = 32768;
+ private static final int NUM_FRAMES = 3096;
+ private static final String TEST_DATAVERSE = "testverse";
+ private static final String TEST_FEED = "testeed";
+ private static final String TEST_DATASET = "testset";
+ private static final FilenameFilter SPILL_FILE_FILTER = new FilenameFilter() {
+ @Override
+ public boolean accept(java.io.File dir, String name) {
+ return name.startsWith(TEST_DATAVERSE);
+ }
+ };
+
+ public FeedSpillerUnitTest(String testName) {
+ super(testName);
+ }
+
+ public void removeSpillFiles() throws IOException {
+ File cwd = new File("./");
+ String[] spills = cwd.list(SPILL_FILE_FILTER);
+ for (String fileName : spills) {
+ Files.delete(Paths.get(fileName));
+ }
+ }
+
+ public int countSpillFiles() throws IOException {
+ File cwd = new File("./");
+ return cwd.list(SPILL_FILE_FILTER).length;
+ }
+
+ /**
+ * @return the suite of tests being tested
+ */
+ public static Test suite() {
+ return new TestSuite(FeedSpillerUnitTest.class);
+ }
+ /*
+ * Spiller spills each 1024 frames to a file
+ */
+
+ /*
+ * The following tests:
+ * 1. Test writer only.
+ * Write 1023 frames.
+ * Check only 1 file exist.
+ * Write 1 more frame
+ * Check two files exist.
+ * Insert 1023 more frames.
+ * Check that we still have 2 files.
+ * Write 1 more frame.
+ * Check that we have 3 files.
+ * Check that we have 2048 frames to read.
+ * Close the spiller
+ * Check files were deleted.
+ */
+ @org.junit.Test
+ public void testWriteFixedSizeSpill() {
+ try {
+ removeSpillFiles();
+ IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
+ FrameSpiller spiller = new FrameSpiller(ctx, TEST_DATAVERSE + "_" + TEST_FEED + "_" + TEST_DATASET,
+ new Long(NUM_FRAMES * DEFAULT_FRAME_SIZE));
+ spiller.open();
+ VSizeFrame frame = new VSizeFrame(ctx);
+ spiller.spill(frame.getBuffer());
+ Assert.assertEquals(1, spiller.remaining());
+ Assert.assertEquals(1, countSpillFiles());
+ for (int i = 0; i < 1022; i++) {
+ spiller.spill(frame.getBuffer());
+ }
+ Assert.assertEquals(1023, spiller.remaining());
+ Assert.assertEquals(1, countSpillFiles());
+ spiller.spill(frame.getBuffer());
+ Assert.assertEquals(1024, spiller.remaining());
+ Assert.assertEquals(2, countSpillFiles());
+ for (int i = 0; i < 1023; i++) {
+ spiller.spill(frame.getBuffer());
+ }
+ Assert.assertEquals(2047, spiller.remaining());
+ Assert.assertEquals(2, countSpillFiles());
+ spiller.spill(frame.getBuffer());
+ Assert.assertEquals(2048, spiller.remaining());
+ Assert.assertEquals(3, countSpillFiles());
+ spiller.close();
+ Assert.assertEquals(0, spiller.remaining());
+ Assert.assertEquals(0, countSpillFiles());
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+
+ /*
+ * 2. Test writer and reader
+ * Write 1047 and Read 1042 frames.
+ * Check only 1 file exists.
+ * Check switchToMemory() returns false.
+ * Check remaining() returns 5.
+ * Read the remaining 5 frames.
+ * Check switchToMemory() returns true.
+ * Check that the reader returns null
+ * Close the spiller.
+ * Check files were deleted.
+ */
+ @org.junit.Test
+ public void testWriteReadFixedSizeSpill() {
+ try {
+ removeSpillFiles();
+ IHyracksTaskContext ctx = TestUtils.create(DEFAULT_FRAME_SIZE);
+ FrameSpiller spiller = new FrameSpiller(ctx, TEST_DATAVERSE + "_" + TEST_FEED + "_" + TEST_DATASET,
+ new Long(NUM_FRAMES * DEFAULT_FRAME_SIZE));
+ spiller.open();
+ VSizeFrame frame = new VSizeFrame(ctx);
+ for (int i = 0; i < 1047; i++) {
+ spiller.spill(frame.getBuffer());
+ }
+ for (int i = 0; i < 1042; i++) {
+ spiller.next();
+ }
+ Assert.assertEquals(5, spiller.remaining());
+ Assert.assertEquals(1, countSpillFiles());
+ Assert.assertEquals(false, spiller.switchToMemory());
+ for (int i = 0; i < 4; i++) {
+ spiller.next();
+ }
+ Assert.assertEquals(false, spiller.next() == null);
+ Assert.assertEquals(true, spiller.next() == null);
+ Assert.assertEquals(true, spiller.switchToMemory());
+ Assert.assertEquals(1, countSpillFiles());
+ spiller.close();
+ Assert.assertEquals(0, spiller.remaining());
+ Assert.assertEquals(0, countSpillFiles());
+ Assert.assertEquals(true, spiller.switchToMemory());
+ } catch (Throwable th) {
+ th.printStackTrace();
+ Assert.fail(th.getMessage());
+ }
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 5e2e227..28f8a79 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -21,12 +21,9 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
import org.apache.asterix.common.config.AsterixStorageProperties;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -52,11 +49,8 @@
import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.api.IDataSourceAdapter;
-import org.apache.asterix.external.feed.api.ICentralFeedManager;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
-import org.apache.asterix.external.feed.watch.FeedActivity;
-import org.apache.asterix.external.feed.watch.FeedActivity.FeedActivityDetails;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
import org.apache.asterix.external.operators.ExternalBTreeSearchOperatorDescriptor;
@@ -184,7 +178,6 @@
public class AqlMetadataProvider implements IMetadataProvider<AqlSourceId, String> {
- private static Logger LOGGER = Logger.getLogger(AqlMetadataProvider.class.getName());
private MetadataTransactionContext mdTxnCtx;
private boolean isWriteTransaction;
private final Map<String, String[]> stores;
@@ -194,7 +187,6 @@
private boolean asyncResults;
private ResultSetId resultSetId;
private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
- private final ICentralFeedManager centralFeedManager;
private final Dataverse defaultDataverse;
private JobId jobId;
@@ -219,11 +211,10 @@
return config;
}
- public AqlMetadataProvider(Dataverse defaultDataverse, ICentralFeedManager centralFeedManager) {
+ public AqlMetadataProvider(Dataverse defaultDataverse) {
this.defaultDataverse = defaultDataverse;
this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
- this.centralFeedManager = centralFeedManager;
}
public void setJobId(JobId jobId) {
@@ -361,13 +352,13 @@
// This is a load into dataset operation
LoadableDataSource alds = (LoadableDataSource) dataSource;
List<List<String>> partitioningKeys = alds.getPartitioningKeys();
- boolean isPKAutoGenerated = ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails())
- .isAutogenerated();
+ boolean isPKAutoGenerated =
+ ((InternalDatasetDetails) alds.getTargetDataset().getDatasetDetails()).isAutogenerated();
ARecordType itemType = (ARecordType) alds.getLoadedType();
int pkIndex = 0;
- IAdapterFactory adapterFactory = getConfiguredAdapterFactory(alds.getTargetDataset(),
- alds.getAdapter(), alds.getAdapterProperties(), itemType, isPKAutoGenerated,
- partitioningKeys, null);
+ IAdapterFactory adapterFactory =
+ getConfiguredAdapterFactory(alds.getTargetDataset(), alds.getAdapter(),
+ alds.getAdapterProperties(), itemType, isPKAutoGenerated, partitioningKeys, null);
RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
return buildLoadableDatasetScan(jobSpec, alds, adapterFactory, rDesc, isPKAutoGenerated,
partitioningKeys, itemType, pkIndex);
@@ -382,14 +373,13 @@
}
}
- @SuppressWarnings("rawtypes")
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedCollectRuntime(JobSpecification jobSpec,
FeedDataSource feedDataSource) throws AlgebricksException {
try {
ARecordType feedOutputType = (ARecordType) feedDataSource.getItemType();
- ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
- .getSerializerDeserializer(feedOutputType);
+ ISerializerDeserializer payloadSerde =
+ NonTaggedDataFormat.INSTANCE.getSerdeProvider().getSerializerDeserializer(feedOutputType);
IAType metaType = feedDataSource.getMetaItemType();
List<IAType> pkTypes = feedDataSource.getPkTypes();
ArrayList<ISerializerDeserializer> serdes = new ArrayList<>();
@@ -402,19 +392,19 @@
serdes.add(AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(type));
}
}
- RecordDescriptor feedDesc = new RecordDescriptor(
- serdes.toArray(new ISerializerDeserializer[serdes.size()]));
- FeedPolicyEntity feedPolicy = (FeedPolicyEntity) feedDataSource.getProperties()
- .get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ RecordDescriptor feedDesc =
+ new RecordDescriptor(serdes.toArray(new ISerializerDeserializer[serdes.size()]));
+ FeedPolicyEntity feedPolicy =
+ (FeedPolicyEntity) feedDataSource.getProperties().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
if (feedPolicy == null) {
throw new AlgebricksException("Feed not configured with a policy");
}
feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
FeedConnectionId feedConnectionId = new FeedConnectionId(feedDataSource.getId().getDataverseName(),
feedDataSource.getId().getDatasourceName(), feedDataSource.getTargetDataset());
- FeedCollectOperatorDescriptor feedCollector = new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId,
- feedDataSource.getSourceFeedId(), feedOutputType, feedDesc, feedPolicy.getProperties(),
- feedDataSource.getLocation());
+ FeedCollectOperatorDescriptor feedCollector =
+ new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId, feedDataSource.getSourceFeedId(),
+ feedOutputType, feedDesc, feedPolicy.getProperties(), feedDataSource.getLocation());
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedCollector,
determineLocationConstraint(feedDataSource));
@@ -426,87 +416,14 @@
private AlgebricksAbsolutePartitionConstraint determineLocationConstraint(FeedDataSource feedDataSource)
throws AsterixException {
- String[] locationArray = null;
- String locations = null;;
- switch (feedDataSource.getSourceFeedType()) {
- case PRIMARY:
- switch (feedDataSource.getLocation()) {
- case COMPUTE:
- if (feedDataSource.getFeed().getFeedId().equals(feedDataSource.getSourceFeedId())) {
- locationArray = feedDataSource.getLocations();
- } else {
- Collection<FeedActivity> activities = centralFeedManager.getFeedLoadManager()
- .getFeedActivities();
- Iterator<FeedActivity> it = activities.iterator();
- FeedActivity activity = null;
- while (it.hasNext()) {
- activity = it.next();
- if (activity.getDataverseName().equals(feedDataSource.getSourceFeedId().getDataverse())
- && activity.getFeedName()
- .equals(feedDataSource.getSourceFeedId().getFeedName())) {
- locations = activity.getFeedActivityDetails()
- .get(FeedActivityDetails.COMPUTE_LOCATIONS);
- locationArray = locations.split(",");
- break;
- }
- }
- }
- break;
- case INTAKE:
- locationArray = feedDataSource.getLocations();
- break;
- default:
- throw new AsterixException(
- "Can't subscibe to a FeedRuntime with type: " + feedDataSource.getLocation());
- }
- break;
- case SECONDARY:
- Collection<FeedActivity> activities = centralFeedManager.getFeedLoadManager().getFeedActivities();
- Iterator<FeedActivity> it = activities.iterator();
- FeedActivity activity = null;
- while (it.hasNext()) {
- activity = it.next();
- if (activity.getDataverseName().equals(feedDataSource.getSourceFeedId().getDataverse())
- && activity.getFeedName().equals(feedDataSource.getSourceFeedId().getFeedName())) {
- switch (feedDataSource.getLocation()) {
- case INTAKE:
- locations = activity.getFeedActivityDetails()
- .get(FeedActivityDetails.COLLECT_LOCATIONS);
- break;
- case COMPUTE:
- locations = activity.getFeedActivityDetails()
- .get(FeedActivityDetails.COMPUTE_LOCATIONS);
- break;
- default:
- throw new AsterixException(
- "Can't subscibe to a FeedRuntime with type: " + feedDataSource.getLocation());
- }
- break;
- }
- }
-
- if (locations != null) {
- locationArray = locations.split(",");
- } else {
- String message = "Unable to discover location(s) for source feed data hand-off "
- + feedDataSource.getSourceFeedId();
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(message);
- }
- throw new AsterixException(message);
- }
- break;
- }
- AlgebricksAbsolutePartitionConstraint locationConstraint = new AlgebricksAbsolutePartitionConstraint(
- locationArray);
- return locationConstraint;
+ return new AlgebricksAbsolutePartitionConstraint(feedDataSource.getLocations());
}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(JobSpecification jobSpec,
LoadableDataSource alds, IAdapterFactory adapterFactory, RecordDescriptor rDesc, boolean isPKAutoGenerated,
List<List<String>> primaryKeys, ARecordType recType, int pkIndex) throws AlgebricksException {
- ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, rDesc,
- adapterFactory);
+ ExternalDataScanOperatorDescriptor dataScanner =
+ new ExternalDataScanOperatorDescriptor(jobSpec, rDesc, adapterFactory);
AlgebricksPartitionConstraint constraint;
try {
constraint = adapterFactory.getPartitionConstraint();
@@ -565,13 +482,13 @@
List<List<String>> primaryKeys, ARecordType metaType) throws AlgebricksException {
try {
configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataset.getDataverseName());
- IAdapterFactory adapterFactory = AdapterFactoryProvider.getAdapterFactory(adapterName, configuration,
- itemType, metaType);
+ IAdapterFactory adapterFactory =
+ AdapterFactoryProvider.getAdapterFactory(adapterName, configuration, itemType, metaType);
// check to see if dataset is indexed
- Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
- dataset.getDatasetName(),
- dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
+ Index filesIndex =
+ MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(), dataset.getDatasetName(),
+ dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX));
if (filesIndex != null && filesIndex.getPendingOp() == 0) {
// get files
@@ -599,12 +516,11 @@
throw new AlgebricksException("Can only scan datasets of records.");
}
- @SuppressWarnings("rawtypes")
ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
- ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc,
- adapterFactory);
+ ExternalDataScanOperatorDescriptor dataScanner =
+ new ExternalDataScanOperatorDescriptor(jobSpec, scannerDesc, adapterFactory);
AlgebricksPartitionConstraint constraint;
try {
@@ -663,8 +579,8 @@
ITypeTraits[] typeTraits;
IBinaryComparatorFactory[] comparatorFactories;
- ARecordType itemType = (ARecordType) this.findType(dataset.getItemTypeDataverseName(),
- dataset.getItemTypeName());
+ ARecordType itemType =
+ (ARecordType) this.findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
ITypeTraits[] filterTypeTraits = DatasetUtils.computeFilterTypeTraits(dataset, itemType);
IBinaryComparatorFactory[] filterCmpFactories = DatasetUtils.computeFilterBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
@@ -679,10 +595,10 @@
for (int i = 0; i < numSecondaryKeys; i++) {
bloomFilterKeyFields[i] = i;
}
- Pair<IBinaryComparatorFactory[], ITypeTraits[]> comparatorFactoriesAndTypeTraits = getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(
- secondaryIndex.getIndexType(), secondaryIndex.getKeyFieldNames(),
- secondaryIndex.getKeyFieldTypes(), DatasetUtils.getPartitioningKeys(dataset), itemType,
- dataset.getDatasetType());
+ Pair<IBinaryComparatorFactory[], ITypeTraits[]> comparatorFactoriesAndTypeTraits =
+ getComparatorFactoriesAndTypeTraitsOfSecondaryBTreeIndex(secondaryIndex.getIndexType(),
+ secondaryIndex.getKeyFieldNames(), secondaryIndex.getKeyFieldTypes(),
+ DatasetUtils.getPartitioningKeys(dataset), itemType, dataset.getDatasetType());
comparatorFactories = comparatorFactoriesAndTypeTraits.first;
typeTraits = comparatorFactoriesAndTypeTraits.second;
if (filterTypeTraits != null) {
@@ -739,8 +655,8 @@
: new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
txnSubsystemProvider, ResourceType.LSM_BTREE);
}
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
BTreeSearchOperatorDescriptor btreeSearchOp;
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
@@ -761,13 +677,13 @@
// External dataset <- use the btree with buddy btree->
// Be Careful of Key Start Index ?
int[] buddyBreeFields = new int[] { numSecondaryKeys };
- ExternalBTreeWithBuddyDataflowHelperFactory indexDataflowHelperFactory = new ExternalBTreeWithBuddyDataflowHelperFactory(
- compactionInfo.first, compactionInfo.second,
- new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
- AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
- LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
- getStorageProperties().getBloomFilterFalsePositiveRate(), buddyBreeFields,
- ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp);
+ ExternalBTreeWithBuddyDataflowHelperFactory indexDataflowHelperFactory =
+ new ExternalBTreeWithBuddyDataflowHelperFactory(compactionInfo.first, compactionInfo.second,
+ new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+ AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+ LSMBTreeWithBuddyIOOperationCallbackFactory.INSTANCE,
+ getStorageProperties().getBloomFilterFalsePositiveRate(), buddyBreeFields,
+ ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this), !temp);
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, rtcProvider,
rtcProvider, spPc.first, typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexDataflowHelperFactory, retainInput,
@@ -794,11 +710,11 @@
int i = 0;
for (; i < sidxKeyFieldCount; ++i) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i),
- sidxKeyFieldNames.get(i), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(sidxKeyFieldTypes.get(i), sidxKeyFieldNames.get(i), recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
@@ -818,8 +734,8 @@
} catch (AsterixException e) {
throw new AlgebricksException(e);
}
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
@@ -849,8 +765,8 @@
throw new AlgebricksException("Cannot use " + numSecondaryKeys
+ " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyTypePair =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
IAType keyType = keyTypePair.first;
if (keyType == null) {
throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -858,7 +774,8 @@
int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
boolean isPointMBR = keyType.getTypeTag() == ATypeTag.POINT || keyType.getTypeTag() == ATypeTag.POINT3D;
int numNestedSecondaryKeyFields = numDimensions * 2;
- IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+ IPrimitiveValueProviderFactory[] valueProviderFactories =
+ new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
@@ -874,8 +791,9 @@
ITypeTraits[] typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex,
numNestedSecondaryKeyFields + numPrimaryKeys, typeEnv, context);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = splitProviderAndPartitionConstraintsForDataset(
- dataset.getDataverseName(), dataset.getDatasetName(), indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
+ splitProviderAndPartitionConstraintsForDataset(dataset.getDataverseName(), dataset.getDatasetName(),
+ indexName, temp);
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
dataset, recType, context.getBinaryComparatorFactoryProvider());
@@ -899,10 +817,10 @@
}
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
- ISearchOperationCallbackFactory searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
- : new SecondaryIndexSearchOperationCallbackFactory();
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
+ ISearchOperationCallbackFactory searchCallbackFactory =
+ temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
RTreeSearchOperatorDescriptor rtreeSearchOp;
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
@@ -969,8 +887,8 @@
File outFile = fs.getLocalFile().getFile();
String nodeId = fs.getNodeName();
- SinkWriterRuntimeFactory runtime = new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile,
- getWriterFactory(), inputDesc);
+ SinkWriterRuntimeFactory runtime =
+ new SinkWriterRuntimeFactory(printColumns, printerFactories, outFile, getWriterFactory(), inputDesc);
AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeId });
return new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(runtime, apc);
}
@@ -1104,8 +1022,9 @@
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataSource.getId().getDataverseName(), datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
+ indexName, temp);
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
long numElementsHint = getCardinalityPerPartitionHint(dataset);
@@ -1121,8 +1040,8 @@
// right callback
// (ex. what's the expected behavior when there is an error during
// bulkload?)
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, null,
appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
@@ -1193,8 +1112,9 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataSource.getId().getDataverseName(), datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
+ indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1217,8 +1137,8 @@
: new PrimaryIndexModificationOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -1514,8 +1434,9 @@
IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, dataset.getDatasetDetails().isTemp());
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName,
+ dataset.getDatasetDetails().isTemp());
// Generate Output Record format
ISerializerDeserializer<?>[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
@@ -1584,8 +1505,8 @@
return null;
}
IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
- IScalarEvaluatorFactory filterEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(filterExpr,
- typeEnv, inputSchemas, context);
+ IScalarEvaluatorFactory filterEvalFactory =
+ expressionRuntimeProvider.createEvaluatorFactory(filterExpr, typeEnv, inputSchemas, context);
return new AsterixTupleFilterFactory(filterEvalFactory, context.getBinaryBooleanInspectorFactory());
}
@@ -1665,25 +1586,25 @@
ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i),
- secondaryKeyNames.get(i), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i), secondaryKeyNames.get(i), recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
for (List<String> partitioningKey : partitioningKeys) {
IAType keyType = recType.getSubFieldType(partitioningKey);
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
++i;
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1696,8 +1617,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
ResourceType.LSM_BTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -1836,8 +1757,8 @@
IAType secondaryKeyType = null;
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
secondaryKeyType = keyPairType.first;
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -1885,8 +1806,8 @@
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -1900,8 +1821,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
ResourceType.LSM_INVERTED_INDEX);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory indexDataFlowFactory;
if (!isPartitioned) {
indexDataFlowFactory = new LSMInvertedIndexDataflowHelperFactory(
@@ -1965,8 +1886,8 @@
dataset.getDatasetName(), indexName);
List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
|| spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -2001,10 +1922,11 @@
fieldPermutation[numKeys] = idx;
}
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
- IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
+ IPrimitiveValueProviderFactory[] valueProviderFactories =
+ new IPrimitiveValueProviderFactory[numSecondaryKeys];
for (i = 0; i < numSecondaryKeys; i++) {
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
- .getBinaryComparatorFactory(nestedKeyType, true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
@@ -2018,8 +1940,8 @@
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
dataset, recType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
int[] btreeFields = new int[primaryComparatorFactories.length];
for (int k = 0; k < btreeFields.length; k++) {
btreeFields[k] = k + numSecondaryKeys;
@@ -2050,8 +1972,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, indexOp,
ResourceType.LSM_RTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IBinaryComparatorFactory[] deletedKeyBTreeCompFactories = getMergedComparatorFactories(comparatorFactories,
primaryComparatorFactories);
@@ -2112,8 +2034,8 @@
numElementsHint = Long.parseLong(numElementsHintString);
}
int numPartitions = 0;
- List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName())
- .getNodeNames();
+ List<String> nodeGroup =
+ MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, dataset.getNodeGroupName()).getNodeNames();
for (String nd : nodeGroup) {
numPartitions += AsterixClusterProperties.INSTANCE.getNodePartitionsCount(nd);
}
@@ -2131,8 +2053,8 @@
return StoragePathUtil.splitProviderAndPartitionConstraints(splits);
}
- public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
- String dataverse) {
+ public Pair<IFileSplitProvider, AlgebricksPartitionConstraint>
+ splitProviderAndPartitionConstraintsForDataverse(String dataverse) {
return SplitsAndConstraintsUtil.splitProviderAndPartitionConstraintsForDataverse(dataverse);
}
@@ -2282,8 +2204,8 @@
spPc = metadataProvider.splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
dataset.getDatasetName(),
dataset.getDatasetName().concat(IndexingConstants.EXTERNAL_FILE_INDEX_NAME_SUFFIX), false);
- ISearchOperationCallbackFactory searchOpCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
- : new SecondaryIndexSearchOperationCallbackFactory();
+ ISearchOperationCallbackFactory searchOpCallbackFactory =
+ temp ? NoOpOperationCallbackFactory.INSTANCE : new SecondaryIndexSearchOperationCallbackFactory();
// Create the operator
ExternalLookupOperatorDescriptor op = new ExternalLookupOperatorDescriptor(jobSpec, adapterFactory,
outRecDesc, indexDataflowHelperFactory, retainInput, appContext.getIndexLifecycleManagerProvider(),
@@ -2355,8 +2277,9 @@
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
itemType, context.getBinaryComparatorFactoryProvider());
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataSource.getId().getDataverseName(), datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataSource.getId().getDataverseName(), datasetName,
+ indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -2382,8 +2305,8 @@
LockThenSearchOperationCallbackFactory searchCallbackFactory = new LockThenSearchOperationCallbackFactory(
jobId, datasetId, primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
@@ -2393,16 +2316,16 @@
AsterixLSMTreeUpsertOperatorDescriptor op;
ITypeTraits[] outputTypeTraits = new ITypeTraits[recordDesc.getFieldCount() + 1 + numFilterFields];
- ISerializerDeserializer[] outputSerDes = new ISerializerDeserializer[recordDesc.getFieldCount() + 1
- + numFilterFields];
+ ISerializerDeserializer[] outputSerDes =
+ new ISerializerDeserializer[recordDesc.getFieldCount() + 1 + numFilterFields];
for (int j = 0; j < recordDesc.getFieldCount(); j++) {
outputTypeTraits[j] = recordDesc.getTypeTraits()[j];
outputSerDes[j] = recordDesc.getFields()[j];
}
- outputSerDes[outputSerDes.length - 1 - numFilterFields] = FormatUtils.getDefaultFormat().getSerdeProvider()
- .getSerializerDeserializer(itemType);
- outputTypeTraits[outputTypeTraits.length - 1 - numFilterFields] = FormatUtils.getDefaultFormat()
- .getTypeTraitProvider().getTypeTrait(itemType);
+ outputSerDes[outputSerDes.length - 1 - numFilterFields] =
+ FormatUtils.getDefaultFormat().getSerdeProvider().getSerializerDeserializer(itemType);
+ outputTypeTraits[outputTypeTraits.length - 1 - numFilterFields] =
+ FormatUtils.getDefaultFormat().getTypeTraitProvider().getTypeTrait(itemType);
int fieldIdx = -1;
if (numFilterFields > 0) {
String filterField = DatasetUtils.getFilterField(dataset).get(0);
@@ -2616,8 +2539,8 @@
IAType secondaryKeyType = null;
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
secondaryKeyType = keyPairType.first;
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -2665,8 +2588,8 @@
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -2680,8 +2603,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT,
ResourceType.LSM_INVERTED_INDEX);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory indexDataFlowFactory;
if (!isPartitioned) {
indexDataFlowFactory = new LSMInvertedIndexDataflowHelperFactory(
@@ -2739,8 +2662,8 @@
List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
- secondaryKeyExprs.get(0), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
@@ -2798,10 +2721,11 @@
}
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
- IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
+ IPrimitiveValueProviderFactory[] valueProviderFactories =
+ new IPrimitiveValueProviderFactory[numSecondaryKeys];
for (i = 0; i < numSecondaryKeys; i++) {
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
- .getBinaryComparatorFactory(nestedKeyType, true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(nestedKeyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
}
@@ -2815,8 +2739,8 @@
IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
dataset, recType, context.getBinaryComparatorFactoryProvider());
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
int[] btreeFields = new int[primaryComparatorFactories.length];
for (int k = 0; k < btreeFields.length; k++) {
btreeFields[k] = k + numSecondaryKeys;
@@ -2968,25 +2892,25 @@
ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
for (i = 0; i < secondaryKeys.size(); ++i) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i),
- secondaryKeyNames.get(i), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(i), secondaryKeyNames.get(i), recType);
IAType keyType = keyPairType.first;
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
}
List<List<String>> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
for (List<String> partitioningKey : partitioningKeys) {
IAType keyType = recType.getSubFieldType(partitioningKey);
- comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
- true);
+ comparatorFactories[i] =
+ AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType, true);
typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
++i;
}
IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
- Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForDataset(
- dataverseName, datasetName, indexName, temp);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ splitProviderAndPartitionConstraintsForDataset(dataverseName, datasetName, indexName, temp);
// prepare callback
JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
@@ -3000,8 +2924,8 @@
modificationCallbackPrimaryKeyFields, txnSubsystemProvider, IndexOperation.UPSERT,
ResourceType.LSM_BTREE);
- Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
- .getMergePolicyFactory(dataset, mdTxnCtx);
+ Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo =
+ DatasetUtils.getMergePolicyFactory(dataset, mdTxnCtx);
IIndexDataflowHelperFactory idfh = new LSMBTreeDataflowHelperFactory(
new AsterixVirtualBufferCacheProvider(datasetId), compactionInfo.first, compactionInfo.second,
new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 8561bf5..08a98a4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -70,7 +70,6 @@
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
import org.apache.hyracks.api.constraints.Constraint;
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.constraints.expressions.ConstantExpression;
@@ -87,7 +86,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.common.data.partition.RandomPartitionComputerFactory;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningWithMessageConnectorDescriptor;
@@ -131,8 +129,8 @@
MetadataTransactionContext ctx) throws AsterixException {
FeedPolicyEntity feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, dataverse, policyName);
if (feedPolicy == null) {
- feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME,
- policyName);
+ feedPolicy =
+ MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME, policyName);
if (feedPolicy == null) {
throw new AsterixException("Unknown feed policy" + policyName);
}
@@ -142,7 +140,6 @@
public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
FeedConnectionId feedConnectionId, Map<String, String> feedPolicyProperties) {
-
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Original Job Spec:" + spec);
}
@@ -152,10 +149,11 @@
boolean preProcessingRequired = preProcessingRequired(feedConnectionId);
// copy operators
String operandId = null;
- Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<OperatorDescriptorId, OperatorDescriptorId>();
+ Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID =
+ new HashMap<OperatorDescriptorId, OperatorDescriptorId>();
FeedMetaOperatorDescriptor metaOp = null;
for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
- operandId = FeedRuntimeId.DEFAULT_OPERAND_ID;
+ operandId = FeedRuntimeId.DEFAULT_TARGET_ID;
IOperatorDescriptor opDesc = entry.getValue();
if (opDesc instanceof FeedCollectOperatorDescriptor) {
FeedCollectOperatorDescriptor orig = (FeedCollectOperatorDescriptor) opDesc;
@@ -173,45 +171,33 @@
} else {
FeedRuntimeType runtimeType = null;
boolean enableSubscriptionMode = false;
- boolean createMetaOp = true;
OperatorDescriptorId opId = null;
if (opDesc instanceof AlgebricksMetaOperatorDescriptor) {
- IPushRuntimeFactory runtimeFactory = ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline()
- .getRuntimeFactories()[0];
- if (runtimeFactory instanceof AssignRuntimeFactory) {
- IConnectorDescriptor connectorDesc = spec.getOperatorInputMap().get(opDesc.getOperatorId())
- .get(0);
+ IPushRuntimeFactory[] runtimeFactories =
+ ((AlgebricksMetaOperatorDescriptor) opDesc).getPipeline().getRuntimeFactories();
+ if (runtimeFactories[0] instanceof AssignRuntimeFactory && runtimeFactories.length > 1) {
+ IConnectorDescriptor connectorDesc =
+ spec.getOperatorInputMap().get(opDesc.getOperatorId()).get(0);
IOperatorDescriptor sourceOp = spec.getProducer(connectorDesc);
if (sourceOp instanceof FeedCollectOperatorDescriptor) {
- runtimeType = preProcessingRequired ? FeedRuntimeType.COMPUTE : FeedRuntimeType.OTHER;
+ runtimeType = FeedRuntimeType.COMPUTE;
enableSubscriptionMode = preProcessingRequired;
- } else {
- runtimeType = FeedRuntimeType.OTHER;
+ metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
+ feedPolicyProperties, runtimeType, enableSubscriptionMode, operandId);
+ opId = metaOp.getOperatorId();
}
- } else if (runtimeFactory instanceof EmptyTupleSourceRuntimeFactory) {
- runtimeType = FeedRuntimeType.ETS;
- } else {
- runtimeType = FeedRuntimeType.OTHER;
- }
- } else {
- if (opDesc instanceof AbstractSingleActivityOperatorDescriptor) {
- runtimeType = FeedRuntimeType.OTHER;
- } else {
- opId = altered.createOperatorDescriptorId(opDesc);
- createMetaOp = false;
}
}
- if (createMetaOp) {
- metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc, feedPolicyProperties,
- runtimeType, enableSubscriptionMode, operandId);
- opId = metaOp.getOperatorId();
+ if (opId == null) {
+ opId = altered.createOperatorDescriptorId(opDesc);
}
oldNewOID.put(opDesc.getOperatorId(), opId);
}
}
// copy connectors
- Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorMapping = new HashMap<ConnectorDescriptorId, ConnectorDescriptorId>();
+ Map<ConnectorDescriptorId, ConnectorDescriptorId> connectorMapping =
+ new HashMap<ConnectorDescriptorId, ConnectorDescriptorId>();
for (Entry<ConnectorDescriptorId, IConnectorDescriptor> entry : spec.getConnectorMap().entrySet()) {
IConnectorDescriptor connDesc = entry.getValue();
ConnectorDescriptorId newConnId;
@@ -233,16 +219,17 @@
Pair<IOperatorDescriptor, Integer> leftOp = entry.getValue().getLeft();
Pair<IOperatorDescriptor, Integer> rightOp = entry.getValue().getRight();
- IOperatorDescriptor leftOpDesc = altered.getOperatorMap()
- .get(oldNewOID.get(leftOp.getLeft().getOperatorId()));
- IOperatorDescriptor rightOpDesc = altered.getOperatorMap()
- .get(oldNewOID.get(rightOp.getLeft().getOperatorId()));
+ IOperatorDescriptor leftOpDesc =
+ altered.getOperatorMap().get(oldNewOID.get(leftOp.getLeft().getOperatorId()));
+ IOperatorDescriptor rightOpDesc =
+ altered.getOperatorMap().get(oldNewOID.get(rightOp.getLeft().getOperatorId()));
altered.connect(connDesc, leftOpDesc, leftOp.getRight(), rightOpDesc, rightOp.getRight());
}
// prepare for setting partition constraints
- Map<OperatorDescriptorId, List<LocationConstraint>> operatorLocations = new HashMap<OperatorDescriptorId, List<LocationConstraint>>();
+ Map<OperatorDescriptorId, List<LocationConstraint>> operatorLocations =
+ new HashMap<OperatorDescriptorId, List<LocationConstraint>>();
Map<OperatorDescriptorId, Integer> operatorCounts = new HashMap<OperatorDescriptorId, Integer>();
for (Constraint constraint : spec.getUserConstraints()) {
@@ -339,8 +326,8 @@
private static IOperatorDescriptor alterJobSpecForComputeCardinality(JobSpecification spec, int requiredCardinality)
throws AsterixException {
Map<ConnectorDescriptorId, IConnectorDescriptor> connectors = spec.getConnectorMap();
- Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap = spec
- .getConnectorOperatorMap();
+ Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap =
+ spec.getConnectorOperatorMap();
IOperatorDescriptor sourceOp = null;
IOperatorDescriptor targetOp = null;
@@ -467,9 +454,9 @@
}
@SuppressWarnings("rawtypes")
- public static Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> getPrimaryFeedFactoryAndOutput(
- Feed feed, FeedPolicyAccessor policyAccessor, MetadataTransactionContext mdTxnCtx)
- throws AlgebricksException {
+ public static Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType>
+ getPrimaryFeedFactoryAndOutput(Feed feed, FeedPolicyAccessor policyAccessor,
+ MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
// This method needs to be re-visited
String adapterName = null;
DatasourceAdapter adapterEntity = null;
@@ -503,8 +490,8 @@
case EXTERNAL:
String[] anameComponents = adapterName.split("#");
String libraryName = anameComponents[0];
- ClassLoader cl = ExternalLibraryManager.getLibraryClassLoader(feed.getDataverseName(),
- libraryName);
+ ClassLoader cl =
+ ExternalLibraryManager.getLibraryClassLoader(feed.getDataverseName(), libraryName);
adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
break;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
index 5ad1b82..d94f933 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameWriter.java
@@ -23,7 +23,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
- * {@link IFrameWriter} is the interface implemented by a stream consumer. An {@link IFrameWriter} could be in one of the following states:
+ * {@link IFrameWriter} is the interface implemented by a stream consumer. An {@link IFrameWriter} could be in one of
+ * the following states:
* <ul>
* <li>INITIAL</li>
* <li>OPENED</li>
@@ -32,20 +33,28 @@
* </ul>
* A producer follows the following protocol when using an {@link IFrameWriter}.
* Initially, the {@link IFrameWriter} is in the INITIAL state.
- * The first valid call to an {@link IFrameWriter} is always the {@link IFrameWriter#open()}. This call provides the opportunity for the {@link IFrameWriter} implementation to allocate any resources for its
+ * The first valid call to an {@link IFrameWriter} is always the {@link IFrameWriter#open()}. This call provides the
+ * opportunity for the {@link IFrameWriter} implementation to allocate any resources for its
* processing. Once open() is called, no matter successfully or not, the {@link IFrameWriter} is in the OPENED
* state.
* While the {@link IFrameWriter} is in the OPENED state, the producer can call one of:
* <ul>
- * <li>{@link IFrameWriter#close()} to give up any resources owned by the {@link IFrameWriter} and enter the CLOSED state.</li>
- * <li>{@link IFrameWriter#nextFrame(ByteBuffer)} to provide data to the {@link IFrameWriter}. The call returns normally on success and the {@link IFrameWriter} remains in the OPENED state. On failure, the call throws a {@link HyracksDataException}, the {@link IFrameWriter} remains in the OPENED state.</li>
- * <li>{@link IFrameWriter#flush()} to push tuples that are ready in the output frame. The {@link IFrameWriter} remains in the OPENED state.</li>
- * <li>{@link IFrameWriter#fail()} to indicate that stream is to be aborted. The {@link IFrameWriter} enters the FAILED state.</li>
+ * <li>{@link IFrameWriter#close()} to give up any resources owned by the {@link IFrameWriter} and enter the CLOSED
+ * state.</li>
+ * <li>{@link IFrameWriter#nextFrame(ByteBuffer)} to provide data to the {@link IFrameWriter}. The call returns normally
+ * on success and the {@link IFrameWriter} remains in the OPENED state. On failure, the call throws a
+ * {@link HyracksDataException}, the {@link IFrameWriter} remains in the OPENED state.</li>
+ * <li>{@link IFrameWriter#flush()} to push tuples that are ready in the output frame. The {@link IFrameWriter} remains
+ * in the OPENED state.</li>
+ * <li>{@link IFrameWriter#fail()} to indicate that stream is to be aborted. The {@link IFrameWriter} enters the FAILED
+ * state.</li>
* </ul>
- * In the FAILED state, the only call allowed is the {@link IFrameWriter#close()} to move the {@link IFrameWriter} into the CLOSED
+ * In the FAILED state, the only call allowed is the {@link IFrameWriter#close()} to move the {@link IFrameWriter} into
+ * the CLOSED
* state and give up all resources.
* No calls are allowed when the {@link IFrameWriter} is in the CLOSED state.
- * Note: If the call to {@link IFrameWriter#open()} failed, the {@link IFrameWriter#close()} must still be called by the producer.
+ * Note: If the call to {@link IFrameWriter#open()} failed, the {@link IFrameWriter#close()} must still be called by the
+ * producer.
*/
public interface IFrameWriter {
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
index d43680a..0db27fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
@@ -25,19 +25,14 @@
public interface IHyracksFrameMgrContext {
- int getInitialFrameSize();
+ public int getInitialFrameSize();
- ByteBuffer allocateFrame() throws HyracksDataException;
+ public ByteBuffer allocateFrame() throws HyracksDataException;
- ByteBuffer allocateFrame(int bytes) throws HyracksDataException;
+ public ByteBuffer allocateFrame(int bytes) throws HyracksDataException;
- ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+ public ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
throws HyracksDataException;
- /**
- * The caller should call this method to return the pre-allocated frames.
- *
- * @param bytes
- */
- void deallocateFrames(int bytes);
+ public void deallocateFrames(int bytes);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
index b0f6269..3781489 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java
@@ -43,6 +43,8 @@
public void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) throws Exception;
+ public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymendId) throws Exception;
+
public void setSharedObject(Object sharedObject);
public Object getSharedObject();
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 5eee4b3..134154c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -384,6 +384,11 @@
}
@Override
+ public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymentId) throws Exception {
+ this.ncs.sendApplicationMessageToCC(message, deploymentId);
+ }
+
+ @Override
public void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) throws Exception {
this.ncs.sendApplicationMessageToCC(JavaSerializationUtils.serialize(message), deploymentId);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
index 6c236a6..345c506 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/MessagingFrameTupleAppender.java
@@ -52,19 +52,16 @@
@Override
public void write(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
- if (tupleCount > 0) {
- appendMessage();
- getBuffer().clear();
- outWriter.nextFrame(getBuffer());
- if (clearFrame) {
- frame.reset();
- reset(getBuffer(), true);
- }
+ appendMessage((ByteBuffer) ctx.getSharedObject());
+ getBuffer().clear();
+ outWriter.nextFrame(getBuffer());
+ if (clearFrame) {
+ frame.reset();
+ reset(getBuffer(), true);
}
}
- public void appendMessage() {
- ByteBuffer message = (ByteBuffer) ctx.getSharedObject();
+ private void appendMessage(ByteBuffer message) {
System.arraycopy(message.array(), message.position(), array, tupleDataEndOffset, message.limit());
tupleDataEndOffset += message.limit();
IntSerDeUtils.putInt(getBuffer().array(),
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index f84c3e4..c047567 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -103,14 +103,13 @@
isOpen[i] = true;
pWriters[i].open();
}
+ if (!allocatedFrame) {
+ allocateFrames();
+ }
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (!allocatedFrame) {
- allocateFrames();
- allocatedFrame = true;
- }
tupleAccessor.reset(buffer);
int tupleCount = tupleAccessor.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
@@ -123,6 +122,7 @@
for (int i = 0; i < appenders.length; ++i) {
appenders[i].reset(new VSizeFrame(ctx), true);
}
+ allocatedFrame = true;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index a3b67ca..98bd860 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -134,7 +134,12 @@
}
@Override
- public void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) throws Exception {
+ public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymentId) {
+
+ }
+
+ @Override
+ public void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) {
}
@Override