001/* 002 * Copyright (c) 2007-2016 Concurrent, Inc. All Rights Reserved. 003 * 004 * Project and contact information: http://www.cascading.org/ 005 * 006 * This file is part of the Cascading project. 007 * 008 * Licensed under the Apache License, Version 2.0 (the "License"); 009 * you may not use this file except in compliance with the License. 010 * You may obtain a copy of the License at 011 * 012 * http://www.apache.org/licenses/LICENSE-2.0 013 * 014 * Unless required by applicable law or agreed to in writing, software 015 * distributed under the License is distributed on an "AS IS" BASIS, 016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 017 * See the License for the specific language governing permissions and 018 * limitations under the License. 019 */ 020 021package cascading.flow.planner.rule; 022 023import java.util.Arrays; 024import java.util.Collection; 025import java.util.Collections; 026import java.util.Comparator; 027import java.util.LinkedList; 028 029/** 030 * Class RuleRegistrySet manages the set of {@link cascading.flow.planner.rule.RuleRegistry} instances that should be 031 * applied via the {@link cascading.flow.planner.FlowPlanner} to the given assembly. 032 * <p/> 033 * The current RuleRegistrySet can be reached via {@link cascading.flow.FlowConnector#getRuleRegistrySet()}, or set with 034 * the appropriate {@link cascading.flow.FlowConnector} constructor sub-class. 035 * <p/> 036 * RuleRegistrySet configuration is mutable to address different situations. 037 * <p/> 038 * During planner execution, all RuleRegistry instances are applied simultaneously within individual threads. 039 * <p/> 040 * If the planner duration exceeds {@link #getPlannerTimeoutSec()} (with default 041 * {@link cascading.flow.planner.rule.RuleSetExec#DEFAULT_TIMEOUT}) any incomplete planner executions 042 * will be cancelled. 043 * <p/> 044 * If no planner executions complete successfully within the timeout period, an exception will be thrown. 045 * <p/> 046 * If there are multiple successful completions, the default cost comparator, 047 * {@link cascading.flow.planner.rule.RuleSetExec#DEFAULT_PLAN_COMPARATOR}, will be applied to find the lower 048 * cost plan. Use {@link #setPlanComparator(java.util.Comparator)} to override. 049 * <p/> 050 * If all plans have equivalent costs, the plan corresponding to the first most RuleRegistry, as given to the 051 * RuleRegistrySet, will be selected. 052 * <p/> 053 * If {@link #setSelect(cascading.flow.planner.rule.RuleRegistrySet.Select)} is set to 054 * {@link cascading.flow.planner.rule.RuleRegistrySet.Select#FIRST}, the first RuleRegistry to complete will be used 055 * regardless of cost ordering provided by the plan Comparator. All remaining running planner executions will be 056 * cancelled. 057 * <p/> 058 * If {@link #isIgnoreFailed()} is {@code false}, if any planner execution is times out or fails, an exception will be 059 * thrown. 060 */ 061public class RuleRegistrySet 062 { 063 public enum Select 064 { 065 FIRST, 066 COMPARED 067 } 068 069 int plannerTimeoutSec = RuleSetExec.DEFAULT_TIMEOUT; 070 boolean ignoreFailed = true; 071 Select select = Select.COMPARED; 072 Comparator<RuleResult> planComparator = RuleSetExec.DEFAULT_PLAN_COMPARATOR; 073 074 LinkedList<RuleRegistry> ruleRegistries = new LinkedList<>(); // preserve order, no duplicates 075 076 /** 077 * Instantiates a new Rule Registry Set. 078 * 079 * @param ruleRegistries the rule registries 080 */ 081 public RuleRegistrySet( RuleRegistry... ruleRegistries ) 082 { 083 this( Arrays.asList( ruleRegistries ) ); 084 } 085 086 /** 087 * Instantiates a new Rule Registry Set. 088 * 089 * @param ruleRegistries the rule registries 090 */ 091 public RuleRegistrySet( Collection<RuleRegistry> ruleRegistries ) 092 { 093 this.ruleRegistries.addAll( ruleRegistries ); 094 095 for( RuleRegistry ruleRegistry : ruleRegistries ) 096 { 097 if( Collections.frequency( this.ruleRegistries, ruleRegistry ) > 1 ) 098 throw new IllegalArgumentException( "may not include duplicate registries" ); 099 } 100 } 101 102 /** 103 * Gets planner timeout, in seconds. 104 * 105 * @return the planner timeout sec 106 */ 107 public int getPlannerTimeoutSec() 108 { 109 return plannerTimeoutSec; 110 } 111 112 /** 113 * Sets planner timeout, in seconds. 114 * <p/> 115 * Extremely large assemblies may take longer than the default timeout. This allows for increasing 116 * that timeout period when necessary. 117 * 118 * @param plannerTimeoutSec the planner timeout sec 119 */ 120 public void setPlannerTimeoutSec( int plannerTimeoutSec ) 121 { 122 this.plannerTimeoutSec = plannerTimeoutSec; 123 } 124 125 /** 126 * When true, failures during planning will be ignored unless no registries are successful. When false, 127 * any failure will cause the planner to fail. 128 * 129 * @return type boolean 130 */ 131 public boolean isIgnoreFailed() 132 { 133 return ignoreFailed; 134 } 135 136 /** 137 * Sets ignore failed setting. The default is {@code true}. 138 * <p/> 139 * When {@code true}, failures during planning will be ignored unless no registries are successful. When {@code false}, 140 * any failure will cause the planner to fail. 141 * 142 * @param ignoreFailed {@code true} if any failures should be ignored, the default 143 */ 144 public void setIgnoreFailed( boolean ignoreFailed ) 145 { 146 this.ignoreFailed = ignoreFailed; 147 } 148 149 /** 150 * Returns the result selection type. 151 * 152 * @return the selection type 153 */ 154 public Select getSelect() 155 { 156 return select; 157 } 158 159 /** 160 * Sets the result selection type. 161 * 162 * @param select the selection type 163 */ 164 public void setSelect( Select select ) 165 { 166 if( select == null ) 167 throw new IllegalArgumentException( "select may not be null" ); 168 169 this.select = select; 170 } 171 172 /** 173 * Returns planner comparator. 174 * 175 * @return the planner result comparator 176 */ 177 public Comparator<RuleResult> getPlanComparator() 178 { 179 return planComparator; 180 } 181 182 /** 183 * Sets planner result comparator, used only if {@link #getSelect()} is {@link Select#COMPARED}. 184 * 185 * @param planComparator the plan comparator 186 */ 187 public void setPlanComparator( Comparator<RuleResult> planComparator ) 188 { 189 if( planComparator == null ) 190 throw new IllegalArgumentException( "planComparator may not be null" ); 191 192 this.planComparator = planComparator; 193 } 194 195 public int size() 196 { 197 return ruleRegistries.size(); 198 } 199 200 public RuleRegistry findRegistryWith( String ruleName ) 201 { 202 for( RuleRegistry ruleRegistry : ruleRegistries ) 203 { 204 if( ruleRegistry.hasRule( ruleName ) ) 205 return ruleRegistry; 206 } 207 208 return null; 209 } 210 211 protected int indexOf( RuleRegistry ruleRegistry ) 212 { 213 return ruleRegistries.indexOf( ruleRegistry ); 214 } 215 }